diff --git common/src/java/org/apache/hadoop/hive/conf/HiveConf.java common/src/java/org/apache/hadoop/hive/conf/HiveConf.java index 388a604..caad75f 100644 --- common/src/java/org/apache/hadoop/hive/conf/HiveConf.java +++ common/src/java/org/apache/hadoop/hive/conf/HiveConf.java @@ -18,26 +18,6 @@ package org.apache.hadoop.hive.conf; -import java.io.ByteArrayOutputStream; -import java.io.File; -import java.io.IOException; -import java.io.InputStream; -import java.io.PrintStream; -import java.net.URL; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.Iterator; -import java.util.LinkedHashSet; -import java.util.List; -import java.util.Map; -import java.util.Map.Entry; -import java.util.Properties; -import java.util.Set; -import java.util.regex.Matcher; -import java.util.regex.Pattern; - -import javax.security.auth.login.LoginException; - import org.apache.commons.lang.StringUtils; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -47,6 +27,14 @@ import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.util.Shell; +import javax.security.auth.login.LoginException; +import java.io.*; +import java.net.URL; +import java.util.*; +import java.util.Map.Entry; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + /** * Hive Configuration. */ @@ -134,7 +122,12 @@ HiveConf.ConfVars.METASTORE_PARTITION_NAME_WHITELIST_PATTERN, HiveConf.ConfVars.METASTORE_DISALLOW_INCOMPATIBLE_COL_TYPE_CHANGES, HiveConf.ConfVars.USERS_IN_ADMIN_ROLE, - HiveConf.ConfVars.HIVE_AUTHORIZATION_MANAGER + HiveConf.ConfVars.HIVE_AUTHORIZATION_MANAGER, + HiveConf.ConfVars.HIVE_TXN_MANAGER, + HiveConf.ConfVars.HIVE_TXN_JDBC_DRIVER, + HiveConf.ConfVars.HIVE_TXN_JDBC_CONNECT_STRING, + HiveConf.ConfVars.HIVE_TXN_TIMEOUT, + HiveConf.ConfVars.HIVE_TXN_MAX_OPEN_BATCH, }; /** @@ -703,6 +696,48 @@ HIVE_ZOOKEEPER_NAMESPACE("hive.zookeeper.namespace", "hive_zookeeper_namespace"), HIVE_ZOOKEEPER_CLEAN_EXTRA_NODES("hive.zookeeper.clean.extra.nodes", false), + // Transactions + HIVE_TXN_MANAGER("hive.txn.manager", + "org.apache.hadoop.hive.ql.lockmgr.DummyTxnManager"), + HIVE_TXN_JDBC_DRIVER("hive.txn.driver", ""), + HIVE_TXN_JDBC_CONNECT_STRING("hive.txn.connection.string", ""), + // time after which transactions are declared aborted if the client has + // not sent a heartbeat, in seconds. + HIVE_TXN_TIMEOUT("hive.txn.timeout", 300), + + // Maximum number of transactions that can be fetched in one call to + // open_txns(). + // Increasing this will decrease the number of delta files created when + // streaming data into Hive. But it will also increase the number of + // open transactions at any given time, possibly impacting read + // performance. + HIVE_TXN_MAX_OPEN_BATCH("hive.txn.max.open.batch", 1000), + + // Whether to run the compactor's initiator thread in this metastore instance or not. + HIVE_COMPACTOR_INITIATOR_ON("hive.compactor.initiator.on", false), + + // Number of compactor worker threads to run on this metastore instance. + HIVE_COMPACTOR_WORKER_THREADS("hive.compactor.worker.threads", 0), + + // Time, in seconds, before a given compaction in working state is declared a failure and + // returned to the initiated state. + HIVE_COMPACTOR_WORKER_TIMEOUT("hive.compactor.worker.timeout", 86400L), + + // Time in seconds between checks to see if any partitions need compacted. This should be + // kept high because each check for compaction requires many calls against the NameNode. + HIVE_COMPACTOR_CHECK_INTERVAL("hive.compactor.check.interval", 300L), + + // Number of delta files that must exist in a directory before the compactor will attempt a + // minor compaction. + HIVE_COMPACTOR_DELTA_NUM_THRESHOLD("hive.compactor.delta.num.threshold", 10), + + // Percentage (by size) of base that deltas can be before major compaction is initiated. + HIVE_COMPACTOR_DELTA_PCT_THRESHOLD("hive.compactor.delta.pct.threshold", 0.1f), + + // Number of aborted transactions involving a particular table or partition before major + // compaction is initiated. + HIVE_COMPACTOR_ABORTEDTXN_THRESHOLD("hive.compactor.abortedtxn.threshold", 1000), + // For HBase storage handler HIVE_HBASE_WAL_ENABLED("hive.hbase.wal.enabled", true), diff --git itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStoreTxns.java itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStoreTxns.java new file mode 100644 index 0000000..76b9c76 --- /dev/null +++ itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStoreTxns.java @@ -0,0 +1,242 @@ +/** + * 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. + */ +package org.apache.hadoop.hive.metastore; + +import junit.framework.Assert; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.metastore.HiveMetaStoreClient; +import org.apache.hadoop.hive.metastore.IMetaStoreClient; +import org.apache.hadoop.hive.metastore.LockComponentBuilder; +import org.apache.hadoop.hive.metastore.LockRequestBuilder; +import org.apache.hadoop.hive.metastore.api.GetOpenTxnsResponse; +import org.apache.hadoop.hive.metastore.api.LockResponse; +import org.apache.hadoop.hive.metastore.api.LockState; +import org.apache.hadoop.hive.metastore.txn.TxnDbUtil; +import org.apache.log4j.Level; +import org.apache.log4j.LogManager; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import java.util.HashSet; +import java.util.List; +import java.util.Set; + +/** + * Unit tests for {@link org.apache.hadoop.hive.metastore.HiveMetaStoreClient}. For now this just has + * transaction and locking tests. The goal here is not to test all + * functionality possible through the interface, as all permutations of DB + * operations should be tested in the appropriate DB handler classes. The + * goal is to test that we can properly pass the messages through the thrift + * service. + * + * This is in the ql directory rather than the metastore directory because it + * required the hive-exec jar, and hive-exec jar already depends on + * hive-metastore jar, thus I can't make hive-metastore depend on hive-exec. + */ +public class TestHiveMetaStoreTxns { + + private HiveConf conf = new HiveConf(); + private IMetaStoreClient client; + + public TestHiveMetaStoreTxns() throws Exception { + TxnDbUtil.setConfValues(conf); + LogManager.getRootLogger().setLevel(Level.DEBUG); + tearDown(); + } + + @Test + public void testTxns() throws Exception { + List tids = client.openTxns("me", 3).getTxn_ids(); + Assert.assertEquals(1L, (long) tids.get(0)); + Assert.assertEquals(2L, (long) tids.get(1)); + Assert.assertEquals(3L, (long) tids.get(2)); + client.rollbackTxn(1); + client.commitTxn(2); + IMetaStoreClient.ValidTxnList validTxns = client.getValidTxns(); + Assert.assertFalse(validTxns.isTxnCommitted(1)); + Assert.assertTrue(validTxns.isTxnCommitted(2)); + Assert.assertFalse(validTxns.isTxnCommitted(3)); + Assert.assertFalse(validTxns.isTxnCommitted(4)); + } + + @Test + public void testTxnRange() throws Exception { + IMetaStoreClient.ValidTxnList validTxns = client.getValidTxns(); + Assert.assertEquals(IMetaStoreClient.ValidTxnList.RangeResponse.NONE, + validTxns.isTxnRangeCommitted(1L, 3L)); + + List tids = client.openTxns("me", 5).getTxn_ids(); + client.rollbackTxn(1L); + client.commitTxn(2L); + client.commitTxn(3L); + client.commitTxn(4L); + validTxns = client.getValidTxns(); + Assert.assertEquals(IMetaStoreClient.ValidTxnList.RangeResponse.ALL, + validTxns.isTxnRangeCommitted(2L, 2L)); + Assert.assertEquals(IMetaStoreClient.ValidTxnList.RangeResponse.ALL, + validTxns.isTxnRangeCommitted(2L, 3L)); + Assert.assertEquals(IMetaStoreClient.ValidTxnList.RangeResponse.ALL, + validTxns.isTxnRangeCommitted(2L, 4L)); + Assert.assertEquals(IMetaStoreClient.ValidTxnList.RangeResponse.ALL, + validTxns.isTxnRangeCommitted(3L, 4L)); + + Assert.assertEquals(IMetaStoreClient.ValidTxnList.RangeResponse.SOME, + validTxns.isTxnRangeCommitted(1L, 4L)); + Assert.assertEquals(IMetaStoreClient.ValidTxnList.RangeResponse.SOME, + validTxns.isTxnRangeCommitted(2L, 5L)); + Assert.assertEquals(IMetaStoreClient.ValidTxnList.RangeResponse.SOME, + validTxns.isTxnRangeCommitted(1L, 2L)); + Assert.assertEquals(IMetaStoreClient.ValidTxnList.RangeResponse.SOME, + validTxns.isTxnRangeCommitted(4L, 5L)); + + Assert.assertEquals(IMetaStoreClient.ValidTxnList.RangeResponse.NONE, + validTxns.isTxnRangeCommitted(1L, 1L)); + Assert.assertEquals(IMetaStoreClient.ValidTxnList.RangeResponse.NONE, + validTxns.isTxnRangeCommitted(5L, 10L)); + + } + + @Test + public void testLocks() throws Exception { + LockRequestBuilder rqstBuilder = new LockRequestBuilder(); + rqstBuilder.addLockComponent(new LockComponentBuilder() + .setDbName("mydb") + .setTableName("mytable") + .setPartitionName("mypartition") + .setExclusive() + .build()); + rqstBuilder.addLockComponent(new LockComponentBuilder() + .setDbName("mydb") + .setTableName("yourtable") + .setSemiShared() + .build()); + rqstBuilder.addLockComponent(new LockComponentBuilder() + .setDbName("yourdb") + .setShared() + .build()); + rqstBuilder.setUser("fred"); + + LockResponse res = client.lock(rqstBuilder.build()); + Assert.assertEquals(1L, res.getLockid()); + Assert.assertEquals(LockState.ACQUIRED, res.getState()); + + res = client.checkLock(1); + Assert.assertEquals(1L, res.getLockid()); + Assert.assertEquals(LockState.ACQUIRED, res.getState()); + + client.heartbeat(0, 1); + + client.unlock(1); + } + + @Test + public void testLocksWithTxn() throws Exception { + long txnid = client.openTxn("me"); + + LockRequestBuilder rqstBuilder = new LockRequestBuilder(); + rqstBuilder.setTransactionId(txnid) + .addLockComponent(new LockComponentBuilder() + .setDbName("mydb") + .setTableName("mytable") + .setPartitionName("mypartition") + .setExclusive() + .build()) + .addLockComponent(new LockComponentBuilder() + .setDbName("mydb") + .setTableName("yourtable") + .setSemiShared() + .build()) + .addLockComponent(new LockComponentBuilder() + .setDbName("yourdb") + .setShared() + .build()) + .setUser("fred"); + + LockResponse res = client.lock(rqstBuilder.build()); + Assert.assertEquals(1L, res.getLockid()); + Assert.assertEquals(LockState.ACQUIRED, res.getState()); + + res = client.checkLock(1); + Assert.assertEquals(1L, res.getLockid()); + Assert.assertEquals(LockState.ACQUIRED, res.getState()); + + client.heartbeat(txnid, 1); + + client.commitTxn(txnid); + } + + @Test + public void stringifyValidTxns() throws Exception { + // Test with just high water mark + GetOpenTxnsResponse rsp = new GetOpenTxnsResponse(); + rsp.setTxn_high_water_mark(1); + IMetaStoreClient.ValidTxnList validTxns = new HiveMetaStoreClient.ValidTxnListImpl(rsp); + String asString = validTxns.toString(); + Assert.assertEquals("1:", asString); + validTxns = new HiveMetaStoreClient.ValidTxnListImpl(); + validTxns.fromString(asString); + Assert.assertEquals(1, validTxns.getOpenTxns().getTxn_high_water_mark()); + Assert.assertNotNull(validTxns.getOpenTxns().getOpen_txns()); + Assert.assertEquals(0, validTxns.getOpenTxns().getOpen_txnsSize()); + asString = validTxns.toString(); + Assert.assertEquals("1:", asString); + validTxns = new HiveMetaStoreClient.ValidTxnListImpl(); + validTxns.fromString(asString); + Assert.assertEquals(1, validTxns.getOpenTxns().getTxn_high_water_mark()); + Assert.assertNotNull(validTxns.getOpenTxns().getOpen_txns()); + Assert.assertEquals(0, validTxns.getOpenTxns().getOpen_txnsSize()); + + // Test with open transactions + rsp = new GetOpenTxnsResponse(); + rsp.setTxn_high_water_mark(10); + Set openTxns = new HashSet(2); + openTxns.add(3L); + openTxns.add(5L); + rsp.setOpen_txns(openTxns); + validTxns = new HiveMetaStoreClient.ValidTxnListImpl(rsp); + asString = validTxns.toString(); + if (!asString.equals("10:3:5") && !asString.equals("10:5:3")) { + Assert.fail("Unexpected string value " + asString); + } + validTxns = new HiveMetaStoreClient.ValidTxnListImpl(); + validTxns.fromString(asString); + Assert.assertEquals(10, validTxns.getOpenTxns().getTxn_high_water_mark()); + Assert.assertNotNull(validTxns.getOpenTxns().getOpen_txns()); + Assert.assertEquals(2, validTxns.getOpenTxns().getOpen_txnsSize()); + boolean sawThree = false, sawFive = false; + for (long tid : validTxns.getOpenTxns().getOpen_txns()) { + if (tid == 3) sawThree = true; + else if (tid == 5) sawFive = true; + else Assert.fail("Unexpected value " + tid); + } + Assert.assertTrue(sawThree); + Assert.assertTrue(sawFive); + } + + @Before + public void setUp() throws Exception { + TxnDbUtil.prepDb(); + client = new HiveMetaStoreClient(conf); + } + + @After + public void tearDown() throws Exception { + TxnDbUtil.cleanDb(); + } +} diff --git metastore/if/hive_metastore.thrift metastore/if/hive_metastore.thrift index 58b2357..c400cdf 100755 --- metastore/if/hive_metastore.thrift +++ metastore/if/hive_metastore.thrift @@ -70,6 +70,37 @@ 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, +} + struct HiveObjectRef{ 1: HiveObjectType objectType, 2: string dbName, @@ -383,6 +414,122 @@ struct Function { 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 +} + +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 set open_txns, +} + +struct OpenTxnRequest { + 1: required i32 num_txns, + 2: required string user, + 3: required string hostname, +} + +struct OpenTxnsResponse { + 1: required list txn_ids, +} + +struct AbortTxnRequest { + 1: required i64 txnid, +} + +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, +} + +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 +} + +struct LockResponse { + 1: required i64 lockid, + 2: required LockState state, +} + +struct CheckLockRequest { + 1: required i64 lockid, +} + +struct UnlockRequest { + 1: required i64 lockid, +} + +struct ShowLocksRequest { +} + +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, +} + +struct ShowLocksResponse { + 1: list locks, +} + +struct HeartbeatRequest { + 1: optional i64 lockid, + 2: optional i64 txnid +} + +struct CompactionRequest { + 1: required string dbname, + 2: required string tablename, + 3: optional string partitionname, + 4: required CompactionType type, + 5: optional string runas, +} + +struct ShowCompactRequest { +} + +struct ShowCompactResponseElement { + 1: required string dbname, + 2: required string tablename, + 3: required string partitionname, + 4: required CompactionType type, + 5: required string state, + 6: required string workerid, + 7: required i64 start, + 8: required string runAs, +} + +struct ShowCompactResponse { + 1: required list compacts, +} + exception MetaException { 1: string message } @@ -431,6 +578,23 @@ 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. */ @@ -776,6 +940,23 @@ service ThriftHiveMetastore extends fb303.FacebookService // method to cancel delegation token obtained from metastore server void cancel_delegation_token(1:string token_str_form) throws (1:MetaException o1) + + // 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 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) + void compact(1:CompactionRequest rqst) + ShowCompactResponse show_compact(1:ShowCompactRequest rqst) } // * Note about the DDL_TIME: When creating or altering a table or a partition, diff --git metastore/pom.xml metastore/pom.xml index 5f7b4ad..98878eb 100644 --- metastore/pom.xml +++ metastore/pom.xml @@ -54,7 +54,6 @@ com.jolbox bonecp ${bonecp.version} - runtime commons-cli diff --git metastore/scripts/upgrade/derby/hive-schema-0.13.0.derby.sql metastore/scripts/upgrade/derby/hive-schema-0.13.0.derby.sql index 6a5df4b..47db916 100644 --- metastore/scripts/upgrade/derby/hive-schema-0.13.0.derby.sql +++ metastore/scripts/upgrade/derby/hive-schema-0.13.0.derby.sql @@ -315,3 +315,8 @@ ALTER TABLE "APP"."IDXS" ADD CONSTRAINT "SQL110318025504980" CHECK (DEFERRED_REB ALTER TABLE "APP"."SDS" ADD CONSTRAINT "SQL110318025505550" CHECK (IS_COMPRESSED IN ('Y','N')); INSERT INTO "APP"."VERSION" (VER_ID, SCHEMA_VERSION, VERSION_COMMENT) VALUES (1, '0.13.0', 'Hive release version 0.13.0'); + +-- ---------------------------- +-- Transaction and Lock Tables +-- ---------------------------- +RUN 'hive-txn-schema-0.13.0.derby.sql'; diff --git metastore/scripts/upgrade/derby/hive-txn-schema-0.13.0.derby.sql metastore/scripts/upgrade/derby/hive-txn-schema-0.13.0.derby.sql new file mode 100644 index 0000000..d6c01b6 --- /dev/null +++ metastore/scripts/upgrade/derby/hive-txn-schema-0.13.0.derby.sql @@ -0,0 +1,88 @@ +-- 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. + +-- +-- Tables for transaction management +-- +CREATE TABLE TXNS ( + TXN_ID bigint PRIMARY KEY, + TXN_STATE char(1) NOT NULL, + TXN_STARTED bigint NOT NULL, + TXN_LAST_HEARTBEAT bigint NOT NULL, + TXN_USER varchar(128) NOT NULL, + TXN_HOST varchar(128) NOT NULL +); + +CREATE TABLE TXN_COMPONENTS ( + TC_TXNID bigint REFERENCES TXNS (TXN_ID), + TC_DATABASE varchar(128) NOT NULL, + TC_TABLE varchar(128), + TC_PARTITION varchar(767) +); + +CREATE TABLE COMPLETED_TXN_COMPONENTS ( + CTC_TXNID bigint, + CTC_DATABASE varchar(128) NOT NULL, + CTC_TABLE varchar(128), + CTC_PARTITION varchar(767) +); + +CREATE TABLE NEXT_TXN_ID ( + NTXN_NEXT bigint NOT NULL +); +INSERT INTO NEXT_TXN_ID VALUES(1); + +CREATE TABLE HIVE_LOCKS ( + HL_LOCK_EXT_ID bigint NOT NULL, + HL_LOCK_INT_ID bigint NOT NULL, + HL_TXNID bigint, + HL_DB varchar(128) NOT NULL, + HL_TABLE varchar(128), + HL_PARTITION varchar(767), + HL_LOCK_STATE char(1) NOT NULL, + HL_LOCK_TYPE char(1) NOT NULL, + HL_LAST_HEARTBEAT bigint NOT NULL, + HL_ACQUIRED_AT bigint, + HL_USER varchar(128) NOT NULL, + HL_HOST varchar(128) NOT NULL, + PRIMARY KEY(HL_LOCK_EXT_ID, HL_LOCK_INT_ID) +); + +CREATE INDEX HL_TXNID_INDEX ON HIVE_LOCKS (HL_TXNID); + +CREATE TABLE NEXT_LOCK_ID ( + NL_NEXT bigint NOT NULL +); +INSERT INTO NEXT_LOCK_ID VALUES(1); + +CREATE TABLE COMPACTION_QUEUE ( + CQ_ID bigint PRIMARY KEY, + CQ_DATABASE varchar(128) NOT NULL, + CQ_TABLE varchar(128) NOT NULL, + CQ_PARTITION varchar(767), + CQ_STATE char(1) NOT NULL, + CQ_TYPE char(1) NOT NULL, + CQ_WORKER_ID varchar(128), + CQ_START bigint, + CQ_RUN_AS varchar(128) +); + +CREATE TABLE NEXT_COMPACTION_QUEUE_ID ( + NCQ_NEXT bigint NOT NULL +); +INSERT INTO NEXT_COMPACTION_QUEUE_ID VALUES(1); + + + diff --git metastore/scripts/upgrade/derby/upgrade-0.12.0-to-0.13.0.derby.sql metastore/scripts/upgrade/derby/upgrade-0.12.0-to-0.13.0.derby.sql index 0ad429f..7d09e27 100644 --- metastore/scripts/upgrade/derby/upgrade-0.12.0-to-0.13.0.derby.sql +++ metastore/scripts/upgrade/derby/upgrade-0.12.0-to-0.13.0.derby.sql @@ -1,4 +1,5 @@ -- Upgrade MetaStore schema from 0.12.0 to 0.13.0 +RUN 'hive-txn-schema-0.13.0.derby.sql'; RUN '016-HIVE-6386.derby.sql'; RUN '017-HIVE-6458.derby.sql'; diff --git metastore/scripts/upgrade/mysql/hive-schema-0.13.0.mysql.sql metastore/scripts/upgrade/mysql/hive-schema-0.13.0.mysql.sql index 1c03a40..623f9c2 100644 --- metastore/scripts/upgrade/mysql/hive-schema-0.13.0.mysql.sql +++ metastore/scripts/upgrade/mysql/hive-schema-0.13.0.mysql.sql @@ -792,7 +792,12 @@ CREATE TABLE IF NOT EXISTS `FUNC_RU` ( ) ENGINE=InnoDB DEFAULT CHARSET=latin1; -INSERT INTO VERSION (VER_ID, SCHEMA_VERSION, VERSION_COMMENT) VALUES (1, '0.13.0', 'Hive release version 0.13.0'); +INSERT INTO VERSION (VER_ID, SCHEMA_VERSION, VERSION_COMMENT) VALUES (1, '0.12.0', 'Hive release version 0.12.0'); + +------------------------------ +-- Transaction and Lock Tables +------------------------------ +SOURCE hive-txn-schema-0.13.0.mysql.sql; /*!40101 SET character_set_client = @saved_cs_client */; /*!40103 SET TIME_ZONE=@OLD_TIME_ZONE */; diff --git metastore/scripts/upgrade/mysql/hive-txn-schema-0.13.0.mysql.sql metastore/scripts/upgrade/mysql/hive-txn-schema-0.13.0.mysql.sql new file mode 100644 index 0000000..7107ffc --- /dev/null +++ metastore/scripts/upgrade/mysql/hive-txn-schema-0.13.0.mysql.sql @@ -0,0 +1,90 @@ +-- 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. + +-- +-- Tables for transaction management +-- + +CREATE TABLE TXNS ( + TXN_ID bigint PRIMARY KEY, + TXN_STATE char(1) NOT NULL, + TXN_STARTED bigint NOT NULL, + TXN_LAST_HEARTBEAT bigint NOT NULL, + TXN_USER varchar(128) NOT NULL, + TXN_HOST varchar(128) NOT NULL +) ENGINE=InnoDB; + +CREATE TABLE TXN_COMPONENTS ( + TC_TXNID bigint, + TC_DATABASE varchar(128) NOT NULL, + TC_TABLE varchar(128), + TC_PARTITION varchar(767), + FOREIGN KEY (TC_TXNID) REFERENCES TXNS (TXN_ID) +) ENGINE=InnoDB; + +CREATE TABLE COMPLETED_TXN_COMPONENTS ( + CTC_TXNID bigint, + CTC_DATABASE varchar(128) NOT NULL, + CTC_TABLE varchar(128), + CTC_PARTITION varchar(767) +) ENGINE=InnoDB; + +CREATE TABLE NEXT_TXN_ID ( + NTXN_NEXT bigint NOT NULL +) ENGINE=InnoDB; +INSERT INTO NEXT_TXN_ID VALUES(1); + +CREATE TABLE HIVE_LOCKS ( + HL_LOCK_EXT_ID bigint NOT NULL, + HL_LOCK_INT_ID bigint NOT NULL, + HL_TXNID bigint, + HL_DB varchar(128) NOT NULL, + HL_TABLE varchar(128), + HL_PARTITION varchar(767), + HL_LOCK_STATE char(1) not null, + HL_LOCK_TYPE char(1) not null, + HL_LAST_HEARTBEAT bigint NOT NULL, + HL_ACQUIRED_AT bigint, + HL_USER varchar(128) NOT NULL, + HL_HOST varchar(128) NOT NULL, + PRIMARY KEY(HL_LOCK_EXT_ID, HL_LOCK_INT_ID), + KEY HIVE_LOCK_TXNID_INDEX (HL_TXNID) +) ENGINE=InnoDB; + +CREATE INDEX HL_TXNID_IDX ON HIVE_LOCKS (HL_TXNID) USING HASH; + +CREATE TABLE NEXT_LOCK_ID ( + NL_NEXT bigint NOT NULL +) ENGINE=InnoDB; +INSERT INTO NEXT_LOCK_ID VALUES(1); + +CREATE TABLE COMPACTION_QUEUE ( + CQ_ID bigint PRIMARY KEY, + CQ_DATABASE varchar(128) NOT NULL, + CQ_TABLE varchar(128) NOT NULL, + CQ_PARTITION varchar(767), + CQ_STATE char(1) NOT NULL, + CQ_TYPE char(1) NOT NULL, + CQ_WORKER_ID varchar(128), + CQ_START bigint, + CQ_RUN_AS varchar(128) +); + +CREATE TABLE NEXT_COMPACTION_QUEUE_ID ( + NCQ_NEXT bigint NOT NULL +); +INSERT INTO NEXT_COMPACTION_QUEUE_ID VALUES(1); + + diff --git metastore/scripts/upgrade/mysql/upgrade-0.12.0-to-0.13.0.mysql.sql metastore/scripts/upgrade/mysql/upgrade-0.12.0-to-0.13.0.mysql.sql index f5d6793..0e428e4 100644 --- metastore/scripts/upgrade/mysql/upgrade-0.12.0-to-0.13.0.mysql.sql +++ metastore/scripts/upgrade/mysql/upgrade-0.12.0-to-0.13.0.mysql.sql @@ -3,6 +3,7 @@ SELECT 'Upgrading MetaStore schema from 0.12.0 to 0.13.0' AS ' '; SOURCE 015-HIVE-5700.mysql.sql; SOURCE 016-HIVE-6386.mysql.sql; SOURCE 017-HIVE-6458.mysql.sql; +SOURCE hive-txn-schema-0.13.0.mysql.sql; UPDATE VERSION SET SCHEMA_VERSION='0.13.0', VERSION_COMMENT='Hive release version 0.13.0' where VER_ID=1; SELECT 'Finished upgrading MetaStore schema from 0.12.0 to 0.13.0' AS ' '; diff --git metastore/scripts/upgrade/oracle/hive-schema-0.13.0.oracle.sql metastore/scripts/upgrade/oracle/hive-schema-0.13.0.oracle.sql index 517d2dc..8554a03 100644 --- metastore/scripts/upgrade/oracle/hive-schema-0.13.0.oracle.sql +++ metastore/scripts/upgrade/oracle/hive-schema-0.13.0.oracle.sql @@ -754,3 +754,7 @@ CREATE INDEX FUNC_RU_N49 ON FUNC_RU (FUNC_ID); INSERT INTO VERSION (VER_ID, SCHEMA_VERSION, VERSION_COMMENT) VALUES (1, '0.13.0', 'Hive release version 0.13.0'); +------------------------------ +-- Transaction and lock tables +------------------------------ +@hive-txn-schema-0.13.0.oracle.sql; diff --git metastore/scripts/upgrade/oracle/hive-txn-schema-0.13.0.oracle.sql metastore/scripts/upgrade/oracle/hive-txn-schema-0.13.0.oracle.sql new file mode 100644 index 0000000..c2ff663 --- /dev/null +++ metastore/scripts/upgrade/oracle/hive-txn-schema-0.13.0.oracle.sql @@ -0,0 +1,88 @@ +-- 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. + +-- +-- Tables for transaction management +-- + +CREATE TABLE TXNS ( + TXN_ID NUMBER(10) PRIMARY KEY, + TXN_STATE char(1) NOT NULL, + TXN_STARTED NUMBER(10) NOT NULL, + TXN_LAST_HEARTBEAT NUMBER(10) NOT NULL, + TXN_USER varchar(128) NOT NULL, + TXN_HOST varchar(128) NOT NULL +); + +CREATE TABLE TXN_COMPONENTS ( + TC_TXNID NUMBER(10) REFERENCES TXNS (TXN_ID), + TC_DATABASE VARCHAR2(128) NOT NULL, + TC_TABLE VARCHAR2(128), + TC_PARTITION VARCHAR2(767) NULL +); + +CREATE TABLE COMPLETED_TXN_COMPONENTS ( + CTC_TXNID NUMBER(10), + CTC_DATABASE varchar(128) NOT NULL, + CTC_TABLE varchar(128), + CTC_PARTITION varchar(767) +); + +CREATE TABLE NEXT_TXN_ID ( + NTXN_NEXT NUMBER(10) NOT NULL +); +INSERT INTO NEXT_TXN_ID VALUES(1); + +CREATE TABLE HIVE_LOCKS ( + HL_LOCK_EXT_ID NUMBER(10) NOT NULL, + HL_LOCK_INT_ID NUMBER(10) NOT NULL, + HL_TXNID NUMBER(10), + HL_DB VARCHAR2(128) NOT NULL, + HL_TABLE VARCHAR2(128), + HL_PARTITION VARCHAR2(767), + HL_LOCK_STATE CHAR(1) NOT NULL, + HL_LOCK_TYPE CHAR(1) NOT NULL, + HL_LAST_HEARTBEAT NUMBER(10) NOT NULL, + HL_ACQUIRED_AT NUMBER(10), + HL_USER varchar(128) NOT NULL, + HL_HOST varchar(128) NOT NULL, + PRIMARY KEY(HL_LOCK_EXT_ID, HL_LOCK_INT_ID) +); + +CREATE INDEX HL_TXNID_INDEX ON HIVE_LOCKS (HL_TXNID); + +CREATE TABLE NEXT_LOCK_ID ( + NL_NEXT NUMBER(10) NOT NULL +); +INSERT INTO NEXT_LOCK_ID VALUES(1); + +CREATE TABLE COMPACTION_QUEUE ( + CQ_ID NUMBER(10) PRIMARY KEY, + CQ_DATABASE varchar(128) NOT NULL, + CQ_TABLE varchar(128) NOT NULL, + CQ_PARTITION varchar(767), + CQ_STATE char(1) NOT NULL, + CQ_TYPE char(1) NOT NULL, + CQ_WORKER_ID varchar(128), + CQ_START NUMBER(10), + CQ_RUN_AS varchar(128) +); + +CREATE TABLE NEXT_COMPACTION_QUEUE_ID ( + NCQ_NEXT NUMBER(10) NOT NULL +); +INSERT INTO NEXT_COMPACTION_QUEUE_ID VALUES(1); + + diff --git metastore/scripts/upgrade/oracle/upgrade-0.12.0-to-0.13.0.oracle.sql metastore/scripts/upgrade/oracle/upgrade-0.12.0-to-0.13.0.oracle.sql index fc1f23d..3a36298 100644 --- metastore/scripts/upgrade/oracle/upgrade-0.12.0-to-0.13.0.oracle.sql +++ metastore/scripts/upgrade/oracle/upgrade-0.12.0-to-0.13.0.oracle.sql @@ -3,6 +3,7 @@ SELECT 'Upgrading MetaStore schema from 0.12.0 to 0.13.0' AS Status from dual; @015-HIVE-5700.oracle.sql; @016-HIVE-6386.oracle.sql; @017-HIVE-6458.oracle.sql; +@hive-txn-schema-0.13.0.oracle.sql; UPDATE VERSION SET SCHEMA_VERSION='0.13.0', VERSION_COMMENT='Hive release version 0.13.0' where VER_ID=1; SELECT 'Finished upgrading MetaStore schema from 0.12.0 to 0.13.0' AS Status from dual; diff --git metastore/scripts/upgrade/postgres/hive-schema-0.13.0.postgres.sql metastore/scripts/upgrade/postgres/hive-schema-0.13.0.postgres.sql index 15c5404..04d3537 100644 --- metastore/scripts/upgrade/postgres/hive-schema-0.13.0.postgres.sql +++ metastore/scripts/upgrade/postgres/hive-schema-0.13.0.postgres.sql @@ -1456,3 +1456,7 @@ INSERT INTO "VERSION" ("VER_ID", "SCHEMA_VERSION", "VERSION_COMMENT") VALUES (1, -- PostgreSQL database dump complete -- +------------------------------ +-- Transaction and lock tables +------------------------------ +\i hive-txn-schema-0.13.0.postgres.sql; diff --git metastore/scripts/upgrade/postgres/hive-txn-schema-0.13.0.postgres.sql metastore/scripts/upgrade/postgres/hive-txn-schema-0.13.0.postgres.sql new file mode 100644 index 0000000..2ebd3b0 --- /dev/null +++ metastore/scripts/upgrade/postgres/hive-txn-schema-0.13.0.postgres.sql @@ -0,0 +1,89 @@ +-- 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. + +-- +-- Tables for transaction management +-- + +CREATE TABLE TXNS ( + TXN_ID bigint PRIMARY KEY, + TXN_STATE char(1) NOT NULL, + TXN_STARTED bigint NOT NULL, + TXN_LAST_HEARTBEAT bigint NOT NULL, + TXN_USER varchar(128) NOT NULL, + TXN_HOST varchar(128) NOT NULL +); + +CREATE TABLE TXN_COMPONENTS ( + TC_TXNID bigint REFERENCES TXNS (TXN_ID), + TC_DATABASE varchar(128) NOT NULL, + TC_TABLE varchar(128), + TC_PARTITION varchar(767) DEFAULT NULL +); + +CREATE TABLE COMPLETED_TXN_COMPONENTS ( + CTC_TXNID bigint, + CTC_DATABASE varchar(128) NOT NULL, + CTC_TABLE varchar(128), + CTC_PARTITION varchar(767) +); + +CREATE TABLE NEXT_TXN_ID ( + NTXN_NEXT bigint NOT NULL +); +INSERT INTO NEXT_TXN_ID VALUES(1); + +CREATE TABLE HIVE_LOCKS ( + HL_LOCK_EXT_ID bigint NOT NULL, + HL_LOCK_INT_ID bigint NOT NULL, + HL_TXNID bigint, + HL_DB varchar(128) NOT NULL, + HL_TABLE varchar(128), + HL_PARTITION varchar(767) DEFAULT NULL, + HL_LOCK_STATE char(1) NOT NULL, + HL_LOCK_TYPE char(1) NOT NULL, + HL_LAST_HEARTBEAT bigint NOT NULL, + HL_ACQUIRED_AT bigint, + HL_USER varchar(128) NOT NULL, + HL_HOST varchar(128) NOT NULL, + PRIMARY KEY(HL_LOCK_EXT_ID, HL_LOCK_INT_ID) +); + +CREATE INDEX HL_TXNID_INDEX ON HIVE_LOCKS USING hash (HL_TXNID); + +CREATE TABLE NEXT_LOCK_ID ( + NL_NEXT bigint NOT NULL +); +INSERT INTO NEXT_LOCK_ID VALUES(1); + +CREATE TABLE COMPACTION_QUEUE ( + CQ_ID bigint PRIMARY KEY, + CQ_DATABASE varchar(128) NOT NULL, + CQ_TABLE varchar(128) NOT NULL, + CQ_PARTITION varchar(767), + CQ_STATE char(1) NOT NULL, + CQ_TYPE char(1) NOT NULL, + CQ_WORKER_ID varchar(128), + CQ_START bigint, + CQ_RUN_AS varchar(128) +); + +CREATE TABLE NEXT_COMPACTION_QUEUE_ID ( + NCQ_NEXT bigint NOT NULL +); +INSERT INTO NEXT_COMPACTION_QUEUE_ID VALUES(1); + + + diff --git metastore/scripts/upgrade/postgres/upgrade-0.12.0-to-0.13.0.postgres.sql metastore/scripts/upgrade/postgres/upgrade-0.12.0-to-0.13.0.postgres.sql index 5c97b34..dcddf77 100644 --- metastore/scripts/upgrade/postgres/upgrade-0.12.0-to-0.13.0.postgres.sql +++ metastore/scripts/upgrade/postgres/upgrade-0.12.0-to-0.13.0.postgres.sql @@ -3,6 +3,7 @@ SELECT 'Upgrading MetaStore schema from 0.12.0 to 0.13.0'; \i 015-HIVE-5700.postgres.sql; \i 016-HIVE-6386.postgres.sql; \i 017-HIVE-6458.postgres.sql; +\i hive-txn-schema-0.13.0.postgres.sql; UPDATE "VERSION" SET "SCHEMA_VERSION"='0.13.0', "VERSION_COMMENT"='Hive release version 0.13.0' where "VER_ID"=1; SELECT 'Finished upgrading MetaStore schema from 0.12.0 to 0.13.0'; diff --git metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp index 8998182..76a3f5d 100644 --- metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp +++ metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp @@ -736,14 +736,14 @@ uint32_t ThriftHiveMetastore_get_databases_result::read(::apache::thrift::protoc if (ftype == ::apache::thrift::protocol::T_LIST) { { this->success.clear(); - uint32_t _size285; - ::apache::thrift::protocol::TType _etype288; - xfer += iprot->readListBegin(_etype288, _size285); - this->success.resize(_size285); - uint32_t _i289; - for (_i289 = 0; _i289 < _size285; ++_i289) + uint32_t _size330; + ::apache::thrift::protocol::TType _etype333; + xfer += iprot->readListBegin(_etype333, _size330); + this->success.resize(_size330); + uint32_t _i334; + for (_i334 = 0; _i334 < _size330; ++_i334) { - xfer += iprot->readString(this->success[_i289]); + xfer += iprot->readString(this->success[_i334]); } xfer += iprot->readListEnd(); } @@ -782,10 +782,10 @@ uint32_t ThriftHiveMetastore_get_databases_result::write(::apache::thrift::proto 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 _iter290; - for (_iter290 = this->success.begin(); _iter290 != this->success.end(); ++_iter290) + std::vector ::const_iterator _iter335; + for (_iter335 = this->success.begin(); _iter335 != this->success.end(); ++_iter335) { - xfer += oprot->writeString((*_iter290)); + xfer += oprot->writeString((*_iter335)); } xfer += oprot->writeListEnd(); } @@ -824,14 +824,14 @@ uint32_t ThriftHiveMetastore_get_databases_presult::read(::apache::thrift::proto if (ftype == ::apache::thrift::protocol::T_LIST) { { (*(this->success)).clear(); - uint32_t _size291; - ::apache::thrift::protocol::TType _etype294; - xfer += iprot->readListBegin(_etype294, _size291); - (*(this->success)).resize(_size291); - uint32_t _i295; - for (_i295 = 0; _i295 < _size291; ++_i295) + uint32_t _size336; + ::apache::thrift::protocol::TType _etype339; + xfer += iprot->readListBegin(_etype339, _size336); + (*(this->success)).resize(_size336); + uint32_t _i340; + for (_i340 = 0; _i340 < _size336; ++_i340) { - xfer += iprot->readString((*(this->success))[_i295]); + xfer += iprot->readString((*(this->success))[_i340]); } xfer += iprot->readListEnd(); } @@ -929,14 +929,14 @@ uint32_t ThriftHiveMetastore_get_all_databases_result::read(::apache::thrift::pr if (ftype == ::apache::thrift::protocol::T_LIST) { { this->success.clear(); - uint32_t _size296; - ::apache::thrift::protocol::TType _etype299; - xfer += iprot->readListBegin(_etype299, _size296); - this->success.resize(_size296); - uint32_t _i300; - for (_i300 = 0; _i300 < _size296; ++_i300) + uint32_t _size341; + ::apache::thrift::protocol::TType _etype344; + xfer += iprot->readListBegin(_etype344, _size341); + this->success.resize(_size341); + uint32_t _i345; + for (_i345 = 0; _i345 < _size341; ++_i345) { - xfer += iprot->readString(this->success[_i300]); + xfer += iprot->readString(this->success[_i345]); } xfer += iprot->readListEnd(); } @@ -975,10 +975,10 @@ uint32_t ThriftHiveMetastore_get_all_databases_result::write(::apache::thrift::p 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 _iter301; - for (_iter301 = this->success.begin(); _iter301 != this->success.end(); ++_iter301) + std::vector ::const_iterator _iter346; + for (_iter346 = this->success.begin(); _iter346 != this->success.end(); ++_iter346) { - xfer += oprot->writeString((*_iter301)); + xfer += oprot->writeString((*_iter346)); } xfer += oprot->writeListEnd(); } @@ -1017,14 +1017,14 @@ uint32_t ThriftHiveMetastore_get_all_databases_presult::read(::apache::thrift::p if (ftype == ::apache::thrift::protocol::T_LIST) { { (*(this->success)).clear(); - uint32_t _size302; - ::apache::thrift::protocol::TType _etype305; - xfer += iprot->readListBegin(_etype305, _size302); - (*(this->success)).resize(_size302); - uint32_t _i306; - for (_i306 = 0; _i306 < _size302; ++_i306) + uint32_t _size347; + ::apache::thrift::protocol::TType _etype350; + xfer += iprot->readListBegin(_etype350, _size347); + (*(this->success)).resize(_size347); + uint32_t _i351; + for (_i351 = 0; _i351 < _size347; ++_i351) { - xfer += iprot->readString((*(this->success))[_i306]); + xfer += iprot->readString((*(this->success))[_i351]); } xfer += iprot->readListEnd(); } @@ -1967,17 +1967,17 @@ uint32_t ThriftHiveMetastore_get_type_all_result::read(::apache::thrift::protoco if (ftype == ::apache::thrift::protocol::T_MAP) { { this->success.clear(); - uint32_t _size307; - ::apache::thrift::protocol::TType _ktype308; - ::apache::thrift::protocol::TType _vtype309; - xfer += iprot->readMapBegin(_ktype308, _vtype309, _size307); - uint32_t _i311; - for (_i311 = 0; _i311 < _size307; ++_i311) + uint32_t _size352; + ::apache::thrift::protocol::TType _ktype353; + ::apache::thrift::protocol::TType _vtype354; + xfer += iprot->readMapBegin(_ktype353, _vtype354, _size352); + uint32_t _i356; + for (_i356 = 0; _i356 < _size352; ++_i356) { - std::string _key312; - xfer += iprot->readString(_key312); - Type& _val313 = this->success[_key312]; - xfer += _val313.read(iprot); + std::string _key357; + xfer += iprot->readString(_key357); + Type& _val358 = this->success[_key357]; + xfer += _val358.read(iprot); } xfer += iprot->readMapEnd(); } @@ -2016,11 +2016,11 @@ uint32_t ThriftHiveMetastore_get_type_all_result::write(::apache::thrift::protoc 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 _iter314; - for (_iter314 = this->success.begin(); _iter314 != this->success.end(); ++_iter314) + std::map ::const_iterator _iter359; + for (_iter359 = this->success.begin(); _iter359 != this->success.end(); ++_iter359) { - xfer += oprot->writeString(_iter314->first); - xfer += _iter314->second.write(oprot); + xfer += oprot->writeString(_iter359->first); + xfer += _iter359->second.write(oprot); } xfer += oprot->writeMapEnd(); } @@ -2059,17 +2059,17 @@ uint32_t ThriftHiveMetastore_get_type_all_presult::read(::apache::thrift::protoc if (ftype == ::apache::thrift::protocol::T_MAP) { { (*(this->success)).clear(); - uint32_t _size315; - ::apache::thrift::protocol::TType _ktype316; - ::apache::thrift::protocol::TType _vtype317; - xfer += iprot->readMapBegin(_ktype316, _vtype317, _size315); - uint32_t _i319; - for (_i319 = 0; _i319 < _size315; ++_i319) + uint32_t _size360; + ::apache::thrift::protocol::TType _ktype361; + ::apache::thrift::protocol::TType _vtype362; + xfer += iprot->readMapBegin(_ktype361, _vtype362, _size360); + uint32_t _i364; + for (_i364 = 0; _i364 < _size360; ++_i364) { - std::string _key320; - xfer += iprot->readString(_key320); - Type& _val321 = (*(this->success))[_key320]; - xfer += _val321.read(iprot); + std::string _key365; + xfer += iprot->readString(_key365); + Type& _val366 = (*(this->success))[_key365]; + xfer += _val366.read(iprot); } xfer += iprot->readMapEnd(); } @@ -2204,14 +2204,14 @@ uint32_t ThriftHiveMetastore_get_fields_result::read(::apache::thrift::protocol: if (ftype == ::apache::thrift::protocol::T_LIST) { { this->success.clear(); - uint32_t _size322; - ::apache::thrift::protocol::TType _etype325; - xfer += iprot->readListBegin(_etype325, _size322); - this->success.resize(_size322); - uint32_t _i326; - for (_i326 = 0; _i326 < _size322; ++_i326) + uint32_t _size367; + ::apache::thrift::protocol::TType _etype370; + xfer += iprot->readListBegin(_etype370, _size367); + this->success.resize(_size367); + uint32_t _i371; + for (_i371 = 0; _i371 < _size367; ++_i371) { - xfer += this->success[_i326].read(iprot); + xfer += this->success[_i371].read(iprot); } xfer += iprot->readListEnd(); } @@ -2266,10 +2266,10 @@ uint32_t ThriftHiveMetastore_get_fields_result::write(::apache::thrift::protocol 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 _iter327; - for (_iter327 = this->success.begin(); _iter327 != this->success.end(); ++_iter327) + std::vector ::const_iterator _iter372; + for (_iter372 = this->success.begin(); _iter372 != this->success.end(); ++_iter372) { - xfer += (*_iter327).write(oprot); + xfer += (*_iter372).write(oprot); } xfer += oprot->writeListEnd(); } @@ -2316,14 +2316,14 @@ uint32_t ThriftHiveMetastore_get_fields_presult::read(::apache::thrift::protocol if (ftype == ::apache::thrift::protocol::T_LIST) { { (*(this->success)).clear(); - uint32_t _size328; - ::apache::thrift::protocol::TType _etype331; - xfer += iprot->readListBegin(_etype331, _size328); - (*(this->success)).resize(_size328); - uint32_t _i332; - for (_i332 = 0; _i332 < _size328; ++_i332) + uint32_t _size373; + ::apache::thrift::protocol::TType _etype376; + xfer += iprot->readListBegin(_etype376, _size373); + (*(this->success)).resize(_size373); + uint32_t _i377; + for (_i377 = 0; _i377 < _size373; ++_i377) { - xfer += (*(this->success))[_i332].read(iprot); + xfer += (*(this->success))[_i377].read(iprot); } xfer += iprot->readListEnd(); } @@ -2474,14 +2474,14 @@ uint32_t ThriftHiveMetastore_get_schema_result::read(::apache::thrift::protocol: if (ftype == ::apache::thrift::protocol::T_LIST) { { this->success.clear(); - uint32_t _size333; - ::apache::thrift::protocol::TType _etype336; - xfer += iprot->readListBegin(_etype336, _size333); - this->success.resize(_size333); - uint32_t _i337; - for (_i337 = 0; _i337 < _size333; ++_i337) + uint32_t _size378; + ::apache::thrift::protocol::TType _etype381; + xfer += iprot->readListBegin(_etype381, _size378); + this->success.resize(_size378); + uint32_t _i382; + for (_i382 = 0; _i382 < _size378; ++_i382) { - xfer += this->success[_i337].read(iprot); + xfer += this->success[_i382].read(iprot); } xfer += iprot->readListEnd(); } @@ -2536,10 +2536,10 @@ uint32_t ThriftHiveMetastore_get_schema_result::write(::apache::thrift::protocol 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 _iter338; - for (_iter338 = this->success.begin(); _iter338 != this->success.end(); ++_iter338) + std::vector ::const_iterator _iter383; + for (_iter383 = this->success.begin(); _iter383 != this->success.end(); ++_iter383) { - xfer += (*_iter338).write(oprot); + xfer += (*_iter383).write(oprot); } xfer += oprot->writeListEnd(); } @@ -2586,14 +2586,14 @@ uint32_t ThriftHiveMetastore_get_schema_presult::read(::apache::thrift::protocol if (ftype == ::apache::thrift::protocol::T_LIST) { { (*(this->success)).clear(); - uint32_t _size339; - ::apache::thrift::protocol::TType _etype342; - xfer += iprot->readListBegin(_etype342, _size339); - (*(this->success)).resize(_size339); - uint32_t _i343; - for (_i343 = 0; _i343 < _size339; ++_i343) + uint32_t _size384; + ::apache::thrift::protocol::TType _etype387; + xfer += iprot->readListBegin(_etype387, _size384); + (*(this->success)).resize(_size384); + uint32_t _i388; + for (_i388 = 0; _i388 < _size384; ++_i388) { - xfer += (*(this->success))[_i343].read(iprot); + xfer += (*(this->success))[_i388].read(iprot); } xfer += iprot->readListEnd(); } @@ -3648,14 +3648,14 @@ uint32_t ThriftHiveMetastore_get_tables_result::read(::apache::thrift::protocol: if (ftype == ::apache::thrift::protocol::T_LIST) { { this->success.clear(); - uint32_t _size344; - ::apache::thrift::protocol::TType _etype347; - xfer += iprot->readListBegin(_etype347, _size344); - this->success.resize(_size344); - uint32_t _i348; - for (_i348 = 0; _i348 < _size344; ++_i348) + uint32_t _size389; + ::apache::thrift::protocol::TType _etype392; + xfer += iprot->readListBegin(_etype392, _size389); + this->success.resize(_size389); + uint32_t _i393; + for (_i393 = 0; _i393 < _size389; ++_i393) { - xfer += iprot->readString(this->success[_i348]); + xfer += iprot->readString(this->success[_i393]); } xfer += iprot->readListEnd(); } @@ -3694,10 +3694,10 @@ uint32_t ThriftHiveMetastore_get_tables_result::write(::apache::thrift::protocol 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 _iter349; - for (_iter349 = this->success.begin(); _iter349 != this->success.end(); ++_iter349) + std::vector ::const_iterator _iter394; + for (_iter394 = this->success.begin(); _iter394 != this->success.end(); ++_iter394) { - xfer += oprot->writeString((*_iter349)); + xfer += oprot->writeString((*_iter394)); } xfer += oprot->writeListEnd(); } @@ -3736,14 +3736,14 @@ uint32_t ThriftHiveMetastore_get_tables_presult::read(::apache::thrift::protocol if (ftype == ::apache::thrift::protocol::T_LIST) { { (*(this->success)).clear(); - uint32_t _size350; - ::apache::thrift::protocol::TType _etype353; - xfer += iprot->readListBegin(_etype353, _size350); - (*(this->success)).resize(_size350); - uint32_t _i354; - for (_i354 = 0; _i354 < _size350; ++_i354) + uint32_t _size395; + ::apache::thrift::protocol::TType _etype398; + xfer += iprot->readListBegin(_etype398, _size395); + (*(this->success)).resize(_size395); + uint32_t _i399; + for (_i399 = 0; _i399 < _size395; ++_i399) { - xfer += iprot->readString((*(this->success))[_i354]); + xfer += iprot->readString((*(this->success))[_i399]); } xfer += iprot->readListEnd(); } @@ -3862,14 +3862,14 @@ uint32_t ThriftHiveMetastore_get_all_tables_result::read(::apache::thrift::proto if (ftype == ::apache::thrift::protocol::T_LIST) { { this->success.clear(); - uint32_t _size355; - ::apache::thrift::protocol::TType _etype358; - xfer += iprot->readListBegin(_etype358, _size355); - this->success.resize(_size355); - uint32_t _i359; - for (_i359 = 0; _i359 < _size355; ++_i359) + uint32_t _size400; + ::apache::thrift::protocol::TType _etype403; + xfer += iprot->readListBegin(_etype403, _size400); + this->success.resize(_size400); + uint32_t _i404; + for (_i404 = 0; _i404 < _size400; ++_i404) { - xfer += iprot->readString(this->success[_i359]); + xfer += iprot->readString(this->success[_i404]); } xfer += iprot->readListEnd(); } @@ -3908,10 +3908,10 @@ uint32_t ThriftHiveMetastore_get_all_tables_result::write(::apache::thrift::prot 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 _iter360; - for (_iter360 = this->success.begin(); _iter360 != this->success.end(); ++_iter360) + std::vector ::const_iterator _iter405; + for (_iter405 = this->success.begin(); _iter405 != this->success.end(); ++_iter405) { - xfer += oprot->writeString((*_iter360)); + xfer += oprot->writeString((*_iter405)); } xfer += oprot->writeListEnd(); } @@ -3950,14 +3950,14 @@ uint32_t ThriftHiveMetastore_get_all_tables_presult::read(::apache::thrift::prot if (ftype == ::apache::thrift::protocol::T_LIST) { { (*(this->success)).clear(); - uint32_t _size361; - ::apache::thrift::protocol::TType _etype364; - xfer += iprot->readListBegin(_etype364, _size361); - (*(this->success)).resize(_size361); - uint32_t _i365; - for (_i365 = 0; _i365 < _size361; ++_i365) + uint32_t _size406; + ::apache::thrift::protocol::TType _etype409; + xfer += iprot->readListBegin(_etype409, _size406); + (*(this->success)).resize(_size406); + uint32_t _i410; + for (_i410 = 0; _i410 < _size406; ++_i410) { - xfer += iprot->readString((*(this->success))[_i365]); + xfer += iprot->readString((*(this->success))[_i410]); } xfer += iprot->readListEnd(); } @@ -4236,14 +4236,14 @@ uint32_t ThriftHiveMetastore_get_table_objects_by_name_args::read(::apache::thri if (ftype == ::apache::thrift::protocol::T_LIST) { { this->tbl_names.clear(); - uint32_t _size366; - ::apache::thrift::protocol::TType _etype369; - xfer += iprot->readListBegin(_etype369, _size366); - this->tbl_names.resize(_size366); - uint32_t _i370; - for (_i370 = 0; _i370 < _size366; ++_i370) + uint32_t _size411; + ::apache::thrift::protocol::TType _etype414; + xfer += iprot->readListBegin(_etype414, _size411); + this->tbl_names.resize(_size411); + uint32_t _i415; + for (_i415 = 0; _i415 < _size411; ++_i415) { - xfer += iprot->readString(this->tbl_names[_i370]); + xfer += iprot->readString(this->tbl_names[_i415]); } xfer += iprot->readListEnd(); } @@ -4275,10 +4275,10 @@ uint32_t ThriftHiveMetastore_get_table_objects_by_name_args::write(::apache::thr 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 _iter371; - for (_iter371 = this->tbl_names.begin(); _iter371 != this->tbl_names.end(); ++_iter371) + std::vector ::const_iterator _iter416; + for (_iter416 = this->tbl_names.begin(); _iter416 != this->tbl_names.end(); ++_iter416) { - xfer += oprot->writeString((*_iter371)); + xfer += oprot->writeString((*_iter416)); } xfer += oprot->writeListEnd(); } @@ -4300,10 +4300,10 @@ uint32_t ThriftHiveMetastore_get_table_objects_by_name_pargs::write(::apache::th 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 _iter372; - for (_iter372 = (*(this->tbl_names)).begin(); _iter372 != (*(this->tbl_names)).end(); ++_iter372) + std::vector ::const_iterator _iter417; + for (_iter417 = (*(this->tbl_names)).begin(); _iter417 != (*(this->tbl_names)).end(); ++_iter417) { - xfer += oprot->writeString((*_iter372)); + xfer += oprot->writeString((*_iter417)); } xfer += oprot->writeListEnd(); } @@ -4338,14 +4338,14 @@ uint32_t ThriftHiveMetastore_get_table_objects_by_name_result::read(::apache::th if (ftype == ::apache::thrift::protocol::T_LIST) { { this->success.clear(); - uint32_t _size373; - ::apache::thrift::protocol::TType _etype376; - xfer += iprot->readListBegin(_etype376, _size373); - this->success.resize(_size373); - uint32_t _i377; - for (_i377 = 0; _i377 < _size373; ++_i377) + uint32_t _size418; + ::apache::thrift::protocol::TType _etype421; + xfer += iprot->readListBegin(_etype421, _size418); + this->success.resize(_size418); + uint32_t _i422; + for (_i422 = 0; _i422 < _size418; ++_i422) { - xfer += this->success[_i377].read(iprot); + xfer += this->success[_i422].read(iprot); } xfer += iprot->readListEnd(); } @@ -4400,10 +4400,10 @@ uint32_t ThriftHiveMetastore_get_table_objects_by_name_result::write(::apache::t 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 _iter378; - for (_iter378 = this->success.begin(); _iter378 != this->success.end(); ++_iter378) + std::vector
::const_iterator _iter423; + for (_iter423 = this->success.begin(); _iter423 != this->success.end(); ++_iter423) { - xfer += (*_iter378).write(oprot); + xfer += (*_iter423).write(oprot); } xfer += oprot->writeListEnd(); } @@ -4450,14 +4450,14 @@ uint32_t ThriftHiveMetastore_get_table_objects_by_name_presult::read(::apache::t if (ftype == ::apache::thrift::protocol::T_LIST) { { (*(this->success)).clear(); - uint32_t _size379; - ::apache::thrift::protocol::TType _etype382; - xfer += iprot->readListBegin(_etype382, _size379); - (*(this->success)).resize(_size379); - uint32_t _i383; - for (_i383 = 0; _i383 < _size379; ++_i383) + uint32_t _size424; + ::apache::thrift::protocol::TType _etype427; + xfer += iprot->readListBegin(_etype427, _size424); + (*(this->success)).resize(_size424); + uint32_t _i428; + for (_i428 = 0; _i428 < _size424; ++_i428) { - xfer += (*(this->success))[_i383].read(iprot); + xfer += (*(this->success))[_i428].read(iprot); } xfer += iprot->readListEnd(); } @@ -4624,14 +4624,14 @@ uint32_t ThriftHiveMetastore_get_table_names_by_filter_result::read(::apache::th if (ftype == ::apache::thrift::protocol::T_LIST) { { this->success.clear(); - uint32_t _size384; - ::apache::thrift::protocol::TType _etype387; - xfer += iprot->readListBegin(_etype387, _size384); - this->success.resize(_size384); - uint32_t _i388; - for (_i388 = 0; _i388 < _size384; ++_i388) + uint32_t _size429; + ::apache::thrift::protocol::TType _etype432; + xfer += iprot->readListBegin(_etype432, _size429); + this->success.resize(_size429); + uint32_t _i433; + for (_i433 = 0; _i433 < _size429; ++_i433) { - xfer += iprot->readString(this->success[_i388]); + xfer += iprot->readString(this->success[_i433]); } xfer += iprot->readListEnd(); } @@ -4686,10 +4686,10 @@ uint32_t ThriftHiveMetastore_get_table_names_by_filter_result::write(::apache::t 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 _iter389; - for (_iter389 = this->success.begin(); _iter389 != this->success.end(); ++_iter389) + std::vector ::const_iterator _iter434; + for (_iter434 = this->success.begin(); _iter434 != this->success.end(); ++_iter434) { - xfer += oprot->writeString((*_iter389)); + xfer += oprot->writeString((*_iter434)); } xfer += oprot->writeListEnd(); } @@ -4736,14 +4736,14 @@ uint32_t ThriftHiveMetastore_get_table_names_by_filter_presult::read(::apache::t if (ftype == ::apache::thrift::protocol::T_LIST) { { (*(this->success)).clear(); - uint32_t _size390; - ::apache::thrift::protocol::TType _etype393; - xfer += iprot->readListBegin(_etype393, _size390); - (*(this->success)).resize(_size390); - uint32_t _i394; - for (_i394 = 0; _i394 < _size390; ++_i394) + uint32_t _size435; + ::apache::thrift::protocol::TType _etype438; + xfer += iprot->readListBegin(_etype438, _size435); + (*(this->success)).resize(_size435); + uint32_t _i439; + for (_i439 = 0; _i439 < _size435; ++_i439) { - xfer += iprot->readString((*(this->success))[_i394]); + xfer += iprot->readString((*(this->success))[_i439]); } xfer += iprot->readListEnd(); } @@ -5716,14 +5716,14 @@ uint32_t ThriftHiveMetastore_add_partitions_args::read(::apache::thrift::protoco if (ftype == ::apache::thrift::protocol::T_LIST) { { this->new_parts.clear(); - uint32_t _size395; - ::apache::thrift::protocol::TType _etype398; - xfer += iprot->readListBegin(_etype398, _size395); - this->new_parts.resize(_size395); - uint32_t _i399; - for (_i399 = 0; _i399 < _size395; ++_i399) + uint32_t _size440; + ::apache::thrift::protocol::TType _etype443; + xfer += iprot->readListBegin(_etype443, _size440); + this->new_parts.resize(_size440); + uint32_t _i444; + for (_i444 = 0; _i444 < _size440; ++_i444) { - xfer += this->new_parts[_i399].read(iprot); + xfer += this->new_parts[_i444].read(iprot); } xfer += iprot->readListEnd(); } @@ -5751,10 +5751,10 @@ uint32_t ThriftHiveMetastore_add_partitions_args::write(::apache::thrift::protoc 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 _iter400; - for (_iter400 = this->new_parts.begin(); _iter400 != this->new_parts.end(); ++_iter400) + std::vector ::const_iterator _iter445; + for (_iter445 = this->new_parts.begin(); _iter445 != this->new_parts.end(); ++_iter445) { - xfer += (*_iter400).write(oprot); + xfer += (*_iter445).write(oprot); } xfer += oprot->writeListEnd(); } @@ -5772,10 +5772,10 @@ uint32_t ThriftHiveMetastore_add_partitions_pargs::write(::apache::thrift::proto 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 _iter401; - for (_iter401 = (*(this->new_parts)).begin(); _iter401 != (*(this->new_parts)).end(); ++_iter401) + std::vector ::const_iterator _iter446; + for (_iter446 = (*(this->new_parts)).begin(); _iter446 != (*(this->new_parts)).end(); ++_iter446) { - xfer += (*_iter401).write(oprot); + xfer += (*_iter446).write(oprot); } xfer += oprot->writeListEnd(); } @@ -5982,14 +5982,14 @@ uint32_t ThriftHiveMetastore_append_partition_args::read(::apache::thrift::proto if (ftype == ::apache::thrift::protocol::T_LIST) { { this->part_vals.clear(); - uint32_t _size402; - ::apache::thrift::protocol::TType _etype405; - xfer += iprot->readListBegin(_etype405, _size402); - this->part_vals.resize(_size402); - uint32_t _i406; - for (_i406 = 0; _i406 < _size402; ++_i406) + uint32_t _size447; + ::apache::thrift::protocol::TType _etype450; + xfer += iprot->readListBegin(_etype450, _size447); + this->part_vals.resize(_size447); + uint32_t _i451; + for (_i451 = 0; _i451 < _size447; ++_i451) { - xfer += iprot->readString(this->part_vals[_i406]); + xfer += iprot->readString(this->part_vals[_i451]); } xfer += iprot->readListEnd(); } @@ -6025,10 +6025,10 @@ uint32_t ThriftHiveMetastore_append_partition_args::write(::apache::thrift::prot 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 _iter407; - for (_iter407 = this->part_vals.begin(); _iter407 != this->part_vals.end(); ++_iter407) + std::vector ::const_iterator _iter452; + for (_iter452 = this->part_vals.begin(); _iter452 != this->part_vals.end(); ++_iter452) { - xfer += oprot->writeString((*_iter407)); + xfer += oprot->writeString((*_iter452)); } xfer += oprot->writeListEnd(); } @@ -6054,10 +6054,10 @@ uint32_t ThriftHiveMetastore_append_partition_pargs::write(::apache::thrift::pro 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 _iter408; - for (_iter408 = (*(this->part_vals)).begin(); _iter408 != (*(this->part_vals)).end(); ++_iter408) + std::vector ::const_iterator _iter453; + for (_iter453 = (*(this->part_vals)).begin(); _iter453 != (*(this->part_vals)).end(); ++_iter453) { - xfer += oprot->writeString((*_iter408)); + xfer += oprot->writeString((*_iter453)); } xfer += oprot->writeListEnd(); } @@ -6486,14 +6486,14 @@ uint32_t ThriftHiveMetastore_append_partition_with_environment_context_args::rea if (ftype == ::apache::thrift::protocol::T_LIST) { { this->part_vals.clear(); - uint32_t _size409; - ::apache::thrift::protocol::TType _etype412; - xfer += iprot->readListBegin(_etype412, _size409); - this->part_vals.resize(_size409); - uint32_t _i413; - for (_i413 = 0; _i413 < _size409; ++_i413) + uint32_t _size454; + ::apache::thrift::protocol::TType _etype457; + xfer += iprot->readListBegin(_etype457, _size454); + this->part_vals.resize(_size454); + uint32_t _i458; + for (_i458 = 0; _i458 < _size454; ++_i458) { - xfer += iprot->readString(this->part_vals[_i413]); + xfer += iprot->readString(this->part_vals[_i458]); } xfer += iprot->readListEnd(); } @@ -6537,10 +6537,10 @@ uint32_t ThriftHiveMetastore_append_partition_with_environment_context_args::wri 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 _iter414; - for (_iter414 = this->part_vals.begin(); _iter414 != this->part_vals.end(); ++_iter414) + std::vector ::const_iterator _iter459; + for (_iter459 = this->part_vals.begin(); _iter459 != this->part_vals.end(); ++_iter459) { - xfer += oprot->writeString((*_iter414)); + xfer += oprot->writeString((*_iter459)); } xfer += oprot->writeListEnd(); } @@ -6570,10 +6570,10 @@ uint32_t ThriftHiveMetastore_append_partition_with_environment_context_pargs::wr 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 _iter415; - for (_iter415 = (*(this->part_vals)).begin(); _iter415 != (*(this->part_vals)).end(); ++_iter415) + std::vector ::const_iterator _iter460; + for (_iter460 = (*(this->part_vals)).begin(); _iter460 != (*(this->part_vals)).end(); ++_iter460) { - xfer += oprot->writeString((*_iter415)); + xfer += oprot->writeString((*_iter460)); } xfer += oprot->writeListEnd(); } @@ -7308,14 +7308,14 @@ uint32_t ThriftHiveMetastore_drop_partition_args::read(::apache::thrift::protoco if (ftype == ::apache::thrift::protocol::T_LIST) { { this->part_vals.clear(); - uint32_t _size416; - ::apache::thrift::protocol::TType _etype419; - xfer += iprot->readListBegin(_etype419, _size416); - this->part_vals.resize(_size416); - uint32_t _i420; - for (_i420 = 0; _i420 < _size416; ++_i420) + uint32_t _size461; + ::apache::thrift::protocol::TType _etype464; + xfer += iprot->readListBegin(_etype464, _size461); + this->part_vals.resize(_size461); + uint32_t _i465; + for (_i465 = 0; _i465 < _size461; ++_i465) { - xfer += iprot->readString(this->part_vals[_i420]); + xfer += iprot->readString(this->part_vals[_i465]); } xfer += iprot->readListEnd(); } @@ -7359,10 +7359,10 @@ uint32_t ThriftHiveMetastore_drop_partition_args::write(::apache::thrift::protoc 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 _iter421; - for (_iter421 = this->part_vals.begin(); _iter421 != this->part_vals.end(); ++_iter421) + std::vector ::const_iterator _iter466; + for (_iter466 = this->part_vals.begin(); _iter466 != this->part_vals.end(); ++_iter466) { - xfer += oprot->writeString((*_iter421)); + xfer += oprot->writeString((*_iter466)); } xfer += oprot->writeListEnd(); } @@ -7392,10 +7392,10 @@ uint32_t ThriftHiveMetastore_drop_partition_pargs::write(::apache::thrift::proto 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 _iter422; - for (_iter422 = (*(this->part_vals)).begin(); _iter422 != (*(this->part_vals)).end(); ++_iter422) + std::vector ::const_iterator _iter467; + for (_iter467 = (*(this->part_vals)).begin(); _iter467 != (*(this->part_vals)).end(); ++_iter467) { - xfer += oprot->writeString((*_iter422)); + xfer += oprot->writeString((*_iter467)); } xfer += oprot->writeListEnd(); } @@ -7586,14 +7586,14 @@ uint32_t ThriftHiveMetastore_drop_partition_with_environment_context_args::read( if (ftype == ::apache::thrift::protocol::T_LIST) { { this->part_vals.clear(); - uint32_t _size423; - ::apache::thrift::protocol::TType _etype426; - xfer += iprot->readListBegin(_etype426, _size423); - this->part_vals.resize(_size423); - uint32_t _i427; - for (_i427 = 0; _i427 < _size423; ++_i427) + uint32_t _size468; + ::apache::thrift::protocol::TType _etype471; + xfer += iprot->readListBegin(_etype471, _size468); + this->part_vals.resize(_size468); + uint32_t _i472; + for (_i472 = 0; _i472 < _size468; ++_i472) { - xfer += iprot->readString(this->part_vals[_i427]); + xfer += iprot->readString(this->part_vals[_i472]); } xfer += iprot->readListEnd(); } @@ -7645,10 +7645,10 @@ uint32_t ThriftHiveMetastore_drop_partition_with_environment_context_args::write 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 _iter428; - for (_iter428 = this->part_vals.begin(); _iter428 != this->part_vals.end(); ++_iter428) + std::vector ::const_iterator _iter473; + for (_iter473 = this->part_vals.begin(); _iter473 != this->part_vals.end(); ++_iter473) { - xfer += oprot->writeString((*_iter428)); + xfer += oprot->writeString((*_iter473)); } xfer += oprot->writeListEnd(); } @@ -7682,10 +7682,10 @@ uint32_t ThriftHiveMetastore_drop_partition_with_environment_context_pargs::writ 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 _iter429; - for (_iter429 = (*(this->part_vals)).begin(); _iter429 != (*(this->part_vals)).end(); ++_iter429) + std::vector ::const_iterator _iter474; + for (_iter474 = (*(this->part_vals)).begin(); _iter474 != (*(this->part_vals)).end(); ++_iter474) { - xfer += oprot->writeString((*_iter429)); + xfer += oprot->writeString((*_iter474)); } xfer += oprot->writeListEnd(); } @@ -8598,14 +8598,14 @@ uint32_t ThriftHiveMetastore_get_partition_args::read(::apache::thrift::protocol if (ftype == ::apache::thrift::protocol::T_LIST) { { this->part_vals.clear(); - uint32_t _size430; - ::apache::thrift::protocol::TType _etype433; - xfer += iprot->readListBegin(_etype433, _size430); - this->part_vals.resize(_size430); - uint32_t _i434; - for (_i434 = 0; _i434 < _size430; ++_i434) + uint32_t _size475; + ::apache::thrift::protocol::TType _etype478; + xfer += iprot->readListBegin(_etype478, _size475); + this->part_vals.resize(_size475); + uint32_t _i479; + for (_i479 = 0; _i479 < _size475; ++_i479) { - xfer += iprot->readString(this->part_vals[_i434]); + xfer += iprot->readString(this->part_vals[_i479]); } xfer += iprot->readListEnd(); } @@ -8641,10 +8641,10 @@ uint32_t ThriftHiveMetastore_get_partition_args::write(::apache::thrift::protoco 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 _iter435; - for (_iter435 = this->part_vals.begin(); _iter435 != this->part_vals.end(); ++_iter435) + std::vector ::const_iterator _iter480; + for (_iter480 = this->part_vals.begin(); _iter480 != this->part_vals.end(); ++_iter480) { - xfer += oprot->writeString((*_iter435)); + xfer += oprot->writeString((*_iter480)); } xfer += oprot->writeListEnd(); } @@ -8670,10 +8670,10 @@ uint32_t ThriftHiveMetastore_get_partition_pargs::write(::apache::thrift::protoc 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 _iter436; - for (_iter436 = (*(this->part_vals)).begin(); _iter436 != (*(this->part_vals)).end(); ++_iter436) + std::vector ::const_iterator _iter481; + for (_iter481 = (*(this->part_vals)).begin(); _iter481 != (*(this->part_vals)).end(); ++_iter481) { - xfer += oprot->writeString((*_iter436)); + xfer += oprot->writeString((*_iter481)); } xfer += oprot->writeListEnd(); } @@ -8844,17 +8844,17 @@ uint32_t ThriftHiveMetastore_exchange_partition_args::read(::apache::thrift::pro if (ftype == ::apache::thrift::protocol::T_MAP) { { this->partitionSpecs.clear(); - uint32_t _size437; - ::apache::thrift::protocol::TType _ktype438; - ::apache::thrift::protocol::TType _vtype439; - xfer += iprot->readMapBegin(_ktype438, _vtype439, _size437); - uint32_t _i441; - for (_i441 = 0; _i441 < _size437; ++_i441) + uint32_t _size482; + ::apache::thrift::protocol::TType _ktype483; + ::apache::thrift::protocol::TType _vtype484; + xfer += iprot->readMapBegin(_ktype483, _vtype484, _size482); + uint32_t _i486; + for (_i486 = 0; _i486 < _size482; ++_i486) { - std::string _key442; - xfer += iprot->readString(_key442); - std::string& _val443 = this->partitionSpecs[_key442]; - xfer += iprot->readString(_val443); + std::string _key487; + xfer += iprot->readString(_key487); + std::string& _val488 = this->partitionSpecs[_key487]; + xfer += iprot->readString(_val488); } xfer += iprot->readMapEnd(); } @@ -8914,11 +8914,11 @@ uint32_t ThriftHiveMetastore_exchange_partition_args::write(::apache::thrift::pr 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 _iter444; - for (_iter444 = this->partitionSpecs.begin(); _iter444 != this->partitionSpecs.end(); ++_iter444) + std::map ::const_iterator _iter489; + for (_iter489 = this->partitionSpecs.begin(); _iter489 != this->partitionSpecs.end(); ++_iter489) { - xfer += oprot->writeString(_iter444->first); - xfer += oprot->writeString(_iter444->second); + xfer += oprot->writeString(_iter489->first); + xfer += oprot->writeString(_iter489->second); } xfer += oprot->writeMapEnd(); } @@ -8952,11 +8952,11 @@ uint32_t ThriftHiveMetastore_exchange_partition_pargs::write(::apache::thrift::p 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 _iter445; - for (_iter445 = (*(this->partitionSpecs)).begin(); _iter445 != (*(this->partitionSpecs)).end(); ++_iter445) + std::map ::const_iterator _iter490; + for (_iter490 = (*(this->partitionSpecs)).begin(); _iter490 != (*(this->partitionSpecs)).end(); ++_iter490) { - xfer += oprot->writeString(_iter445->first); - xfer += oprot->writeString(_iter445->second); + xfer += oprot->writeString(_iter490->first); + xfer += oprot->writeString(_iter490->second); } xfer += oprot->writeMapEnd(); } @@ -9199,14 +9199,14 @@ uint32_t ThriftHiveMetastore_get_partition_with_auth_args::read(::apache::thrift if (ftype == ::apache::thrift::protocol::T_LIST) { { this->part_vals.clear(); - uint32_t _size446; - ::apache::thrift::protocol::TType _etype449; - xfer += iprot->readListBegin(_etype449, _size446); - this->part_vals.resize(_size446); - uint32_t _i450; - for (_i450 = 0; _i450 < _size446; ++_i450) + uint32_t _size491; + ::apache::thrift::protocol::TType _etype494; + xfer += iprot->readListBegin(_etype494, _size491); + this->part_vals.resize(_size491); + uint32_t _i495; + for (_i495 = 0; _i495 < _size491; ++_i495) { - xfer += iprot->readString(this->part_vals[_i450]); + xfer += iprot->readString(this->part_vals[_i495]); } xfer += iprot->readListEnd(); } @@ -9227,14 +9227,14 @@ uint32_t ThriftHiveMetastore_get_partition_with_auth_args::read(::apache::thrift if (ftype == ::apache::thrift::protocol::T_LIST) { { this->group_names.clear(); - uint32_t _size451; - ::apache::thrift::protocol::TType _etype454; - xfer += iprot->readListBegin(_etype454, _size451); - this->group_names.resize(_size451); - uint32_t _i455; - for (_i455 = 0; _i455 < _size451; ++_i455) + uint32_t _size496; + ::apache::thrift::protocol::TType _etype499; + xfer += iprot->readListBegin(_etype499, _size496); + this->group_names.resize(_size496); + uint32_t _i500; + for (_i500 = 0; _i500 < _size496; ++_i500) { - xfer += iprot->readString(this->group_names[_i455]); + xfer += iprot->readString(this->group_names[_i500]); } xfer += iprot->readListEnd(); } @@ -9270,10 +9270,10 @@ uint32_t ThriftHiveMetastore_get_partition_with_auth_args::write(::apache::thrif 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 _iter456; - for (_iter456 = this->part_vals.begin(); _iter456 != this->part_vals.end(); ++_iter456) + std::vector ::const_iterator _iter501; + for (_iter501 = this->part_vals.begin(); _iter501 != this->part_vals.end(); ++_iter501) { - xfer += oprot->writeString((*_iter456)); + xfer += oprot->writeString((*_iter501)); } xfer += oprot->writeListEnd(); } @@ -9286,10 +9286,10 @@ uint32_t ThriftHiveMetastore_get_partition_with_auth_args::write(::apache::thrif 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 _iter457; - for (_iter457 = this->group_names.begin(); _iter457 != this->group_names.end(); ++_iter457) + std::vector ::const_iterator _iter502; + for (_iter502 = this->group_names.begin(); _iter502 != this->group_names.end(); ++_iter502) { - xfer += oprot->writeString((*_iter457)); + xfer += oprot->writeString((*_iter502)); } xfer += oprot->writeListEnd(); } @@ -9315,10 +9315,10 @@ uint32_t ThriftHiveMetastore_get_partition_with_auth_pargs::write(::apache::thri 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 _iter458; - for (_iter458 = (*(this->part_vals)).begin(); _iter458 != (*(this->part_vals)).end(); ++_iter458) + std::vector ::const_iterator _iter503; + for (_iter503 = (*(this->part_vals)).begin(); _iter503 != (*(this->part_vals)).end(); ++_iter503) { - xfer += oprot->writeString((*_iter458)); + xfer += oprot->writeString((*_iter503)); } xfer += oprot->writeListEnd(); } @@ -9331,10 +9331,10 @@ uint32_t ThriftHiveMetastore_get_partition_with_auth_pargs::write(::apache::thri 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 _iter459; - for (_iter459 = (*(this->group_names)).begin(); _iter459 != (*(this->group_names)).end(); ++_iter459) + std::vector ::const_iterator _iter504; + for (_iter504 = (*(this->group_names)).begin(); _iter504 != (*(this->group_names)).end(); ++_iter504) { - xfer += oprot->writeString((*_iter459)); + xfer += oprot->writeString((*_iter504)); } xfer += oprot->writeListEnd(); } @@ -9837,14 +9837,14 @@ uint32_t ThriftHiveMetastore_get_partitions_result::read(::apache::thrift::proto if (ftype == ::apache::thrift::protocol::T_LIST) { { this->success.clear(); - uint32_t _size460; - ::apache::thrift::protocol::TType _etype463; - xfer += iprot->readListBegin(_etype463, _size460); - this->success.resize(_size460); - uint32_t _i464; - for (_i464 = 0; _i464 < _size460; ++_i464) + uint32_t _size505; + ::apache::thrift::protocol::TType _etype508; + xfer += iprot->readListBegin(_etype508, _size505); + this->success.resize(_size505); + uint32_t _i509; + for (_i509 = 0; _i509 < _size505; ++_i509) { - xfer += this->success[_i464].read(iprot); + xfer += this->success[_i509].read(iprot); } xfer += iprot->readListEnd(); } @@ -9891,10 +9891,10 @@ uint32_t ThriftHiveMetastore_get_partitions_result::write(::apache::thrift::prot 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 _iter465; - for (_iter465 = this->success.begin(); _iter465 != this->success.end(); ++_iter465) + std::vector ::const_iterator _iter510; + for (_iter510 = this->success.begin(); _iter510 != this->success.end(); ++_iter510) { - xfer += (*_iter465).write(oprot); + xfer += (*_iter510).write(oprot); } xfer += oprot->writeListEnd(); } @@ -9937,14 +9937,14 @@ uint32_t ThriftHiveMetastore_get_partitions_presult::read(::apache::thrift::prot if (ftype == ::apache::thrift::protocol::T_LIST) { { (*(this->success)).clear(); - uint32_t _size466; - ::apache::thrift::protocol::TType _etype469; - xfer += iprot->readListBegin(_etype469, _size466); - (*(this->success)).resize(_size466); - uint32_t _i470; - for (_i470 = 0; _i470 < _size466; ++_i470) + uint32_t _size511; + ::apache::thrift::protocol::TType _etype514; + xfer += iprot->readListBegin(_etype514, _size511); + (*(this->success)).resize(_size511); + uint32_t _i515; + for (_i515 = 0; _i515 < _size511; ++_i515) { - xfer += (*(this->success))[_i470].read(iprot); + xfer += (*(this->success))[_i515].read(iprot); } xfer += iprot->readListEnd(); } @@ -10037,14 +10037,14 @@ uint32_t ThriftHiveMetastore_get_partitions_with_auth_args::read(::apache::thrif if (ftype == ::apache::thrift::protocol::T_LIST) { { this->group_names.clear(); - uint32_t _size471; - ::apache::thrift::protocol::TType _etype474; - xfer += iprot->readListBegin(_etype474, _size471); - this->group_names.resize(_size471); - uint32_t _i475; - for (_i475 = 0; _i475 < _size471; ++_i475) + uint32_t _size516; + ::apache::thrift::protocol::TType _etype519; + xfer += iprot->readListBegin(_etype519, _size516); + this->group_names.resize(_size516); + uint32_t _i520; + for (_i520 = 0; _i520 < _size516; ++_i520) { - xfer += iprot->readString(this->group_names[_i475]); + xfer += iprot->readString(this->group_names[_i520]); } xfer += iprot->readListEnd(); } @@ -10088,10 +10088,10 @@ uint32_t ThriftHiveMetastore_get_partitions_with_auth_args::write(::apache::thri 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 _iter476; - for (_iter476 = this->group_names.begin(); _iter476 != this->group_names.end(); ++_iter476) + std::vector ::const_iterator _iter521; + for (_iter521 = this->group_names.begin(); _iter521 != this->group_names.end(); ++_iter521) { - xfer += oprot->writeString((*_iter476)); + xfer += oprot->writeString((*_iter521)); } xfer += oprot->writeListEnd(); } @@ -10125,10 +10125,10 @@ uint32_t ThriftHiveMetastore_get_partitions_with_auth_pargs::write(::apache::thr 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 _iter477; - for (_iter477 = (*(this->group_names)).begin(); _iter477 != (*(this->group_names)).end(); ++_iter477) + std::vector ::const_iterator _iter522; + for (_iter522 = (*(this->group_names)).begin(); _iter522 != (*(this->group_names)).end(); ++_iter522) { - xfer += oprot->writeString((*_iter477)); + xfer += oprot->writeString((*_iter522)); } xfer += oprot->writeListEnd(); } @@ -10163,14 +10163,14 @@ uint32_t ThriftHiveMetastore_get_partitions_with_auth_result::read(::apache::thr if (ftype == ::apache::thrift::protocol::T_LIST) { { this->success.clear(); - uint32_t _size478; - ::apache::thrift::protocol::TType _etype481; - xfer += iprot->readListBegin(_etype481, _size478); - this->success.resize(_size478); - uint32_t _i482; - for (_i482 = 0; _i482 < _size478; ++_i482) + uint32_t _size523; + ::apache::thrift::protocol::TType _etype526; + xfer += iprot->readListBegin(_etype526, _size523); + this->success.resize(_size523); + uint32_t _i527; + for (_i527 = 0; _i527 < _size523; ++_i527) { - xfer += this->success[_i482].read(iprot); + xfer += this->success[_i527].read(iprot); } xfer += iprot->readListEnd(); } @@ -10217,10 +10217,10 @@ uint32_t ThriftHiveMetastore_get_partitions_with_auth_result::write(::apache::th 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 _iter483; - for (_iter483 = this->success.begin(); _iter483 != this->success.end(); ++_iter483) + std::vector ::const_iterator _iter528; + for (_iter528 = this->success.begin(); _iter528 != this->success.end(); ++_iter528) { - xfer += (*_iter483).write(oprot); + xfer += (*_iter528).write(oprot); } xfer += oprot->writeListEnd(); } @@ -10263,14 +10263,14 @@ uint32_t ThriftHiveMetastore_get_partitions_with_auth_presult::read(::apache::th if (ftype == ::apache::thrift::protocol::T_LIST) { { (*(this->success)).clear(); - uint32_t _size484; - ::apache::thrift::protocol::TType _etype487; - xfer += iprot->readListBegin(_etype487, _size484); - (*(this->success)).resize(_size484); - uint32_t _i488; - for (_i488 = 0; _i488 < _size484; ++_i488) + uint32_t _size529; + ::apache::thrift::protocol::TType _etype532; + xfer += iprot->readListBegin(_etype532, _size529); + (*(this->success)).resize(_size529); + uint32_t _i533; + for (_i533 = 0; _i533 < _size529; ++_i533) { - xfer += (*(this->success))[_i488].read(iprot); + xfer += (*(this->success))[_i533].read(iprot); } xfer += iprot->readListEnd(); } @@ -10429,14 +10429,14 @@ uint32_t ThriftHiveMetastore_get_partition_names_result::read(::apache::thrift:: if (ftype == ::apache::thrift::protocol::T_LIST) { { this->success.clear(); - uint32_t _size489; - ::apache::thrift::protocol::TType _etype492; - xfer += iprot->readListBegin(_etype492, _size489); - this->success.resize(_size489); - uint32_t _i493; - for (_i493 = 0; _i493 < _size489; ++_i493) + uint32_t _size534; + ::apache::thrift::protocol::TType _etype537; + xfer += iprot->readListBegin(_etype537, _size534); + this->success.resize(_size534); + uint32_t _i538; + for (_i538 = 0; _i538 < _size534; ++_i538) { - xfer += iprot->readString(this->success[_i493]); + xfer += iprot->readString(this->success[_i538]); } xfer += iprot->readListEnd(); } @@ -10475,10 +10475,10 @@ uint32_t ThriftHiveMetastore_get_partition_names_result::write(::apache::thrift: 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 _iter494; - for (_iter494 = this->success.begin(); _iter494 != this->success.end(); ++_iter494) + std::vector ::const_iterator _iter539; + for (_iter539 = this->success.begin(); _iter539 != this->success.end(); ++_iter539) { - xfer += oprot->writeString((*_iter494)); + xfer += oprot->writeString((*_iter539)); } xfer += oprot->writeListEnd(); } @@ -10517,14 +10517,14 @@ uint32_t ThriftHiveMetastore_get_partition_names_presult::read(::apache::thrift: if (ftype == ::apache::thrift::protocol::T_LIST) { { (*(this->success)).clear(); - uint32_t _size495; - ::apache::thrift::protocol::TType _etype498; - xfer += iprot->readListBegin(_etype498, _size495); - (*(this->success)).resize(_size495); - uint32_t _i499; - for (_i499 = 0; _i499 < _size495; ++_i499) + uint32_t _size540; + ::apache::thrift::protocol::TType _etype543; + xfer += iprot->readListBegin(_etype543, _size540); + (*(this->success)).resize(_size540); + uint32_t _i544; + for (_i544 = 0; _i544 < _size540; ++_i544) { - xfer += iprot->readString((*(this->success))[_i499]); + xfer += iprot->readString((*(this->success))[_i544]); } xfer += iprot->readListEnd(); } @@ -10593,14 +10593,14 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_args::read(::apache::thrift::prot if (ftype == ::apache::thrift::protocol::T_LIST) { { this->part_vals.clear(); - uint32_t _size500; - ::apache::thrift::protocol::TType _etype503; - xfer += iprot->readListBegin(_etype503, _size500); - this->part_vals.resize(_size500); - uint32_t _i504; - for (_i504 = 0; _i504 < _size500; ++_i504) + uint32_t _size545; + ::apache::thrift::protocol::TType _etype548; + xfer += iprot->readListBegin(_etype548, _size545); + this->part_vals.resize(_size545); + uint32_t _i549; + for (_i549 = 0; _i549 < _size545; ++_i549) { - xfer += iprot->readString(this->part_vals[_i504]); + xfer += iprot->readString(this->part_vals[_i549]); } xfer += iprot->readListEnd(); } @@ -10644,10 +10644,10 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_args::write(::apache::thrift::pro 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 _iter505; - for (_iter505 = this->part_vals.begin(); _iter505 != this->part_vals.end(); ++_iter505) + std::vector ::const_iterator _iter550; + for (_iter550 = this->part_vals.begin(); _iter550 != this->part_vals.end(); ++_iter550) { - xfer += oprot->writeString((*_iter505)); + xfer += oprot->writeString((*_iter550)); } xfer += oprot->writeListEnd(); } @@ -10677,10 +10677,10 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_pargs::write(::apache::thrift::pr 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 _iter506; - for (_iter506 = (*(this->part_vals)).begin(); _iter506 != (*(this->part_vals)).end(); ++_iter506) + std::vector ::const_iterator _iter551; + for (_iter551 = (*(this->part_vals)).begin(); _iter551 != (*(this->part_vals)).end(); ++_iter551) { - xfer += oprot->writeString((*_iter506)); + xfer += oprot->writeString((*_iter551)); } xfer += oprot->writeListEnd(); } @@ -10719,14 +10719,14 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_result::read(::apache::thrift::pr if (ftype == ::apache::thrift::protocol::T_LIST) { { this->success.clear(); - uint32_t _size507; - ::apache::thrift::protocol::TType _etype510; - xfer += iprot->readListBegin(_etype510, _size507); - this->success.resize(_size507); - uint32_t _i511; - for (_i511 = 0; _i511 < _size507; ++_i511) + uint32_t _size552; + ::apache::thrift::protocol::TType _etype555; + xfer += iprot->readListBegin(_etype555, _size552); + this->success.resize(_size552); + uint32_t _i556; + for (_i556 = 0; _i556 < _size552; ++_i556) { - xfer += this->success[_i511].read(iprot); + xfer += this->success[_i556].read(iprot); } xfer += iprot->readListEnd(); } @@ -10773,10 +10773,10 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_result::write(::apache::thrift::p 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 _iter512; - for (_iter512 = this->success.begin(); _iter512 != this->success.end(); ++_iter512) + std::vector ::const_iterator _iter557; + for (_iter557 = this->success.begin(); _iter557 != this->success.end(); ++_iter557) { - xfer += (*_iter512).write(oprot); + xfer += (*_iter557).write(oprot); } xfer += oprot->writeListEnd(); } @@ -10819,14 +10819,14 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_presult::read(::apache::thrift::p if (ftype == ::apache::thrift::protocol::T_LIST) { { (*(this->success)).clear(); - uint32_t _size513; - ::apache::thrift::protocol::TType _etype516; - xfer += iprot->readListBegin(_etype516, _size513); - (*(this->success)).resize(_size513); - uint32_t _i517; - for (_i517 = 0; _i517 < _size513; ++_i517) + uint32_t _size558; + ::apache::thrift::protocol::TType _etype561; + xfer += iprot->readListBegin(_etype561, _size558); + (*(this->success)).resize(_size558); + uint32_t _i562; + for (_i562 = 0; _i562 < _size558; ++_i562) { - xfer += (*(this->success))[_i517].read(iprot); + xfer += (*(this->success))[_i562].read(iprot); } xfer += iprot->readListEnd(); } @@ -10903,14 +10903,14 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_with_auth_args::read(::apache::th if (ftype == ::apache::thrift::protocol::T_LIST) { { this->part_vals.clear(); - uint32_t _size518; - ::apache::thrift::protocol::TType _etype521; - xfer += iprot->readListBegin(_etype521, _size518); - this->part_vals.resize(_size518); - uint32_t _i522; - for (_i522 = 0; _i522 < _size518; ++_i522) + uint32_t _size563; + ::apache::thrift::protocol::TType _etype566; + xfer += iprot->readListBegin(_etype566, _size563); + this->part_vals.resize(_size563); + uint32_t _i567; + for (_i567 = 0; _i567 < _size563; ++_i567) { - xfer += iprot->readString(this->part_vals[_i522]); + xfer += iprot->readString(this->part_vals[_i567]); } xfer += iprot->readListEnd(); } @@ -10939,14 +10939,14 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_with_auth_args::read(::apache::th if (ftype == ::apache::thrift::protocol::T_LIST) { { this->group_names.clear(); - uint32_t _size523; - ::apache::thrift::protocol::TType _etype526; - xfer += iprot->readListBegin(_etype526, _size523); - this->group_names.resize(_size523); - uint32_t _i527; - for (_i527 = 0; _i527 < _size523; ++_i527) + uint32_t _size568; + ::apache::thrift::protocol::TType _etype571; + xfer += iprot->readListBegin(_etype571, _size568); + this->group_names.resize(_size568); + uint32_t _i572; + for (_i572 = 0; _i572 < _size568; ++_i572) { - xfer += iprot->readString(this->group_names[_i527]); + xfer += iprot->readString(this->group_names[_i572]); } xfer += iprot->readListEnd(); } @@ -10982,10 +10982,10 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_with_auth_args::write(::apache::t 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 _iter528; - for (_iter528 = this->part_vals.begin(); _iter528 != this->part_vals.end(); ++_iter528) + std::vector ::const_iterator _iter573; + for (_iter573 = this->part_vals.begin(); _iter573 != this->part_vals.end(); ++_iter573) { - xfer += oprot->writeString((*_iter528)); + xfer += oprot->writeString((*_iter573)); } xfer += oprot->writeListEnd(); } @@ -11002,10 +11002,10 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_with_auth_args::write(::apache::t 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 _iter529; - for (_iter529 = this->group_names.begin(); _iter529 != this->group_names.end(); ++_iter529) + std::vector ::const_iterator _iter574; + for (_iter574 = this->group_names.begin(); _iter574 != this->group_names.end(); ++_iter574) { - xfer += oprot->writeString((*_iter529)); + xfer += oprot->writeString((*_iter574)); } xfer += oprot->writeListEnd(); } @@ -11031,10 +11031,10 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_with_auth_pargs::write(::apache:: 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 _iter530; - for (_iter530 = (*(this->part_vals)).begin(); _iter530 != (*(this->part_vals)).end(); ++_iter530) + std::vector ::const_iterator _iter575; + for (_iter575 = (*(this->part_vals)).begin(); _iter575 != (*(this->part_vals)).end(); ++_iter575) { - xfer += oprot->writeString((*_iter530)); + xfer += oprot->writeString((*_iter575)); } xfer += oprot->writeListEnd(); } @@ -11051,10 +11051,10 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_with_auth_pargs::write(::apache:: 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 _iter531; - for (_iter531 = (*(this->group_names)).begin(); _iter531 != (*(this->group_names)).end(); ++_iter531) + std::vector ::const_iterator _iter576; + for (_iter576 = (*(this->group_names)).begin(); _iter576 != (*(this->group_names)).end(); ++_iter576) { - xfer += oprot->writeString((*_iter531)); + xfer += oprot->writeString((*_iter576)); } xfer += oprot->writeListEnd(); } @@ -11089,14 +11089,14 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_with_auth_result::read(::apache:: if (ftype == ::apache::thrift::protocol::T_LIST) { { this->success.clear(); - uint32_t _size532; - ::apache::thrift::protocol::TType _etype535; - xfer += iprot->readListBegin(_etype535, _size532); - this->success.resize(_size532); - uint32_t _i536; - for (_i536 = 0; _i536 < _size532; ++_i536) + uint32_t _size577; + ::apache::thrift::protocol::TType _etype580; + xfer += iprot->readListBegin(_etype580, _size577); + this->success.resize(_size577); + uint32_t _i581; + for (_i581 = 0; _i581 < _size577; ++_i581) { - xfer += this->success[_i536].read(iprot); + xfer += this->success[_i581].read(iprot); } xfer += iprot->readListEnd(); } @@ -11143,10 +11143,10 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_with_auth_result::write(::apache: 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 _iter537; - for (_iter537 = this->success.begin(); _iter537 != this->success.end(); ++_iter537) + std::vector ::const_iterator _iter582; + for (_iter582 = this->success.begin(); _iter582 != this->success.end(); ++_iter582) { - xfer += (*_iter537).write(oprot); + xfer += (*_iter582).write(oprot); } xfer += oprot->writeListEnd(); } @@ -11189,14 +11189,14 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_with_auth_presult::read(::apache: if (ftype == ::apache::thrift::protocol::T_LIST) { { (*(this->success)).clear(); - uint32_t _size538; - ::apache::thrift::protocol::TType _etype541; - xfer += iprot->readListBegin(_etype541, _size538); - (*(this->success)).resize(_size538); - uint32_t _i542; - for (_i542 = 0; _i542 < _size538; ++_i542) + uint32_t _size583; + ::apache::thrift::protocol::TType _etype586; + xfer += iprot->readListBegin(_etype586, _size583); + (*(this->success)).resize(_size583); + uint32_t _i587; + for (_i587 = 0; _i587 < _size583; ++_i587) { - xfer += (*(this->success))[_i542].read(iprot); + xfer += (*(this->success))[_i587].read(iprot); } xfer += iprot->readListEnd(); } @@ -11273,14 +11273,14 @@ uint32_t ThriftHiveMetastore_get_partition_names_ps_args::read(::apache::thrift: if (ftype == ::apache::thrift::protocol::T_LIST) { { this->part_vals.clear(); - uint32_t _size543; - ::apache::thrift::protocol::TType _etype546; - xfer += iprot->readListBegin(_etype546, _size543); - this->part_vals.resize(_size543); - uint32_t _i547; - for (_i547 = 0; _i547 < _size543; ++_i547) + uint32_t _size588; + ::apache::thrift::protocol::TType _etype591; + xfer += iprot->readListBegin(_etype591, _size588); + this->part_vals.resize(_size588); + uint32_t _i592; + for (_i592 = 0; _i592 < _size588; ++_i592) { - xfer += iprot->readString(this->part_vals[_i547]); + xfer += iprot->readString(this->part_vals[_i592]); } xfer += iprot->readListEnd(); } @@ -11324,10 +11324,10 @@ uint32_t ThriftHiveMetastore_get_partition_names_ps_args::write(::apache::thrift 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 _iter548; - for (_iter548 = this->part_vals.begin(); _iter548 != this->part_vals.end(); ++_iter548) + std::vector ::const_iterator _iter593; + for (_iter593 = this->part_vals.begin(); _iter593 != this->part_vals.end(); ++_iter593) { - xfer += oprot->writeString((*_iter548)); + xfer += oprot->writeString((*_iter593)); } xfer += oprot->writeListEnd(); } @@ -11357,10 +11357,10 @@ uint32_t ThriftHiveMetastore_get_partition_names_ps_pargs::write(::apache::thrif 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 _iter549; - for (_iter549 = (*(this->part_vals)).begin(); _iter549 != (*(this->part_vals)).end(); ++_iter549) + std::vector ::const_iterator _iter594; + for (_iter594 = (*(this->part_vals)).begin(); _iter594 != (*(this->part_vals)).end(); ++_iter594) { - xfer += oprot->writeString((*_iter549)); + xfer += oprot->writeString((*_iter594)); } xfer += oprot->writeListEnd(); } @@ -11399,14 +11399,14 @@ uint32_t ThriftHiveMetastore_get_partition_names_ps_result::read(::apache::thrif if (ftype == ::apache::thrift::protocol::T_LIST) { { this->success.clear(); - uint32_t _size550; - ::apache::thrift::protocol::TType _etype553; - xfer += iprot->readListBegin(_etype553, _size550); - this->success.resize(_size550); - uint32_t _i554; - for (_i554 = 0; _i554 < _size550; ++_i554) + uint32_t _size595; + ::apache::thrift::protocol::TType _etype598; + xfer += iprot->readListBegin(_etype598, _size595); + this->success.resize(_size595); + uint32_t _i599; + for (_i599 = 0; _i599 < _size595; ++_i599) { - xfer += iprot->readString(this->success[_i554]); + xfer += iprot->readString(this->success[_i599]); } xfer += iprot->readListEnd(); } @@ -11453,10 +11453,10 @@ uint32_t ThriftHiveMetastore_get_partition_names_ps_result::write(::apache::thri 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 _iter555; - for (_iter555 = this->success.begin(); _iter555 != this->success.end(); ++_iter555) + std::vector ::const_iterator _iter600; + for (_iter600 = this->success.begin(); _iter600 != this->success.end(); ++_iter600) { - xfer += oprot->writeString((*_iter555)); + xfer += oprot->writeString((*_iter600)); } xfer += oprot->writeListEnd(); } @@ -11499,14 +11499,14 @@ uint32_t ThriftHiveMetastore_get_partition_names_ps_presult::read(::apache::thri if (ftype == ::apache::thrift::protocol::T_LIST) { { (*(this->success)).clear(); - uint32_t _size556; - ::apache::thrift::protocol::TType _etype559; - xfer += iprot->readListBegin(_etype559, _size556); - (*(this->success)).resize(_size556); - uint32_t _i560; - for (_i560 = 0; _i560 < _size556; ++_i560) + uint32_t _size601; + ::apache::thrift::protocol::TType _etype604; + xfer += iprot->readListBegin(_etype604, _size601); + (*(this->success)).resize(_size601); + uint32_t _i605; + for (_i605 = 0; _i605 < _size601; ++_i605) { - xfer += iprot->readString((*(this->success))[_i560]); + xfer += iprot->readString((*(this->success))[_i605]); } xfer += iprot->readListEnd(); } @@ -11681,14 +11681,14 @@ uint32_t ThriftHiveMetastore_get_partitions_by_filter_result::read(::apache::thr if (ftype == ::apache::thrift::protocol::T_LIST) { { this->success.clear(); - uint32_t _size561; - ::apache::thrift::protocol::TType _etype564; - xfer += iprot->readListBegin(_etype564, _size561); - this->success.resize(_size561); - uint32_t _i565; - for (_i565 = 0; _i565 < _size561; ++_i565) + uint32_t _size606; + ::apache::thrift::protocol::TType _etype609; + xfer += iprot->readListBegin(_etype609, _size606); + this->success.resize(_size606); + uint32_t _i610; + for (_i610 = 0; _i610 < _size606; ++_i610) { - xfer += this->success[_i565].read(iprot); + xfer += this->success[_i610].read(iprot); } xfer += iprot->readListEnd(); } @@ -11735,10 +11735,10 @@ uint32_t ThriftHiveMetastore_get_partitions_by_filter_result::write(::apache::th 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 _iter566; - for (_iter566 = this->success.begin(); _iter566 != this->success.end(); ++_iter566) + std::vector ::const_iterator _iter611; + for (_iter611 = this->success.begin(); _iter611 != this->success.end(); ++_iter611) { - xfer += (*_iter566).write(oprot); + xfer += (*_iter611).write(oprot); } xfer += oprot->writeListEnd(); } @@ -11781,14 +11781,14 @@ uint32_t ThriftHiveMetastore_get_partitions_by_filter_presult::read(::apache::th if (ftype == ::apache::thrift::protocol::T_LIST) { { (*(this->success)).clear(); - uint32_t _size567; - ::apache::thrift::protocol::TType _etype570; - xfer += iprot->readListBegin(_etype570, _size567); - (*(this->success)).resize(_size567); - uint32_t _i571; - for (_i571 = 0; _i571 < _size567; ++_i571) + uint32_t _size612; + ::apache::thrift::protocol::TType _etype615; + xfer += iprot->readListBegin(_etype615, _size612); + (*(this->success)).resize(_size612); + uint32_t _i616; + for (_i616 = 0; _i616 < _size612; ++_i616) { - xfer += (*(this->success))[_i571].read(iprot); + xfer += (*(this->success))[_i616].read(iprot); } xfer += iprot->readListEnd(); } @@ -12067,14 +12067,14 @@ uint32_t ThriftHiveMetastore_get_partitions_by_names_args::read(::apache::thrift if (ftype == ::apache::thrift::protocol::T_LIST) { { this->names.clear(); - uint32_t _size572; - ::apache::thrift::protocol::TType _etype575; - xfer += iprot->readListBegin(_etype575, _size572); - this->names.resize(_size572); - uint32_t _i576; - for (_i576 = 0; _i576 < _size572; ++_i576) + uint32_t _size617; + ::apache::thrift::protocol::TType _etype620; + xfer += iprot->readListBegin(_etype620, _size617); + this->names.resize(_size617); + uint32_t _i621; + for (_i621 = 0; _i621 < _size617; ++_i621) { - xfer += iprot->readString(this->names[_i576]); + xfer += iprot->readString(this->names[_i621]); } xfer += iprot->readListEnd(); } @@ -12110,10 +12110,10 @@ uint32_t ThriftHiveMetastore_get_partitions_by_names_args::write(::apache::thrif 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 _iter577; - for (_iter577 = this->names.begin(); _iter577 != this->names.end(); ++_iter577) + std::vector ::const_iterator _iter622; + for (_iter622 = this->names.begin(); _iter622 != this->names.end(); ++_iter622) { - xfer += oprot->writeString((*_iter577)); + xfer += oprot->writeString((*_iter622)); } xfer += oprot->writeListEnd(); } @@ -12139,10 +12139,10 @@ uint32_t ThriftHiveMetastore_get_partitions_by_names_pargs::write(::apache::thri 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 _iter578; - for (_iter578 = (*(this->names)).begin(); _iter578 != (*(this->names)).end(); ++_iter578) + std::vector ::const_iterator _iter623; + for (_iter623 = (*(this->names)).begin(); _iter623 != (*(this->names)).end(); ++_iter623) { - xfer += oprot->writeString((*_iter578)); + xfer += oprot->writeString((*_iter623)); } xfer += oprot->writeListEnd(); } @@ -12177,14 +12177,14 @@ uint32_t ThriftHiveMetastore_get_partitions_by_names_result::read(::apache::thri if (ftype == ::apache::thrift::protocol::T_LIST) { { this->success.clear(); - uint32_t _size579; - ::apache::thrift::protocol::TType _etype582; - xfer += iprot->readListBegin(_etype582, _size579); - this->success.resize(_size579); - uint32_t _i583; - for (_i583 = 0; _i583 < _size579; ++_i583) + uint32_t _size624; + ::apache::thrift::protocol::TType _etype627; + xfer += iprot->readListBegin(_etype627, _size624); + this->success.resize(_size624); + uint32_t _i628; + for (_i628 = 0; _i628 < _size624; ++_i628) { - xfer += this->success[_i583].read(iprot); + xfer += this->success[_i628].read(iprot); } xfer += iprot->readListEnd(); } @@ -12231,10 +12231,10 @@ uint32_t ThriftHiveMetastore_get_partitions_by_names_result::write(::apache::thr 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 _iter584; - for (_iter584 = this->success.begin(); _iter584 != this->success.end(); ++_iter584) + std::vector ::const_iterator _iter629; + for (_iter629 = this->success.begin(); _iter629 != this->success.end(); ++_iter629) { - xfer += (*_iter584).write(oprot); + xfer += (*_iter629).write(oprot); } xfer += oprot->writeListEnd(); } @@ -12277,14 +12277,14 @@ uint32_t ThriftHiveMetastore_get_partitions_by_names_presult::read(::apache::thr if (ftype == ::apache::thrift::protocol::T_LIST) { { (*(this->success)).clear(); - uint32_t _size585; - ::apache::thrift::protocol::TType _etype588; - xfer += iprot->readListBegin(_etype588, _size585); - (*(this->success)).resize(_size585); - uint32_t _i589; - for (_i589 = 0; _i589 < _size585; ++_i589) + uint32_t _size630; + ::apache::thrift::protocol::TType _etype633; + xfer += iprot->readListBegin(_etype633, _size630); + (*(this->success)).resize(_size630); + uint32_t _i634; + for (_i634 = 0; _i634 < _size630; ++_i634) { - xfer += (*(this->success))[_i589].read(iprot); + xfer += (*(this->success))[_i634].read(iprot); } xfer += iprot->readListEnd(); } @@ -12575,14 +12575,14 @@ uint32_t ThriftHiveMetastore_alter_partitions_args::read(::apache::thrift::proto if (ftype == ::apache::thrift::protocol::T_LIST) { { this->new_parts.clear(); - uint32_t _size590; - ::apache::thrift::protocol::TType _etype593; - xfer += iprot->readListBegin(_etype593, _size590); - this->new_parts.resize(_size590); - uint32_t _i594; - for (_i594 = 0; _i594 < _size590; ++_i594) + uint32_t _size635; + ::apache::thrift::protocol::TType _etype638; + xfer += iprot->readListBegin(_etype638, _size635); + this->new_parts.resize(_size635); + uint32_t _i639; + for (_i639 = 0; _i639 < _size635; ++_i639) { - xfer += this->new_parts[_i594].read(iprot); + xfer += this->new_parts[_i639].read(iprot); } xfer += iprot->readListEnd(); } @@ -12618,10 +12618,10 @@ uint32_t ThriftHiveMetastore_alter_partitions_args::write(::apache::thrift::prot 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 _iter595; - for (_iter595 = this->new_parts.begin(); _iter595 != this->new_parts.end(); ++_iter595) + std::vector ::const_iterator _iter640; + for (_iter640 = this->new_parts.begin(); _iter640 != this->new_parts.end(); ++_iter640) { - xfer += (*_iter595).write(oprot); + xfer += (*_iter640).write(oprot); } xfer += oprot->writeListEnd(); } @@ -12647,10 +12647,10 @@ uint32_t ThriftHiveMetastore_alter_partitions_pargs::write(::apache::thrift::pro 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 _iter596; - for (_iter596 = (*(this->new_parts)).begin(); _iter596 != (*(this->new_parts)).end(); ++_iter596) + std::vector ::const_iterator _iter641; + for (_iter641 = (*(this->new_parts)).begin(); _iter641 != (*(this->new_parts)).end(); ++_iter641) { - xfer += (*_iter596).write(oprot); + xfer += (*_iter641).write(oprot); } xfer += oprot->writeListEnd(); } @@ -13047,14 +13047,14 @@ uint32_t ThriftHiveMetastore_rename_partition_args::read(::apache::thrift::proto if (ftype == ::apache::thrift::protocol::T_LIST) { { this->part_vals.clear(); - uint32_t _size597; - ::apache::thrift::protocol::TType _etype600; - xfer += iprot->readListBegin(_etype600, _size597); - this->part_vals.resize(_size597); - uint32_t _i601; - for (_i601 = 0; _i601 < _size597; ++_i601) + uint32_t _size642; + ::apache::thrift::protocol::TType _etype645; + xfer += iprot->readListBegin(_etype645, _size642); + this->part_vals.resize(_size642); + uint32_t _i646; + for (_i646 = 0; _i646 < _size642; ++_i646) { - xfer += iprot->readString(this->part_vals[_i601]); + xfer += iprot->readString(this->part_vals[_i646]); } xfer += iprot->readListEnd(); } @@ -13098,10 +13098,10 @@ uint32_t ThriftHiveMetastore_rename_partition_args::write(::apache::thrift::prot 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 _iter602; - for (_iter602 = this->part_vals.begin(); _iter602 != this->part_vals.end(); ++_iter602) + std::vector ::const_iterator _iter647; + for (_iter647 = this->part_vals.begin(); _iter647 != this->part_vals.end(); ++_iter647) { - xfer += oprot->writeString((*_iter602)); + xfer += oprot->writeString((*_iter647)); } xfer += oprot->writeListEnd(); } @@ -13131,10 +13131,10 @@ uint32_t ThriftHiveMetastore_rename_partition_pargs::write(::apache::thrift::pro 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 _iter603; - for (_iter603 = (*(this->part_vals)).begin(); _iter603 != (*(this->part_vals)).end(); ++_iter603) + std::vector ::const_iterator _iter648; + for (_iter648 = (*(this->part_vals)).begin(); _iter648 != (*(this->part_vals)).end(); ++_iter648) { - xfer += oprot->writeString((*_iter603)); + xfer += oprot->writeString((*_iter648)); } xfer += oprot->writeListEnd(); } @@ -13289,14 +13289,14 @@ uint32_t ThriftHiveMetastore_partition_name_has_valid_characters_args::read(::ap if (ftype == ::apache::thrift::protocol::T_LIST) { { this->part_vals.clear(); - uint32_t _size604; - ::apache::thrift::protocol::TType _etype607; - xfer += iprot->readListBegin(_etype607, _size604); - this->part_vals.resize(_size604); - uint32_t _i608; - for (_i608 = 0; _i608 < _size604; ++_i608) + uint32_t _size649; + ::apache::thrift::protocol::TType _etype652; + xfer += iprot->readListBegin(_etype652, _size649); + this->part_vals.resize(_size649); + uint32_t _i653; + for (_i653 = 0; _i653 < _size649; ++_i653) { - xfer += iprot->readString(this->part_vals[_i608]); + xfer += iprot->readString(this->part_vals[_i653]); } xfer += iprot->readListEnd(); } @@ -13332,10 +13332,10 @@ uint32_t ThriftHiveMetastore_partition_name_has_valid_characters_args::write(::a 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 _iter609; - for (_iter609 = this->part_vals.begin(); _iter609 != this->part_vals.end(); ++_iter609) + std::vector ::const_iterator _iter654; + for (_iter654 = this->part_vals.begin(); _iter654 != this->part_vals.end(); ++_iter654) { - xfer += oprot->writeString((*_iter609)); + xfer += oprot->writeString((*_iter654)); } xfer += oprot->writeListEnd(); } @@ -13357,10 +13357,10 @@ uint32_t ThriftHiveMetastore_partition_name_has_valid_characters_pargs::write(:: 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 _iter610; - for (_iter610 = (*(this->part_vals)).begin(); _iter610 != (*(this->part_vals)).end(); ++_iter610) + std::vector ::const_iterator _iter655; + for (_iter655 = (*(this->part_vals)).begin(); _iter655 != (*(this->part_vals)).end(); ++_iter655) { - xfer += oprot->writeString((*_iter610)); + xfer += oprot->writeString((*_iter655)); } xfer += oprot->writeListEnd(); } @@ -13779,14 +13779,14 @@ uint32_t ThriftHiveMetastore_partition_name_to_vals_result::read(::apache::thrif if (ftype == ::apache::thrift::protocol::T_LIST) { { this->success.clear(); - uint32_t _size611; - ::apache::thrift::protocol::TType _etype614; - xfer += iprot->readListBegin(_etype614, _size611); - this->success.resize(_size611); - uint32_t _i615; - for (_i615 = 0; _i615 < _size611; ++_i615) + uint32_t _size656; + ::apache::thrift::protocol::TType _etype659; + xfer += iprot->readListBegin(_etype659, _size656); + this->success.resize(_size656); + uint32_t _i660; + for (_i660 = 0; _i660 < _size656; ++_i660) { - xfer += iprot->readString(this->success[_i615]); + xfer += iprot->readString(this->success[_i660]); } xfer += iprot->readListEnd(); } @@ -13825,10 +13825,10 @@ uint32_t ThriftHiveMetastore_partition_name_to_vals_result::write(::apache::thri 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 _iter616; - for (_iter616 = this->success.begin(); _iter616 != this->success.end(); ++_iter616) + std::vector ::const_iterator _iter661; + for (_iter661 = this->success.begin(); _iter661 != this->success.end(); ++_iter661) { - xfer += oprot->writeString((*_iter616)); + xfer += oprot->writeString((*_iter661)); } xfer += oprot->writeListEnd(); } @@ -13867,14 +13867,14 @@ uint32_t ThriftHiveMetastore_partition_name_to_vals_presult::read(::apache::thri if (ftype == ::apache::thrift::protocol::T_LIST) { { (*(this->success)).clear(); - uint32_t _size617; - ::apache::thrift::protocol::TType _etype620; - xfer += iprot->readListBegin(_etype620, _size617); - (*(this->success)).resize(_size617); - uint32_t _i621; - for (_i621 = 0; _i621 < _size617; ++_i621) + uint32_t _size662; + ::apache::thrift::protocol::TType _etype665; + xfer += iprot->readListBegin(_etype665, _size662); + (*(this->success)).resize(_size662); + uint32_t _i666; + for (_i666 = 0; _i666 < _size662; ++_i666) { - xfer += iprot->readString((*(this->success))[_i621]); + xfer += iprot->readString((*(this->success))[_i666]); } xfer += iprot->readListEnd(); } @@ -13993,17 +13993,17 @@ uint32_t ThriftHiveMetastore_partition_name_to_spec_result::read(::apache::thrif if (ftype == ::apache::thrift::protocol::T_MAP) { { this->success.clear(); - uint32_t _size622; - ::apache::thrift::protocol::TType _ktype623; - ::apache::thrift::protocol::TType _vtype624; - xfer += iprot->readMapBegin(_ktype623, _vtype624, _size622); - uint32_t _i626; - for (_i626 = 0; _i626 < _size622; ++_i626) + uint32_t _size667; + ::apache::thrift::protocol::TType _ktype668; + ::apache::thrift::protocol::TType _vtype669; + xfer += iprot->readMapBegin(_ktype668, _vtype669, _size667); + uint32_t _i671; + for (_i671 = 0; _i671 < _size667; ++_i671) { - std::string _key627; - xfer += iprot->readString(_key627); - std::string& _val628 = this->success[_key627]; - xfer += iprot->readString(_val628); + std::string _key672; + xfer += iprot->readString(_key672); + std::string& _val673 = this->success[_key672]; + xfer += iprot->readString(_val673); } xfer += iprot->readMapEnd(); } @@ -14042,11 +14042,11 @@ uint32_t ThriftHiveMetastore_partition_name_to_spec_result::write(::apache::thri 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 _iter629; - for (_iter629 = this->success.begin(); _iter629 != this->success.end(); ++_iter629) + std::map ::const_iterator _iter674; + for (_iter674 = this->success.begin(); _iter674 != this->success.end(); ++_iter674) { - xfer += oprot->writeString(_iter629->first); - xfer += oprot->writeString(_iter629->second); + xfer += oprot->writeString(_iter674->first); + xfer += oprot->writeString(_iter674->second); } xfer += oprot->writeMapEnd(); } @@ -14085,17 +14085,17 @@ uint32_t ThriftHiveMetastore_partition_name_to_spec_presult::read(::apache::thri if (ftype == ::apache::thrift::protocol::T_MAP) { { (*(this->success)).clear(); - uint32_t _size630; - ::apache::thrift::protocol::TType _ktype631; - ::apache::thrift::protocol::TType _vtype632; - xfer += iprot->readMapBegin(_ktype631, _vtype632, _size630); - uint32_t _i634; - for (_i634 = 0; _i634 < _size630; ++_i634) + uint32_t _size675; + ::apache::thrift::protocol::TType _ktype676; + ::apache::thrift::protocol::TType _vtype677; + xfer += iprot->readMapBegin(_ktype676, _vtype677, _size675); + uint32_t _i679; + for (_i679 = 0; _i679 < _size675; ++_i679) { - std::string _key635; - xfer += iprot->readString(_key635); - std::string& _val636 = (*(this->success))[_key635]; - xfer += iprot->readString(_val636); + std::string _key680; + xfer += iprot->readString(_key680); + std::string& _val681 = (*(this->success))[_key680]; + xfer += iprot->readString(_val681); } xfer += iprot->readMapEnd(); } @@ -14164,17 +14164,17 @@ uint32_t ThriftHiveMetastore_markPartitionForEvent_args::read(::apache::thrift:: if (ftype == ::apache::thrift::protocol::T_MAP) { { this->part_vals.clear(); - uint32_t _size637; - ::apache::thrift::protocol::TType _ktype638; - ::apache::thrift::protocol::TType _vtype639; - xfer += iprot->readMapBegin(_ktype638, _vtype639, _size637); - uint32_t _i641; - for (_i641 = 0; _i641 < _size637; ++_i641) + uint32_t _size682; + ::apache::thrift::protocol::TType _ktype683; + ::apache::thrift::protocol::TType _vtype684; + xfer += iprot->readMapBegin(_ktype683, _vtype684, _size682); + uint32_t _i686; + for (_i686 = 0; _i686 < _size682; ++_i686) { - std::string _key642; - xfer += iprot->readString(_key642); - std::string& _val643 = this->part_vals[_key642]; - xfer += iprot->readString(_val643); + std::string _key687; + xfer += iprot->readString(_key687); + std::string& _val688 = this->part_vals[_key687]; + xfer += iprot->readString(_val688); } xfer += iprot->readMapEnd(); } @@ -14185,9 +14185,9 @@ uint32_t ThriftHiveMetastore_markPartitionForEvent_args::read(::apache::thrift:: break; case 4: if (ftype == ::apache::thrift::protocol::T_I32) { - int32_t ecast644; - xfer += iprot->readI32(ecast644); - this->eventType = (PartitionEventType::type)ecast644; + int32_t ecast689; + xfer += iprot->readI32(ecast689); + this->eventType = (PartitionEventType::type)ecast689; this->__isset.eventType = true; } else { xfer += iprot->skip(ftype); @@ -14220,11 +14220,11 @@ uint32_t ThriftHiveMetastore_markPartitionForEvent_args::write(::apache::thrift: 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 _iter645; - for (_iter645 = this->part_vals.begin(); _iter645 != this->part_vals.end(); ++_iter645) + std::map ::const_iterator _iter690; + for (_iter690 = this->part_vals.begin(); _iter690 != this->part_vals.end(); ++_iter690) { - xfer += oprot->writeString(_iter645->first); - xfer += oprot->writeString(_iter645->second); + xfer += oprot->writeString(_iter690->first); + xfer += oprot->writeString(_iter690->second); } xfer += oprot->writeMapEnd(); } @@ -14254,11 +14254,11 @@ uint32_t ThriftHiveMetastore_markPartitionForEvent_pargs::write(::apache::thrift 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 _iter646; - for (_iter646 = (*(this->part_vals)).begin(); _iter646 != (*(this->part_vals)).end(); ++_iter646) + std::map ::const_iterator _iter691; + for (_iter691 = (*(this->part_vals)).begin(); _iter691 != (*(this->part_vals)).end(); ++_iter691) { - xfer += oprot->writeString(_iter646->first); - xfer += oprot->writeString(_iter646->second); + xfer += oprot->writeString(_iter691->first); + xfer += oprot->writeString(_iter691->second); } xfer += oprot->writeMapEnd(); } @@ -14509,17 +14509,17 @@ uint32_t ThriftHiveMetastore_isPartitionMarkedForEvent_args::read(::apache::thri if (ftype == ::apache::thrift::protocol::T_MAP) { { this->part_vals.clear(); - uint32_t _size647; - ::apache::thrift::protocol::TType _ktype648; - ::apache::thrift::protocol::TType _vtype649; - xfer += iprot->readMapBegin(_ktype648, _vtype649, _size647); - uint32_t _i651; - for (_i651 = 0; _i651 < _size647; ++_i651) + uint32_t _size692; + ::apache::thrift::protocol::TType _ktype693; + ::apache::thrift::protocol::TType _vtype694; + xfer += iprot->readMapBegin(_ktype693, _vtype694, _size692); + uint32_t _i696; + for (_i696 = 0; _i696 < _size692; ++_i696) { - std::string _key652; - xfer += iprot->readString(_key652); - std::string& _val653 = this->part_vals[_key652]; - xfer += iprot->readString(_val653); + std::string _key697; + xfer += iprot->readString(_key697); + std::string& _val698 = this->part_vals[_key697]; + xfer += iprot->readString(_val698); } xfer += iprot->readMapEnd(); } @@ -14530,9 +14530,9 @@ uint32_t ThriftHiveMetastore_isPartitionMarkedForEvent_args::read(::apache::thri break; case 4: if (ftype == ::apache::thrift::protocol::T_I32) { - int32_t ecast654; - xfer += iprot->readI32(ecast654); - this->eventType = (PartitionEventType::type)ecast654; + int32_t ecast699; + xfer += iprot->readI32(ecast699); + this->eventType = (PartitionEventType::type)ecast699; this->__isset.eventType = true; } else { xfer += iprot->skip(ftype); @@ -14565,11 +14565,11 @@ uint32_t ThriftHiveMetastore_isPartitionMarkedForEvent_args::write(::apache::thr 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 _iter655; - for (_iter655 = this->part_vals.begin(); _iter655 != this->part_vals.end(); ++_iter655) + std::map ::const_iterator _iter700; + for (_iter700 = this->part_vals.begin(); _iter700 != this->part_vals.end(); ++_iter700) { - xfer += oprot->writeString(_iter655->first); - xfer += oprot->writeString(_iter655->second); + xfer += oprot->writeString(_iter700->first); + xfer += oprot->writeString(_iter700->second); } xfer += oprot->writeMapEnd(); } @@ -14599,11 +14599,11 @@ uint32_t ThriftHiveMetastore_isPartitionMarkedForEvent_pargs::write(::apache::th 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 _iter656; - for (_iter656 = (*(this->part_vals)).begin(); _iter656 != (*(this->part_vals)).end(); ++_iter656) + std::map ::const_iterator _iter701; + for (_iter701 = (*(this->part_vals)).begin(); _iter701 != (*(this->part_vals)).end(); ++_iter701) { - xfer += oprot->writeString(_iter656->first); - xfer += oprot->writeString(_iter656->second); + xfer += oprot->writeString(_iter701->first); + xfer += oprot->writeString(_iter701->second); } xfer += oprot->writeMapEnd(); } @@ -15908,14 +15908,14 @@ uint32_t ThriftHiveMetastore_get_indexes_result::read(::apache::thrift::protocol if (ftype == ::apache::thrift::protocol::T_LIST) { { this->success.clear(); - uint32_t _size657; - ::apache::thrift::protocol::TType _etype660; - xfer += iprot->readListBegin(_etype660, _size657); - this->success.resize(_size657); - uint32_t _i661; - for (_i661 = 0; _i661 < _size657; ++_i661) + uint32_t _size702; + ::apache::thrift::protocol::TType _etype705; + xfer += iprot->readListBegin(_etype705, _size702); + this->success.resize(_size702); + uint32_t _i706; + for (_i706 = 0; _i706 < _size702; ++_i706) { - xfer += this->success[_i661].read(iprot); + xfer += this->success[_i706].read(iprot); } xfer += iprot->readListEnd(); } @@ -15962,10 +15962,10 @@ uint32_t ThriftHiveMetastore_get_indexes_result::write(::apache::thrift::protoco 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 _iter662; - for (_iter662 = this->success.begin(); _iter662 != this->success.end(); ++_iter662) + std::vector ::const_iterator _iter707; + for (_iter707 = this->success.begin(); _iter707 != this->success.end(); ++_iter707) { - xfer += (*_iter662).write(oprot); + xfer += (*_iter707).write(oprot); } xfer += oprot->writeListEnd(); } @@ -16008,14 +16008,14 @@ uint32_t ThriftHiveMetastore_get_indexes_presult::read(::apache::thrift::protoco if (ftype == ::apache::thrift::protocol::T_LIST) { { (*(this->success)).clear(); - uint32_t _size663; - ::apache::thrift::protocol::TType _etype666; - xfer += iprot->readListBegin(_etype666, _size663); - (*(this->success)).resize(_size663); - uint32_t _i667; - for (_i667 = 0; _i667 < _size663; ++_i667) + uint32_t _size708; + ::apache::thrift::protocol::TType _etype711; + xfer += iprot->readListBegin(_etype711, _size708); + (*(this->success)).resize(_size708); + uint32_t _i712; + for (_i712 = 0; _i712 < _size708; ++_i712) { - xfer += (*(this->success))[_i667].read(iprot); + xfer += (*(this->success))[_i712].read(iprot); } xfer += iprot->readListEnd(); } @@ -16174,14 +16174,14 @@ uint32_t ThriftHiveMetastore_get_index_names_result::read(::apache::thrift::prot if (ftype == ::apache::thrift::protocol::T_LIST) { { this->success.clear(); - uint32_t _size668; - ::apache::thrift::protocol::TType _etype671; - xfer += iprot->readListBegin(_etype671, _size668); - this->success.resize(_size668); - uint32_t _i672; - for (_i672 = 0; _i672 < _size668; ++_i672) + uint32_t _size713; + ::apache::thrift::protocol::TType _etype716; + xfer += iprot->readListBegin(_etype716, _size713); + this->success.resize(_size713); + uint32_t _i717; + for (_i717 = 0; _i717 < _size713; ++_i717) { - xfer += iprot->readString(this->success[_i672]); + xfer += iprot->readString(this->success[_i717]); } xfer += iprot->readListEnd(); } @@ -16220,10 +16220,10 @@ uint32_t ThriftHiveMetastore_get_index_names_result::write(::apache::thrift::pro 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 _iter673; - for (_iter673 = this->success.begin(); _iter673 != this->success.end(); ++_iter673) + std::vector ::const_iterator _iter718; + for (_iter718 = this->success.begin(); _iter718 != this->success.end(); ++_iter718) { - xfer += oprot->writeString((*_iter673)); + xfer += oprot->writeString((*_iter718)); } xfer += oprot->writeListEnd(); } @@ -16262,14 +16262,14 @@ uint32_t ThriftHiveMetastore_get_index_names_presult::read(::apache::thrift::pro if (ftype == ::apache::thrift::protocol::T_LIST) { { (*(this->success)).clear(); - uint32_t _size674; - ::apache::thrift::protocol::TType _etype677; - xfer += iprot->readListBegin(_etype677, _size674); - (*(this->success)).resize(_size674); - uint32_t _i678; - for (_i678 = 0; _i678 < _size674; ++_i678) + uint32_t _size719; + ::apache::thrift::protocol::TType _etype722; + xfer += iprot->readListBegin(_etype722, _size719); + (*(this->success)).resize(_size719); + uint32_t _i723; + for (_i723 = 0; _i723 < _size719; ++_i723) { - xfer += iprot->readString((*(this->success))[_i678]); + xfer += iprot->readString((*(this->success))[_i723]); } xfer += iprot->readListEnd(); } @@ -19054,14 +19054,14 @@ uint32_t ThriftHiveMetastore_get_functions_result::read(::apache::thrift::protoc if (ftype == ::apache::thrift::protocol::T_LIST) { { this->success.clear(); - uint32_t _size679; - ::apache::thrift::protocol::TType _etype682; - xfer += iprot->readListBegin(_etype682, _size679); - this->success.resize(_size679); - uint32_t _i683; - for (_i683 = 0; _i683 < _size679; ++_i683) + uint32_t _size724; + ::apache::thrift::protocol::TType _etype727; + xfer += iprot->readListBegin(_etype727, _size724); + this->success.resize(_size724); + uint32_t _i728; + for (_i728 = 0; _i728 < _size724; ++_i728) { - xfer += iprot->readString(this->success[_i683]); + xfer += iprot->readString(this->success[_i728]); } xfer += iprot->readListEnd(); } @@ -19100,10 +19100,10 @@ uint32_t ThriftHiveMetastore_get_functions_result::write(::apache::thrift::proto 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 _iter684; - for (_iter684 = this->success.begin(); _iter684 != this->success.end(); ++_iter684) + std::vector ::const_iterator _iter729; + for (_iter729 = this->success.begin(); _iter729 != this->success.end(); ++_iter729) { - xfer += oprot->writeString((*_iter684)); + xfer += oprot->writeString((*_iter729)); } xfer += oprot->writeListEnd(); } @@ -19142,14 +19142,14 @@ uint32_t ThriftHiveMetastore_get_functions_presult::read(::apache::thrift::proto if (ftype == ::apache::thrift::protocol::T_LIST) { { (*(this->success)).clear(); - uint32_t _size685; - ::apache::thrift::protocol::TType _etype688; - xfer += iprot->readListBegin(_etype688, _size685); - (*(this->success)).resize(_size685); - uint32_t _i689; - for (_i689 = 0; _i689 < _size685; ++_i689) + uint32_t _size730; + ::apache::thrift::protocol::TType _etype733; + xfer += iprot->readListBegin(_etype733, _size730); + (*(this->success)).resize(_size730); + uint32_t _i734; + for (_i734 = 0; _i734 < _size730; ++_i734) { - xfer += iprot->readString((*(this->success))[_i689]); + xfer += iprot->readString((*(this->success))[_i734]); } xfer += iprot->readListEnd(); } @@ -19829,14 +19829,14 @@ uint32_t ThriftHiveMetastore_get_role_names_result::read(::apache::thrift::proto if (ftype == ::apache::thrift::protocol::T_LIST) { { this->success.clear(); - uint32_t _size690; - ::apache::thrift::protocol::TType _etype693; - xfer += iprot->readListBegin(_etype693, _size690); - this->success.resize(_size690); - uint32_t _i694; - for (_i694 = 0; _i694 < _size690; ++_i694) + uint32_t _size735; + ::apache::thrift::protocol::TType _etype738; + xfer += iprot->readListBegin(_etype738, _size735); + this->success.resize(_size735); + uint32_t _i739; + for (_i739 = 0; _i739 < _size735; ++_i739) { - xfer += iprot->readString(this->success[_i694]); + xfer += iprot->readString(this->success[_i739]); } xfer += iprot->readListEnd(); } @@ -19875,10 +19875,10 @@ uint32_t ThriftHiveMetastore_get_role_names_result::write(::apache::thrift::prot 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 _iter695; - for (_iter695 = this->success.begin(); _iter695 != this->success.end(); ++_iter695) + std::vector ::const_iterator _iter740; + for (_iter740 = this->success.begin(); _iter740 != this->success.end(); ++_iter740) { - xfer += oprot->writeString((*_iter695)); + xfer += oprot->writeString((*_iter740)); } xfer += oprot->writeListEnd(); } @@ -19917,14 +19917,14 @@ uint32_t ThriftHiveMetastore_get_role_names_presult::read(::apache::thrift::prot if (ftype == ::apache::thrift::protocol::T_LIST) { { (*(this->success)).clear(); - uint32_t _size696; - ::apache::thrift::protocol::TType _etype699; - xfer += iprot->readListBegin(_etype699, _size696); - (*(this->success)).resize(_size696); - uint32_t _i700; - for (_i700 = 0; _i700 < _size696; ++_i700) + uint32_t _size741; + ::apache::thrift::protocol::TType _etype744; + xfer += iprot->readListBegin(_etype744, _size741); + (*(this->success)).resize(_size741); + uint32_t _i745; + for (_i745 = 0; _i745 < _size741; ++_i745) { - xfer += iprot->readString((*(this->success))[_i700]); + xfer += iprot->readString((*(this->success))[_i745]); } xfer += iprot->readListEnd(); } @@ -19991,9 +19991,9 @@ uint32_t ThriftHiveMetastore_grant_role_args::read(::apache::thrift::protocol::T break; case 3: if (ftype == ::apache::thrift::protocol::T_I32) { - int32_t ecast701; - xfer += iprot->readI32(ecast701); - this->principal_type = (PrincipalType::type)ecast701; + int32_t ecast746; + xfer += iprot->readI32(ecast746); + this->principal_type = (PrincipalType::type)ecast746; this->__isset.principal_type = true; } else { xfer += iprot->skip(ftype); @@ -20009,9 +20009,9 @@ uint32_t ThriftHiveMetastore_grant_role_args::read(::apache::thrift::protocol::T break; case 5: if (ftype == ::apache::thrift::protocol::T_I32) { - int32_t ecast702; - xfer += iprot->readI32(ecast702); - this->grantorType = (PrincipalType::type)ecast702; + int32_t ecast747; + xfer += iprot->readI32(ecast747); + this->grantorType = (PrincipalType::type)ecast747; this->__isset.grantorType = true; } else { xfer += iprot->skip(ftype); @@ -20257,9 +20257,9 @@ uint32_t ThriftHiveMetastore_revoke_role_args::read(::apache::thrift::protocol:: break; case 3: if (ftype == ::apache::thrift::protocol::T_I32) { - int32_t ecast703; - xfer += iprot->readI32(ecast703); - this->principal_type = (PrincipalType::type)ecast703; + int32_t ecast748; + xfer += iprot->readI32(ecast748); + this->principal_type = (PrincipalType::type)ecast748; this->__isset.principal_type = true; } else { xfer += iprot->skip(ftype); @@ -20465,9 +20465,9 @@ uint32_t ThriftHiveMetastore_list_roles_args::read(::apache::thrift::protocol::T break; case 2: if (ftype == ::apache::thrift::protocol::T_I32) { - int32_t ecast704; - xfer += iprot->readI32(ecast704); - this->principal_type = (PrincipalType::type)ecast704; + int32_t ecast749; + xfer += iprot->readI32(ecast749); + this->principal_type = (PrincipalType::type)ecast749; this->__isset.principal_type = true; } else { xfer += iprot->skip(ftype); @@ -20543,14 +20543,14 @@ uint32_t ThriftHiveMetastore_list_roles_result::read(::apache::thrift::protocol: if (ftype == ::apache::thrift::protocol::T_LIST) { { this->success.clear(); - uint32_t _size705; - ::apache::thrift::protocol::TType _etype708; - xfer += iprot->readListBegin(_etype708, _size705); - this->success.resize(_size705); - uint32_t _i709; - for (_i709 = 0; _i709 < _size705; ++_i709) + uint32_t _size750; + ::apache::thrift::protocol::TType _etype753; + xfer += iprot->readListBegin(_etype753, _size750); + this->success.resize(_size750); + uint32_t _i754; + for (_i754 = 0; _i754 < _size750; ++_i754) { - xfer += this->success[_i709].read(iprot); + xfer += this->success[_i754].read(iprot); } xfer += iprot->readListEnd(); } @@ -20589,10 +20589,10 @@ uint32_t ThriftHiveMetastore_list_roles_result::write(::apache::thrift::protocol 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 _iter710; - for (_iter710 = this->success.begin(); _iter710 != this->success.end(); ++_iter710) + std::vector ::const_iterator _iter755; + for (_iter755 = this->success.begin(); _iter755 != this->success.end(); ++_iter755) { - xfer += (*_iter710).write(oprot); + xfer += (*_iter755).write(oprot); } xfer += oprot->writeListEnd(); } @@ -20631,14 +20631,14 @@ uint32_t ThriftHiveMetastore_list_roles_presult::read(::apache::thrift::protocol if (ftype == ::apache::thrift::protocol::T_LIST) { { (*(this->success)).clear(); - uint32_t _size711; - ::apache::thrift::protocol::TType _etype714; - xfer += iprot->readListBegin(_etype714, _size711); - (*(this->success)).resize(_size711); - uint32_t _i715; - for (_i715 = 0; _i715 < _size711; ++_i715) + uint32_t _size756; + ::apache::thrift::protocol::TType _etype759; + xfer += iprot->readListBegin(_etype759, _size756); + (*(this->success)).resize(_size756); + uint32_t _i760; + for (_i760 = 0; _i760 < _size756; ++_i760) { - xfer += (*(this->success))[_i715].read(iprot); + xfer += (*(this->success))[_i760].read(iprot); } xfer += iprot->readListEnd(); } @@ -20707,14 +20707,14 @@ uint32_t ThriftHiveMetastore_get_privilege_set_args::read(::apache::thrift::prot if (ftype == ::apache::thrift::protocol::T_LIST) { { this->group_names.clear(); - uint32_t _size716; - ::apache::thrift::protocol::TType _etype719; - xfer += iprot->readListBegin(_etype719, _size716); - this->group_names.resize(_size716); - uint32_t _i720; - for (_i720 = 0; _i720 < _size716; ++_i720) + uint32_t _size761; + ::apache::thrift::protocol::TType _etype764; + xfer += iprot->readListBegin(_etype764, _size761); + this->group_names.resize(_size761); + uint32_t _i765; + for (_i765 = 0; _i765 < _size761; ++_i765) { - xfer += iprot->readString(this->group_names[_i720]); + xfer += iprot->readString(this->group_names[_i765]); } xfer += iprot->readListEnd(); } @@ -20750,10 +20750,10 @@ uint32_t ThriftHiveMetastore_get_privilege_set_args::write(::apache::thrift::pro 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 _iter721; - for (_iter721 = this->group_names.begin(); _iter721 != this->group_names.end(); ++_iter721) + std::vector ::const_iterator _iter766; + for (_iter766 = this->group_names.begin(); _iter766 != this->group_names.end(); ++_iter766) { - xfer += oprot->writeString((*_iter721)); + xfer += oprot->writeString((*_iter766)); } xfer += oprot->writeListEnd(); } @@ -20779,10 +20779,10 @@ uint32_t ThriftHiveMetastore_get_privilege_set_pargs::write(::apache::thrift::pr 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 _iter722; - for (_iter722 = (*(this->group_names)).begin(); _iter722 != (*(this->group_names)).end(); ++_iter722) + std::vector ::const_iterator _iter767; + for (_iter767 = (*(this->group_names)).begin(); _iter767 != (*(this->group_names)).end(); ++_iter767) { - xfer += oprot->writeString((*_iter722)); + xfer += oprot->writeString((*_iter767)); } xfer += oprot->writeListEnd(); } @@ -20939,9 +20939,9 @@ uint32_t ThriftHiveMetastore_list_privileges_args::read(::apache::thrift::protoc break; case 2: if (ftype == ::apache::thrift::protocol::T_I32) { - int32_t ecast723; - xfer += iprot->readI32(ecast723); - this->principal_type = (PrincipalType::type)ecast723; + int32_t ecast768; + xfer += iprot->readI32(ecast768); + this->principal_type = (PrincipalType::type)ecast768; this->__isset.principal_type = true; } else { xfer += iprot->skip(ftype); @@ -21033,14 +21033,14 @@ uint32_t ThriftHiveMetastore_list_privileges_result::read(::apache::thrift::prot if (ftype == ::apache::thrift::protocol::T_LIST) { { this->success.clear(); - uint32_t _size724; - ::apache::thrift::protocol::TType _etype727; - xfer += iprot->readListBegin(_etype727, _size724); - this->success.resize(_size724); - uint32_t _i728; - for (_i728 = 0; _i728 < _size724; ++_i728) + uint32_t _size769; + ::apache::thrift::protocol::TType _etype772; + xfer += iprot->readListBegin(_etype772, _size769); + this->success.resize(_size769); + uint32_t _i773; + for (_i773 = 0; _i773 < _size769; ++_i773) { - xfer += this->success[_i728].read(iprot); + xfer += this->success[_i773].read(iprot); } xfer += iprot->readListEnd(); } @@ -21079,10 +21079,10 @@ uint32_t ThriftHiveMetastore_list_privileges_result::write(::apache::thrift::pro 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 _iter729; - for (_iter729 = this->success.begin(); _iter729 != this->success.end(); ++_iter729) + std::vector ::const_iterator _iter774; + for (_iter774 = this->success.begin(); _iter774 != this->success.end(); ++_iter774) { - xfer += (*_iter729).write(oprot); + xfer += (*_iter774).write(oprot); } xfer += oprot->writeListEnd(); } @@ -21121,14 +21121,14 @@ uint32_t ThriftHiveMetastore_list_privileges_presult::read(::apache::thrift::pro if (ftype == ::apache::thrift::protocol::T_LIST) { { (*(this->success)).clear(); - uint32_t _size730; - ::apache::thrift::protocol::TType _etype733; - xfer += iprot->readListBegin(_etype733, _size730); - (*(this->success)).resize(_size730); - uint32_t _i734; - for (_i734 = 0; _i734 < _size730; ++_i734) + uint32_t _size775; + ::apache::thrift::protocol::TType _etype778; + xfer += iprot->readListBegin(_etype778, _size775); + (*(this->success)).resize(_size775); + uint32_t _i779; + for (_i779 = 0; _i779 < _size775; ++_i779) { - xfer += (*(this->success))[_i734].read(iprot); + xfer += (*(this->success))[_i779].read(iprot); } xfer += iprot->readListEnd(); } @@ -21553,14 +21553,14 @@ uint32_t ThriftHiveMetastore_set_ugi_args::read(::apache::thrift::protocol::TPro if (ftype == ::apache::thrift::protocol::T_LIST) { { this->group_names.clear(); - uint32_t _size735; - ::apache::thrift::protocol::TType _etype738; - xfer += iprot->readListBegin(_etype738, _size735); - this->group_names.resize(_size735); - uint32_t _i739; - for (_i739 = 0; _i739 < _size735; ++_i739) + uint32_t _size780; + ::apache::thrift::protocol::TType _etype783; + xfer += iprot->readListBegin(_etype783, _size780); + this->group_names.resize(_size780); + uint32_t _i784; + for (_i784 = 0; _i784 < _size780; ++_i784) { - xfer += iprot->readString(this->group_names[_i739]); + xfer += iprot->readString(this->group_names[_i784]); } xfer += iprot->readListEnd(); } @@ -21592,10 +21592,10 @@ uint32_t ThriftHiveMetastore_set_ugi_args::write(::apache::thrift::protocol::TPr 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 _iter740; - for (_iter740 = this->group_names.begin(); _iter740 != this->group_names.end(); ++_iter740) + std::vector ::const_iterator _iter785; + for (_iter785 = this->group_names.begin(); _iter785 != this->group_names.end(); ++_iter785) { - xfer += oprot->writeString((*_iter740)); + xfer += oprot->writeString((*_iter785)); } xfer += oprot->writeListEnd(); } @@ -21617,10 +21617,10 @@ uint32_t ThriftHiveMetastore_set_ugi_pargs::write(::apache::thrift::protocol::TP 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 _iter741; - for (_iter741 = (*(this->group_names)).begin(); _iter741 != (*(this->group_names)).end(); ++_iter741) + std::vector ::const_iterator _iter786; + for (_iter786 = (*(this->group_names)).begin(); _iter786 != (*(this->group_names)).end(); ++_iter786) { - xfer += oprot->writeString((*_iter741)); + xfer += oprot->writeString((*_iter786)); } xfer += oprot->writeListEnd(); } @@ -21655,14 +21655,14 @@ uint32_t ThriftHiveMetastore_set_ugi_result::read(::apache::thrift::protocol::TP if (ftype == ::apache::thrift::protocol::T_LIST) { { this->success.clear(); - uint32_t _size742; - ::apache::thrift::protocol::TType _etype745; - xfer += iprot->readListBegin(_etype745, _size742); - this->success.resize(_size742); - uint32_t _i746; - for (_i746 = 0; _i746 < _size742; ++_i746) + uint32_t _size787; + ::apache::thrift::protocol::TType _etype790; + xfer += iprot->readListBegin(_etype790, _size787); + this->success.resize(_size787); + uint32_t _i791; + for (_i791 = 0; _i791 < _size787; ++_i791) { - xfer += iprot->readString(this->success[_i746]); + xfer += iprot->readString(this->success[_i791]); } xfer += iprot->readListEnd(); } @@ -21701,10 +21701,10 @@ uint32_t ThriftHiveMetastore_set_ugi_result::write(::apache::thrift::protocol::T 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 _iter747; - for (_iter747 = this->success.begin(); _iter747 != this->success.end(); ++_iter747) + std::vector ::const_iterator _iter792; + for (_iter792 = this->success.begin(); _iter792 != this->success.end(); ++_iter792) { - xfer += oprot->writeString((*_iter747)); + xfer += oprot->writeString((*_iter792)); } xfer += oprot->writeListEnd(); } @@ -21743,14 +21743,14 @@ uint32_t ThriftHiveMetastore_set_ugi_presult::read(::apache::thrift::protocol::T if (ftype == ::apache::thrift::protocol::T_LIST) { { (*(this->success)).clear(); - uint32_t _size748; - ::apache::thrift::protocol::TType _etype751; - xfer += iprot->readListBegin(_etype751, _size748); - (*(this->success)).resize(_size748); - uint32_t _i752; - for (_i752 = 0; _i752 < _size748; ++_i752) + uint32_t _size793; + ::apache::thrift::protocol::TType _etype796; + xfer += iprot->readListBegin(_etype796, _size793); + (*(this->success)).resize(_size793); + uint32_t _i797; + for (_i797 = 0; _i797 < _size793; ++_i797) { - xfer += iprot->readString((*(this->success))[_i752]); + xfer += iprot->readString((*(this->success))[_i797]); } xfer += iprot->readListEnd(); } @@ -22321,1130 +22321,3181 @@ uint32_t ThriftHiveMetastore_cancel_delegation_token_presult::read(::apache::thr return xfer; } -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); +uint32_t ThriftHiveMetastore_get_open_txns_args::read(::apache::thrift::protocol::TProtocol* iprot) { - ThriftHiveMetastore_create_database_pargs args; - args.database = &database; - args.write(oprot_); + uint32_t xfer = 0; + std::string fname; + ::apache::thrift::protocol::TType ftype; + int16_t fid; - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} + xfer += iprot->readStructBegin(fname); -void ThriftHiveMetastoreClient::recv_create_database() -{ + using ::apache::thrift::protocol::TProtocolException; - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - 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(); + while (true) + { + xfer += iprot->readFieldBegin(fname, ftype, fid); + if (ftype == ::apache::thrift::protocol::T_STOP) { + break; + } + xfer += iprot->skip(ftype); + xfer += iprot->readFieldEnd(); } - 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; -} + xfer += iprot->readStructEnd(); -void ThriftHiveMetastoreClient::get_database(Database& _return, const std::string& name) -{ - send_get_database(name); - recv_get_database(_return); + return xfer; } -void ThriftHiveMetastoreClient::send_get_database(const std::string& name) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("get_database", ::apache::thrift::protocol::T_CALL, cseqid); +uint32_t ThriftHiveMetastore_get_open_txns_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_open_txns_args"); - ThriftHiveMetastore_get_database_pargs args; - args.name = &name; - args.write(oprot_); + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); +uint32_t ThriftHiveMetastore_get_open_txns_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_open_txns_pargs"); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; } -void ThriftHiveMetastoreClient::recv_get_database(Database& _return) -{ +uint32_t ThriftHiveMetastore_get_open_txns_result::read(::apache::thrift::protocol::TProtocol* iprot) { - int32_t rseqid = 0; + uint32_t xfer = 0; std::string fname; - ::apache::thrift::protocol::TMessageType mtype; + ::apache::thrift::protocol::TType ftype; + int16_t fid; - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw 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(); + xfer += iprot->readStructBegin(fname); - 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"); -} + using ::apache::thrift::protocol::TProtocolException; -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); + while (true) + { + xfer += iprot->readFieldBegin(fname, ftype, fid); + if (ftype == ::apache::thrift::protocol::T_STOP) { + break; + } + switch (fid) + { + 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(); + } - ThriftHiveMetastore_drop_database_pargs args; - args.name = &name; - args.deleteData = &deleteData; - args.cascade = &cascade; - args.write(oprot_); + xfer += iprot->readStructEnd(); - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); + return xfer; } -void ThriftHiveMetastoreClient::recv_drop_database() -{ +uint32_t ThriftHiveMetastore_get_open_txns_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; + uint32_t xfer = 0; - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - 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(); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_open_txns_result"); - if (result.__isset.o1) { - throw result.o1; - } - if (result.__isset.o2) { - throw result.o2; - } - if (result.__isset.o3) { - throw result.o3; + if (this->__isset.success) { + xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0); + xfer += this->success.write(oprot); + xfer += oprot->writeFieldEnd(); } - return; -} - -void ThriftHiveMetastoreClient::get_databases(std::vector & _return, const std::string& pattern) -{ - send_get_databases(pattern); - recv_get_databases(_return); + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; } -void ThriftHiveMetastoreClient::send_get_databases(const std::string& pattern) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("get_databases", ::apache::thrift::protocol::T_CALL, cseqid); +uint32_t ThriftHiveMetastore_get_open_txns_presult::read(::apache::thrift::protocol::TProtocol* iprot) { - ThriftHiveMetastore_get_databases_pargs args; - args.pattern = &pattern; - args.write(oprot_); + uint32_t xfer = 0; + std::string fname; + ::apache::thrift::protocol::TType ftype; + int16_t fid; - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} + xfer += iprot->readStructBegin(fname); -void ThriftHiveMetastoreClient::recv_get_databases(std::vector & _return) -{ + using ::apache::thrift::protocol::TProtocolException; - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw 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(); + while (true) + { + xfer += iprot->readFieldBegin(fname, ftype, fid); + if (ftype == ::apache::thrift::protocol::T_STOP) { + break; + } + switch (fid) + { + 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(); } - 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"); -} + xfer += iprot->readStructEnd(); -void ThriftHiveMetastoreClient::get_all_databases(std::vector & _return) -{ - send_get_all_databases(); - recv_get_all_databases(_return); + return xfer; } -void ThriftHiveMetastoreClient::send_get_all_databases() -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("get_all_databases", ::apache::thrift::protocol::T_CALL, cseqid); +uint32_t ThriftHiveMetastore_get_open_txns_info_args::read(::apache::thrift::protocol::TProtocol* iprot) { - ThriftHiveMetastore_get_all_databases_pargs args; - args.write(oprot_); + uint32_t xfer = 0; + std::string fname; + ::apache::thrift::protocol::TType ftype; + int16_t fid; - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} + xfer += iprot->readStructBegin(fname); -void ThriftHiveMetastoreClient::recv_get_all_databases(std::vector & _return) -{ + using ::apache::thrift::protocol::TProtocolException; - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw 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(); + while (true) + { + xfer += iprot->readFieldBegin(fname, ftype, fid); + if (ftype == ::apache::thrift::protocol::T_STOP) { + break; + } + xfer += iprot->skip(ftype); + xfer += iprot->readFieldEnd(); } - 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"); -} + xfer += iprot->readStructEnd(); -void ThriftHiveMetastoreClient::alter_database(const std::string& dbname, const Database& db) -{ - send_alter_database(dbname, db); - recv_alter_database(); + return xfer; } -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); +uint32_t ThriftHiveMetastore_get_open_txns_info_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_open_txns_info_args"); - ThriftHiveMetastore_alter_database_pargs args; - args.dbname = &dbname; - args.db = &db; - args.write(oprot_); + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); +uint32_t ThriftHiveMetastore_get_open_txns_info_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_open_txns_info_pargs"); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; } -void ThriftHiveMetastoreClient::recv_alter_database() -{ +uint32_t ThriftHiveMetastore_get_open_txns_info_result::read(::apache::thrift::protocol::TProtocol* iprot) { - int32_t rseqid = 0; + uint32_t xfer = 0; std::string fname; - ::apache::thrift::protocol::TMessageType mtype; + ::apache::thrift::protocol::TType ftype; + int16_t fid; - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - 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(); + xfer += iprot->readStructBegin(fname); - if (result.__isset.o1) { - throw result.o1; - } - if (result.__isset.o2) { - throw result.o2; + using ::apache::thrift::protocol::TProtocolException; + + + while (true) + { + xfer += iprot->readFieldBegin(fname, ftype, fid); + if (ftype == ::apache::thrift::protocol::T_STOP) { + break; + } + switch (fid) + { + 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(); } - return; -} -void ThriftHiveMetastoreClient::get_type(Type& _return, const std::string& name) -{ - send_get_type(name); - recv_get_type(_return); + xfer += iprot->readStructEnd(); + + return xfer; } -void ThriftHiveMetastoreClient::send_get_type(const std::string& name) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("get_type", ::apache::thrift::protocol::T_CALL, cseqid); +uint32_t ThriftHiveMetastore_get_open_txns_info_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - ThriftHiveMetastore_get_type_pargs args; - args.name = &name; - args.write(oprot_); + uint32_t xfer = 0; - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); + 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; } -void ThriftHiveMetastoreClient::recv_get_type(Type& _return) -{ +uint32_t ThriftHiveMetastore_get_open_txns_info_presult::read(::apache::thrift::protocol::TProtocol* iprot) { - int32_t rseqid = 0; + uint32_t xfer = 0; std::string fname; - ::apache::thrift::protocol::TMessageType mtype; + ::apache::thrift::protocol::TType ftype; + int16_t fid; - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw 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(); + xfer += iprot->readStructBegin(fname); - 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"); -} + using ::apache::thrift::protocol::TProtocolException; -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); + while (true) + { + xfer += iprot->readFieldBegin(fname, ftype, fid); + if (ftype == ::apache::thrift::protocol::T_STOP) { + break; + } + switch (fid) + { + 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(); + } - ThriftHiveMetastore_create_type_pargs args; - args.type = &type; - args.write(oprot_); + xfer += iprot->readStructEnd(); - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); + return xfer; } -bool ThriftHiveMetastoreClient::recv_create_type() -{ +uint32_t ThriftHiveMetastore_open_txns_args::read(::apache::thrift::protocol::TProtocol* iprot) { - int32_t rseqid = 0; + uint32_t xfer = 0; std::string fname; - ::apache::thrift::protocol::TMessageType mtype; + ::apache::thrift::protocol::TType ftype; + int16_t fid; - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - 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(); + xfer += iprot->readStructBegin(fname); - 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; + using ::apache::thrift::protocol::TProtocolException; + + + while (true) + { + xfer += iprot->readFieldBegin(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(); } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "create_type failed: unknown result"); + + xfer += iprot->readStructEnd(); + + return xfer; } -bool ThriftHiveMetastoreClient::drop_type(const std::string& type) -{ - send_drop_type(type); - return recv_drop_type(); +uint32_t ThriftHiveMetastore_open_txns_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + 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; } -void ThriftHiveMetastoreClient::send_drop_type(const std::string& type) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("drop_type", ::apache::thrift::protocol::T_CALL, cseqid); +uint32_t ThriftHiveMetastore_open_txns_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + xfer += oprot->writeStructBegin("ThriftHiveMetastore_open_txns_pargs"); - ThriftHiveMetastore_drop_type_pargs args; - args.type = &type; - args.write(oprot_); + xfer += oprot->writeFieldBegin("rqst", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += (*(this->rqst)).write(oprot); + xfer += oprot->writeFieldEnd(); - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; } -bool ThriftHiveMetastoreClient::recv_drop_type() -{ +uint32_t ThriftHiveMetastore_open_txns_result::read(::apache::thrift::protocol::TProtocol* iprot) { - int32_t rseqid = 0; + uint32_t xfer = 0; std::string fname; - ::apache::thrift::protocol::TMessageType mtype; + ::apache::thrift::protocol::TType ftype; + int16_t fid; - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - 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(); + xfer += iprot->readStructBegin(fname); - if (result.__isset.success) { - return _return; - } - if (result.__isset.o1) { - throw result.o1; - } - if (result.__isset.o2) { - throw result.o2; + using ::apache::thrift::protocol::TProtocolException; + + + while (true) + { + xfer += iprot->readFieldBegin(fname, ftype, fid); + if (ftype == ::apache::thrift::protocol::T_STOP) { + break; + } + switch (fid) + { + 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(); } - 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); + xfer += iprot->readStructEnd(); + + return xfer; } -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); +uint32_t ThriftHiveMetastore_open_txns_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - ThriftHiveMetastore_get_type_all_pargs args; - args.name = &name; - args.write(oprot_); + uint32_t xfer = 0; - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); + 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; } -void ThriftHiveMetastoreClient::recv_get_type_all(std::map & _return) -{ +uint32_t ThriftHiveMetastore_open_txns_presult::read(::apache::thrift::protocol::TProtocol* iprot) { - int32_t rseqid = 0; + uint32_t xfer = 0; std::string fname; - ::apache::thrift::protocol::TMessageType mtype; + ::apache::thrift::protocol::TType ftype; + int16_t fid; - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw 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(); + xfer += iprot->readStructBegin(fname); - 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"); -} + using ::apache::thrift::protocol::TProtocolException; -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); + while (true) + { + xfer += iprot->readFieldBegin(fname, ftype, fid); + if (ftype == ::apache::thrift::protocol::T_STOP) { + break; + } + switch (fid) + { + 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(); + } - ThriftHiveMetastore_get_fields_pargs args; - args.db_name = &db_name; - args.table_name = &table_name; - args.write(oprot_); + xfer += iprot->readStructEnd(); - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); + return xfer; } -void ThriftHiveMetastoreClient::recv_get_fields(std::vector & _return) -{ +uint32_t ThriftHiveMetastore_abort_txn_args::read(::apache::thrift::protocol::TProtocol* iprot) { - int32_t rseqid = 0; + uint32_t xfer = 0; std::string fname; - ::apache::thrift::protocol::TMessageType mtype; + ::apache::thrift::protocol::TType ftype; + int16_t fid; - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw 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(); + xfer += iprot->readStructBegin(fname); - 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"); -} + using ::apache::thrift::protocol::TProtocolException; -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); + while (true) + { + xfer += iprot->readFieldBegin(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(); + } - ThriftHiveMetastore_get_schema_pargs args; - args.db_name = &db_name; - args.table_name = &table_name; - args.write(oprot_); + xfer += iprot->readStructEnd(); - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); + return xfer; } -void ThriftHiveMetastoreClient::recv_get_schema(std::vector & _return) -{ +uint32_t ThriftHiveMetastore_abort_txn_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + xfer += oprot->writeStructBegin("ThriftHiveMetastore_abort_txn_args"); - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw 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"); -} + xfer += oprot->writeFieldBegin("rqst", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->rqst.write(oprot); + xfer += oprot->writeFieldEnd(); -void ThriftHiveMetastoreClient::create_table(const Table& tbl) -{ - send_create_table(tbl); - recv_create_table(); + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; } -void ThriftHiveMetastoreClient::send_create_table(const Table& tbl) -{ - int32_t cseqid = 0; - oprot_->writeMessageBegin("create_table", ::apache::thrift::protocol::T_CALL, cseqid); +uint32_t ThriftHiveMetastore_abort_txn_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + xfer += oprot->writeStructBegin("ThriftHiveMetastore_abort_txn_pargs"); - ThriftHiveMetastore_create_table_pargs args; - args.tbl = &tbl; - args.write(oprot_); + xfer += oprot->writeFieldBegin("rqst", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += (*(this->rqst)).write(oprot); + xfer += oprot->writeFieldEnd(); - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; } -void ThriftHiveMetastoreClient::recv_create_table() -{ +uint32_t ThriftHiveMetastore_abort_txn_result::read(::apache::thrift::protocol::TProtocol* iprot) { - int32_t rseqid = 0; + uint32_t xfer = 0; std::string fname; - ::apache::thrift::protocol::TMessageType mtype; + ::apache::thrift::protocol::TType ftype; + int16_t fid; - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - 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(); + xfer += iprot->readStructBegin(fname); - 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; -} + using ::apache::thrift::protocol::TProtocolException; -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); + while (true) + { + xfer += iprot->readFieldBegin(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(); + } - ThriftHiveMetastore_create_table_with_environment_context_pargs args; - args.tbl = &tbl; - args.environment_context = &environment_context; - args.write(oprot_); + xfer += iprot->readStructEnd(); - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); + return xfer; } -void ThriftHiveMetastoreClient::recv_create_table_with_environment_context() -{ +uint32_t ThriftHiveMetastore_abort_txn_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; + uint32_t xfer = 0; - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - 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(); + xfer += oprot->writeStructBegin("ThriftHiveMetastore_abort_txn_result"); - 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 (this->__isset.o1) { + xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += this->o1.write(oprot); + xfer += oprot->writeFieldEnd(); } - 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(); + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; } -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); +uint32_t ThriftHiveMetastore_abort_txn_presult::read(::apache::thrift::protocol::TProtocol* iprot) { - ThriftHiveMetastore_drop_table_pargs args; - args.dbname = &dbname; - args.name = &name; - args.deleteData = &deleteData; - args.write(oprot_); + uint32_t xfer = 0; + std::string fname; + ::apache::thrift::protocol::TType ftype; + int16_t fid; - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} + xfer += iprot->readStructBegin(fname); -void ThriftHiveMetastoreClient::recv_drop_table() -{ + using ::apache::thrift::protocol::TProtocolException; - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - 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(); + while (true) + { + xfer += iprot->readFieldBegin(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(); } - 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; -} + xfer += iprot->readStructEnd(); -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(); + return xfer; } -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); +uint32_t ThriftHiveMetastore_commit_txn_args::read(::apache::thrift::protocol::TProtocol* iprot) { - 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_); + uint32_t xfer = 0; + std::string fname; + ::apache::thrift::protocol::TType ftype; + int16_t fid; - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} + xfer += iprot->readStructBegin(fname); -void ThriftHiveMetastoreClient::recv_drop_table_with_environment_context() -{ + using ::apache::thrift::protocol::TProtocolException; - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - 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(); + while (true) + { + xfer += iprot->readFieldBegin(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(); } - 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; + xfer += iprot->readStructEnd(); + + return xfer; } -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); +uint32_t ThriftHiveMetastore_commit_txn_args::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + 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; } -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); +uint32_t ThriftHiveMetastore_commit_txn_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + xfer += oprot->writeStructBegin("ThriftHiveMetastore_commit_txn_pargs"); - ThriftHiveMetastore_get_tables_pargs args; - args.db_name = &db_name; - args.pattern = &pattern; - args.write(oprot_); + xfer += oprot->writeFieldBegin("rqst", ::apache::thrift::protocol::T_STRUCT, 1); + xfer += (*(this->rqst)).write(oprot); + xfer += oprot->writeFieldEnd(); - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; } -void ThriftHiveMetastoreClient::recv_get_tables(std::vector & _return) -{ +uint32_t ThriftHiveMetastore_commit_txn_result::read(::apache::thrift::protocol::TProtocol* iprot) { - int32_t rseqid = 0; + uint32_t xfer = 0; std::string fname; - ::apache::thrift::protocol::TMessageType mtype; + ::apache::thrift::protocol::TType ftype; + int16_t fid; - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw 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(); + xfer += iprot->readStructBegin(fname); - if (result.__isset.success) { - // _return pointer has now been filled - return; - } - if (result.__isset.o1) { - throw result.o1; + using ::apache::thrift::protocol::TProtocolException; + + + while (true) + { + xfer += iprot->readFieldBegin(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(); } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_tables 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); + xfer += iprot->readStructEnd(); + + return xfer; } -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); +uint32_t ThriftHiveMetastore_commit_txn_result::write(::apache::thrift::protocol::TProtocol* oprot) const { - ThriftHiveMetastore_get_all_tables_pargs args; - args.db_name = &db_name; - args.write(oprot_); + uint32_t xfer = 0; - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); + 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; } -void ThriftHiveMetastoreClient::recv_get_all_tables(std::vector & _return) -{ +uint32_t ThriftHiveMetastore_commit_txn_presult::read(::apache::thrift::protocol::TProtocol* iprot) { - int32_t rseqid = 0; + uint32_t xfer = 0; std::string fname; - ::apache::thrift::protocol::TMessageType mtype; + ::apache::thrift::protocol::TType ftype; + int16_t fid; - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw 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(); - } + xfer += iprot->readStructBegin(fname); + + using ::apache::thrift::protocol::TProtocolException; + + + while (true) + { + xfer += iprot->readFieldBegin(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_lock_args::read(::apache::thrift::protocol::TProtocol* iprot) { + + uint32_t xfer = 0; + std::string fname; + ::apache::thrift::protocol::TType ftype; + int16_t fid; + + xfer += iprot->readStructBegin(fname); + + using ::apache::thrift::protocol::TProtocolException; + + + while (true) + { + xfer += iprot->readFieldBegin(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; + 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; +} + +uint32_t ThriftHiveMetastore_lock_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + 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; +} + +uint32_t ThriftHiveMetastore_lock_result::read(::apache::thrift::protocol::TProtocol* iprot) { + + uint32_t xfer = 0; + std::string fname; + ::apache::thrift::protocol::TType ftype; + int16_t fid; + + xfer += iprot->readStructBegin(fname); + + using ::apache::thrift::protocol::TProtocolException; + + + while (true) + { + xfer += iprot->readFieldBegin(fname, ftype, fid); + if (ftype == ::apache::thrift::protocol::T_STOP) { + break; + } + switch (fid) + { + 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; +} + +uint32_t ThriftHiveMetastore_lock_presult::read(::apache::thrift::protocol::TProtocol* iprot) { + + uint32_t xfer = 0; + std::string fname; + ::apache::thrift::protocol::TType ftype; + int16_t fid; + + xfer += iprot->readStructBegin(fname); + + using ::apache::thrift::protocol::TProtocolException; + + + while (true) + { + xfer += iprot->readFieldBegin(fname, ftype, fid); + if (ftype == ::apache::thrift::protocol::T_STOP) { + break; + } + switch (fid) + { + 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_check_lock_args::read(::apache::thrift::protocol::TProtocol* iprot) { + + uint32_t xfer = 0; + std::string fname; + ::apache::thrift::protocol::TType ftype; + int16_t fid; + + xfer += iprot->readStructBegin(fname); + + using ::apache::thrift::protocol::TProtocolException; + + + while (true) + { + xfer += iprot->readFieldBegin(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; + 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; +} + +uint32_t ThriftHiveMetastore_check_lock_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + 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; +} + +uint32_t ThriftHiveMetastore_check_lock_result::read(::apache::thrift::protocol::TProtocol* iprot) { + + uint32_t xfer = 0; + std::string fname; + ::apache::thrift::protocol::TType ftype; + int16_t fid; + + xfer += iprot->readStructBegin(fname); + + using ::apache::thrift::protocol::TProtocolException; + + + while (true) + { + xfer += iprot->readFieldBegin(fname, ftype, fid); + if (ftype == ::apache::thrift::protocol::T_STOP) { + break; + } + switch (fid) + { + 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; +} + +uint32_t ThriftHiveMetastore_check_lock_presult::read(::apache::thrift::protocol::TProtocol* iprot) { + + uint32_t xfer = 0; + std::string fname; + ::apache::thrift::protocol::TType ftype; + int16_t fid; + + xfer += iprot->readStructBegin(fname); + + using ::apache::thrift::protocol::TProtocolException; + + + while (true) + { + xfer += iprot->readFieldBegin(fname, ftype, fid); + if (ftype == ::apache::thrift::protocol::T_STOP) { + break; + } + switch (fid) + { + 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_unlock_args::read(::apache::thrift::protocol::TProtocol* iprot) { + + uint32_t xfer = 0; + std::string fname; + ::apache::thrift::protocol::TType ftype; + int16_t fid; + + xfer += iprot->readStructBegin(fname); + + using ::apache::thrift::protocol::TProtocolException; + + + while (true) + { + xfer += iprot->readFieldBegin(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; + 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; +} + +uint32_t ThriftHiveMetastore_unlock_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + 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; +} + +uint32_t ThriftHiveMetastore_unlock_result::read(::apache::thrift::protocol::TProtocol* iprot) { + + uint32_t xfer = 0; + std::string fname; + ::apache::thrift::protocol::TType ftype; + int16_t fid; + + xfer += iprot->readStructBegin(fname); + + using ::apache::thrift::protocol::TProtocolException; + + + while (true) + { + xfer += iprot->readFieldBegin(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; +} + +uint32_t ThriftHiveMetastore_unlock_presult::read(::apache::thrift::protocol::TProtocol* iprot) { + + uint32_t xfer = 0; + std::string fname; + ::apache::thrift::protocol::TType ftype; + int16_t fid; + + xfer += iprot->readStructBegin(fname); + + using ::apache::thrift::protocol::TProtocolException; + + + while (true) + { + xfer += iprot->readFieldBegin(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_show_locks_args::read(::apache::thrift::protocol::TProtocol* iprot) { + + uint32_t xfer = 0; + std::string fname; + ::apache::thrift::protocol::TType ftype; + int16_t fid; + + xfer += iprot->readStructBegin(fname); + + using ::apache::thrift::protocol::TProtocolException; + + + while (true) + { + xfer += iprot->readFieldBegin(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; + 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; +} + +uint32_t ThriftHiveMetastore_show_locks_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + 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; +} + +uint32_t ThriftHiveMetastore_show_locks_result::read(::apache::thrift::protocol::TProtocol* iprot) { + + uint32_t xfer = 0; + std::string fname; + ::apache::thrift::protocol::TType ftype; + int16_t fid; + + xfer += iprot->readStructBegin(fname); + + using ::apache::thrift::protocol::TProtocolException; + + + while (true) + { + xfer += iprot->readFieldBegin(fname, ftype, fid); + if (ftype == ::apache::thrift::protocol::T_STOP) { + break; + } + switch (fid) + { + 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; +} + +uint32_t ThriftHiveMetastore_show_locks_presult::read(::apache::thrift::protocol::TProtocol* iprot) { + + uint32_t xfer = 0; + std::string fname; + ::apache::thrift::protocol::TType ftype; + int16_t fid; + + xfer += iprot->readStructBegin(fname); + + using ::apache::thrift::protocol::TProtocolException; + + + while (true) + { + xfer += iprot->readFieldBegin(fname, ftype, fid); + if (ftype == ::apache::thrift::protocol::T_STOP) { + break; + } + switch (fid) + { + 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_args::read(::apache::thrift::protocol::TProtocol* iprot) { + + uint32_t xfer = 0; + std::string fname; + ::apache::thrift::protocol::TType ftype; + int16_t fid; + + xfer += iprot->readStructBegin(fname); + + using ::apache::thrift::protocol::TProtocolException; + + + while (true) + { + xfer += iprot->readFieldBegin(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; + 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; +} + +uint32_t ThriftHiveMetastore_heartbeat_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + 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; +} + +uint32_t ThriftHiveMetastore_heartbeat_result::read(::apache::thrift::protocol::TProtocol* iprot) { + + uint32_t xfer = 0; + std::string fname; + ::apache::thrift::protocol::TType ftype; + int16_t fid; + + xfer += iprot->readStructBegin(fname); + + using ::apache::thrift::protocol::TProtocolException; + + + while (true) + { + xfer += iprot->readFieldBegin(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; +} + +uint32_t ThriftHiveMetastore_heartbeat_presult::read(::apache::thrift::protocol::TProtocol* iprot) { + + uint32_t xfer = 0; + std::string fname; + ::apache::thrift::protocol::TType ftype; + int16_t fid; + + xfer += iprot->readStructBegin(fname); + + using ::apache::thrift::protocol::TProtocolException; + + + while (true) + { + xfer += iprot->readFieldBegin(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_compact_args::read(::apache::thrift::protocol::TProtocol* iprot) { + + uint32_t xfer = 0; + std::string fname; + ::apache::thrift::protocol::TType ftype; + int16_t fid; + + xfer += iprot->readStructBegin(fname); + + using ::apache::thrift::protocol::TProtocolException; + + + while (true) + { + xfer += iprot->readFieldBegin(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; + 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; +} + +uint32_t ThriftHiveMetastore_compact_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + 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; +} + +uint32_t ThriftHiveMetastore_compact_result::read(::apache::thrift::protocol::TProtocol* iprot) { + + uint32_t xfer = 0; + std::string fname; + ::apache::thrift::protocol::TType ftype; + int16_t fid; + + xfer += 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; +} + +uint32_t ThriftHiveMetastore_compact_presult::read(::apache::thrift::protocol::TProtocol* iprot) { + + uint32_t xfer = 0; + std::string fname; + ::apache::thrift::protocol::TType ftype; + int16_t fid; + + xfer += 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_show_compact_args::read(::apache::thrift::protocol::TProtocol* iprot) { + + uint32_t xfer = 0; + std::string fname; + ::apache::thrift::protocol::TType ftype; + int16_t fid; + + xfer += iprot->readStructBegin(fname); + + using ::apache::thrift::protocol::TProtocolException; + + + while (true) + { + xfer += iprot->readFieldBegin(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; + 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; +} + +uint32_t ThriftHiveMetastore_show_compact_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + 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; +} + +uint32_t ThriftHiveMetastore_show_compact_result::read(::apache::thrift::protocol::TProtocol* iprot) { + + uint32_t xfer = 0; + std::string fname; + ::apache::thrift::protocol::TType ftype; + int16_t fid; + + xfer += iprot->readStructBegin(fname); + + using ::apache::thrift::protocol::TProtocolException; + + + while (true) + { + xfer += iprot->readFieldBegin(fname, ftype, fid); + if (ftype == ::apache::thrift::protocol::T_STOP) { + break; + } + switch (fid) + { + 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; +} + +uint32_t ThriftHiveMetastore_show_compact_presult::read(::apache::thrift::protocol::TProtocol* iprot) { + + uint32_t xfer = 0; + std::string fname; + ::apache::thrift::protocol::TType ftype; + int16_t fid; + + xfer += iprot->readStructBegin(fname); + + using ::apache::thrift::protocol::TProtocolException; + + + while (true) + { + xfer += iprot->readFieldBegin(fname, ftype, fid); + if (ftype == ::apache::thrift::protocol::T_STOP) { + break; + } + switch (fid) + { + 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; +} + +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_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::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::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::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_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_); @@ -23458,23 +25509,1230 @@ void ThriftHiveMetastoreClient::recv_get_all_tables(std::vector & _ if (result.__isset.o1) { throw result.o1; } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_all_tables failed: unknown result"); + 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; + } + 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 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::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"); +} + +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_table(Table& _return, const std::string& dbname, const std::string& tbl_name) +void ThriftHiveMetastoreClient::get_partition(Partition& _return, const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals) { - send_get_table(dbname, tbl_name); - recv_get_table(_return); + send_get_partition(db_name, tbl_name, part_vals); + recv_get_partition(_return); } -void ThriftHiveMetastoreClient::send_get_table(const std::string& dbname, const std::string& tbl_name) +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_table", ::apache::thrift::protocol::T_CALL, cseqid); + oprot_->writeMessageBegin("get_partition", ::apache::thrift::protocol::T_CALL, cseqid); - ThriftHiveMetastore_get_table_pargs args; - args.dbname = &dbname; + 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(); @@ -23482,7 +26740,7 @@ void ThriftHiveMetastoreClient::send_get_table(const std::string& dbname, const oprot_->getTransport()->flush(); } -void ThriftHiveMetastoreClient::recv_get_table(Table& _return) +void ThriftHiveMetastoreClient::recv_get_partition(Partition& _return) { int32_t rseqid = 0; @@ -23502,12 +26760,12 @@ void ThriftHiveMetastoreClient::recv_get_table(Table& _return) iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - if (fname.compare("get_table") != 0) { + if (fname.compare("get_partition") != 0) { iprot_->skip(::apache::thrift::protocol::T_STRUCT); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - ThriftHiveMetastore_get_table_presult result; + ThriftHiveMetastore_get_partition_presult result; result.success = &_return; result.read(iprot_); iprot_->readMessageEnd(); @@ -23523,23 +26781,26 @@ void ThriftHiveMetastoreClient::recv_get_table(Table& _return) if (result.__isset.o2) { throw result.o2; } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_table failed: unknown result"); + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_partition failed: unknown result"); } -void ThriftHiveMetastoreClient::get_table_objects_by_name(std::vector
& _return, const std::string& dbname, const std::vector & tbl_names) +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_get_table_objects_by_name(dbname, tbl_names); - recv_get_table_objects_by_name(_return); + send_exchange_partition(partitionSpecs, source_db, source_table_name, dest_db, dest_table_name); + recv_exchange_partition(_return); } -void ThriftHiveMetastoreClient::send_get_table_objects_by_name(const std::string& dbname, const std::vector & tbl_names) +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("get_table_objects_by_name", ::apache::thrift::protocol::T_CALL, cseqid); + oprot_->writeMessageBegin("exchange_partition", ::apache::thrift::protocol::T_CALL, cseqid); - ThriftHiveMetastore_get_table_objects_by_name_pargs args; - args.dbname = &dbname; - args.tbl_names = &tbl_names; + 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(); @@ -23547,7 +26808,7 @@ void ThriftHiveMetastoreClient::send_get_table_objects_by_name(const std::string oprot_->getTransport()->flush(); } -void ThriftHiveMetastoreClient::recv_get_table_objects_by_name(std::vector
& _return) +void ThriftHiveMetastoreClient::recv_exchange_partition(Partition& _return) { int32_t rseqid = 0; @@ -23567,12 +26828,12 @@ void ThriftHiveMetastoreClient::recv_get_table_objects_by_name(std::vector
readMessageEnd(); iprot_->getTransport()->readEnd(); } - if (fname.compare("get_table_objects_by_name") != 0) { + if (fname.compare("exchange_partition") != 0) { iprot_->skip(::apache::thrift::protocol::T_STRUCT); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - ThriftHiveMetastore_get_table_objects_by_name_presult result; + ThriftHiveMetastore_exchange_partition_presult result; result.success = &_return; result.read(iprot_); iprot_->readMessageEnd(); @@ -23591,24 +26852,29 @@ void ThriftHiveMetastoreClient::recv_get_table_objects_by_name(std::vector
& _return, const std::string& dbname, const std::string& filter, const int16_t max_tables) +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_table_names_by_filter(dbname, filter, max_tables); - recv_get_table_names_by_filter(_return); + 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_table_names_by_filter(const std::string& dbname, const std::string& filter, const int16_t max_tables) +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_table_names_by_filter", ::apache::thrift::protocol::T_CALL, cseqid); + oprot_->writeMessageBegin("get_partition_with_auth", ::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; + 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(); @@ -23616,7 +26882,7 @@ void ThriftHiveMetastoreClient::send_get_table_names_by_filter(const std::string oprot_->getTransport()->flush(); } -void ThriftHiveMetastoreClient::recv_get_table_names_by_filter(std::vector & _return) +void ThriftHiveMetastoreClient::recv_get_partition_with_auth(Partition& _return) { int32_t rseqid = 0; @@ -23636,12 +26902,12 @@ void ThriftHiveMetastoreClient::recv_get_table_names_by_filter(std::vectorreadMessageEnd(); iprot_->getTransport()->readEnd(); } - if (fname.compare("get_table_names_by_filter") != 0) { + if (fname.compare("get_partition_with_auth") != 0) { iprot_->skip(::apache::thrift::protocol::T_STRUCT); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - ThriftHiveMetastore_get_table_names_by_filter_presult result; + ThriftHiveMetastore_get_partition_with_auth_presult result; result.success = &_return; result.read(iprot_); iprot_->readMessageEnd(); @@ -23657,27 +26923,90 @@ void ThriftHiveMetastoreClient::recv_get_table_names_by_filter(std::vectorwriteMessageBegin("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; } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_table_names_by_filter failed: unknown result"); + 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::alter_table(const std::string& dbname, const std::string& tbl_name, const Table& new_tbl) +void ThriftHiveMetastoreClient::get_partitions(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const int16_t max_parts) { - send_alter_table(dbname, tbl_name, new_tbl); - recv_alter_table(); + send_get_partitions(db_name, tbl_name, max_parts); + recv_get_partitions(_return); } -void ThriftHiveMetastoreClient::send_alter_table(const std::string& dbname, const std::string& tbl_name, const Table& new_tbl) +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("alter_table", ::apache::thrift::protocol::T_CALL, cseqid); + oprot_->writeMessageBegin("get_partitions", ::apache::thrift::protocol::T_CALL, cseqid); - ThriftHiveMetastore_alter_table_pargs args; - args.dbname = &dbname; + ThriftHiveMetastore_get_partitions_pargs args; + args.db_name = &db_name; args.tbl_name = &tbl_name; - args.new_tbl = &new_tbl; + args.max_parts = &max_parts; args.write(oprot_); oprot_->writeMessageEnd(); @@ -23685,7 +27014,7 @@ void ThriftHiveMetastoreClient::send_alter_table(const std::string& dbname, cons oprot_->getTransport()->flush(); } -void ThriftHiveMetastoreClient::recv_alter_table() +void ThriftHiveMetastoreClient::recv_get_partitions(std::vector & _return) { int32_t rseqid = 0; @@ -23705,41 +27034,47 @@ void ThriftHiveMetastoreClient::recv_alter_table() iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - if (fname.compare("alter_table") != 0) { + if (fname.compare("get_partitions") != 0) { iprot_->skip(::apache::thrift::protocol::T_STRUCT); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - ThriftHiveMetastore_alter_table_presult result; + 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; } - return; + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_partitions failed: unknown result"); } -void ThriftHiveMetastoreClient::alter_table_with_environment_context(const std::string& dbname, const std::string& tbl_name, const Table& new_tbl, const EnvironmentContext& environment_context) +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_alter_table_with_environment_context(dbname, tbl_name, new_tbl, environment_context); - recv_alter_table_with_environment_context(); + send_get_partitions_with_auth(db_name, tbl_name, max_parts, user_name, group_names); + recv_get_partitions_with_auth(_return); } -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) +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("alter_table_with_environment_context", ::apache::thrift::protocol::T_CALL, cseqid); + oprot_->writeMessageBegin("get_partitions_with_auth", ::apache::thrift::protocol::T_CALL, cseqid); - ThriftHiveMetastore_alter_table_with_environment_context_pargs args; - args.dbname = &dbname; + ThriftHiveMetastore_get_partitions_with_auth_pargs args; + args.db_name = &db_name; args.tbl_name = &tbl_name; - args.new_tbl = &new_tbl; - args.environment_context = &environment_context; + args.max_parts = &max_parts; + args.user_name = &user_name; + args.group_names = &group_names; args.write(oprot_); oprot_->writeMessageEnd(); @@ -23747,7 +27082,7 @@ void ThriftHiveMetastoreClient::send_alter_table_with_environment_context(const oprot_->getTransport()->flush(); } -void ThriftHiveMetastoreClient::recv_alter_table_with_environment_context() +void ThriftHiveMetastoreClient::recv_get_partitions_with_auth(std::vector & _return) { int32_t rseqid = 0; @@ -23767,38 +27102,45 @@ void ThriftHiveMetastoreClient::recv_alter_table_with_environment_context() iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - if (fname.compare("alter_table_with_environment_context") != 0) { + if (fname.compare("get_partitions_with_auth") != 0) { iprot_->skip(::apache::thrift::protocol::T_STRUCT); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - ThriftHiveMetastore_alter_table_with_environment_context_presult result; + 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; } - return; + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_partitions_with_auth failed: unknown result"); } -void ThriftHiveMetastoreClient::add_partition(Partition& _return, const Partition& new_part) +void ThriftHiveMetastoreClient::get_partition_names(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const int16_t max_parts) { - send_add_partition(new_part); - recv_add_partition(_return); + send_get_partition_names(db_name, tbl_name, max_parts); + recv_get_partition_names(_return); } -void ThriftHiveMetastoreClient::send_add_partition(const Partition& new_part) +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("add_partition", ::apache::thrift::protocol::T_CALL, cseqid); + oprot_->writeMessageBegin("get_partition_names", ::apache::thrift::protocol::T_CALL, cseqid); - ThriftHiveMetastore_add_partition_pargs args; - args.new_part = &new_part; + 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(); @@ -23806,7 +27148,7 @@ void ThriftHiveMetastoreClient::send_add_partition(const Partition& new_part) oprot_->getTransport()->flush(); } -void ThriftHiveMetastoreClient::recv_add_partition(Partition& _return) +void ThriftHiveMetastoreClient::recv_get_partition_names(std::vector & _return) { int32_t rseqid = 0; @@ -23826,12 +27168,12 @@ void ThriftHiveMetastoreClient::recv_add_partition(Partition& _return) iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - if (fname.compare("add_partition") != 0) { + if (fname.compare("get_partition_names") != 0) { iprot_->skip(::apache::thrift::protocol::T_STRUCT); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - ThriftHiveMetastore_add_partition_presult result; + ThriftHiveMetastore_get_partition_names_presult result; result.success = &_return; result.read(iprot_); iprot_->readMessageEnd(); @@ -23841,32 +27183,28 @@ void ThriftHiveMetastoreClient::recv_add_partition(Partition& _return) // _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"); + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_partition_names failed: unknown result"); } -void ThriftHiveMetastoreClient::add_partition_with_environment_context(Partition& _return, const Partition& new_part, const EnvironmentContext& environment_context) +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_add_partition_with_environment_context(new_part, environment_context); - recv_add_partition_with_environment_context(_return); + send_get_partitions_ps(db_name, tbl_name, part_vals, max_parts); + recv_get_partitions_ps(_return); } -void ThriftHiveMetastoreClient::send_add_partition_with_environment_context(const Partition& new_part, const EnvironmentContext& environment_context) +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("add_partition_with_environment_context", ::apache::thrift::protocol::T_CALL, cseqid); + oprot_->writeMessageBegin("get_partitions_ps", ::apache::thrift::protocol::T_CALL, cseqid); - ThriftHiveMetastore_add_partition_with_environment_context_pargs args; - args.new_part = &new_part; - args.environment_context = &environment_context; + 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(); @@ -23874,7 +27212,7 @@ void ThriftHiveMetastoreClient::send_add_partition_with_environment_context(cons oprot_->getTransport()->flush(); } -void ThriftHiveMetastoreClient::recv_add_partition_with_environment_context(Partition& _return) +void ThriftHiveMetastoreClient::recv_get_partitions_ps(std::vector & _return) { int32_t rseqid = 0; @@ -23894,12 +27232,12 @@ void ThriftHiveMetastoreClient::recv_add_partition_with_environment_context(Part iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - if (fname.compare("add_partition_with_environment_context") != 0) { + if (fname.compare("get_partitions_ps") != 0) { iprot_->skip(::apache::thrift::protocol::T_STRUCT); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - ThriftHiveMetastore_add_partition_with_environment_context_presult result; + ThriftHiveMetastore_get_partitions_ps_presult result; result.success = &_return; result.read(iprot_); iprot_->readMessageEnd(); @@ -23915,25 +27253,27 @@ void ThriftHiveMetastoreClient::recv_add_partition_with_environment_context(Part 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"); + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_partitions_ps failed: unknown result"); } -int32_t ThriftHiveMetastoreClient::add_partitions(const std::vector & new_parts) +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_add_partitions(new_parts); - return recv_add_partitions(); + 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_add_partitions(const std::vector & new_parts) +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("add_partitions", ::apache::thrift::protocol::T_CALL, cseqid); + oprot_->writeMessageBegin("get_partitions_ps_with_auth", ::apache::thrift::protocol::T_CALL, cseqid); - ThriftHiveMetastore_add_partitions_pargs args; - args.new_parts = &new_parts; + 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(); @@ -23941,7 +27281,7 @@ void ThriftHiveMetastoreClient::send_add_partitions(const std::vector oprot_->getTransport()->flush(); } -int32_t ThriftHiveMetastoreClient::recv_add_partitions() +void ThriftHiveMetastoreClient::recv_get_partitions_ps_with_auth(std::vector & _return) { int32_t rseqid = 0; @@ -23961,20 +27301,20 @@ int32_t ThriftHiveMetastoreClient::recv_add_partitions() iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - if (fname.compare("add_partitions") != 0) { + if (fname.compare("get_partitions_ps_with_auth") != 0) { iprot_->skip(::apache::thrift::protocol::T_STRUCT); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - int32_t _return; - ThriftHiveMetastore_add_partitions_presult result; + ThriftHiveMetastore_get_partitions_ps_with_auth_presult result; result.success = &_return; result.read(iprot_); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); if (result.__isset.success) { - return _return; + // _return pointer has now been filled + return; } if (result.__isset.o1) { throw result.o1; @@ -23982,27 +27322,25 @@ int32_t ThriftHiveMetastoreClient::recv_add_partitions() 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"); + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_partitions_ps_with_auth failed: unknown result"); } -void ThriftHiveMetastoreClient::append_partition(Partition& _return, const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals) +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_append_partition(db_name, tbl_name, part_vals); - recv_append_partition(_return); + send_get_partition_names_ps(db_name, tbl_name, part_vals, max_parts); + recv_get_partition_names_ps(_return); } -void ThriftHiveMetastoreClient::send_append_partition(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals) +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("append_partition", ::apache::thrift::protocol::T_CALL, cseqid); + oprot_->writeMessageBegin("get_partition_names_ps", ::apache::thrift::protocol::T_CALL, cseqid); - ThriftHiveMetastore_append_partition_pargs args; + 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(); @@ -24010,7 +27348,7 @@ void ThriftHiveMetastoreClient::send_append_partition(const std::string& db_name oprot_->getTransport()->flush(); } -void ThriftHiveMetastoreClient::recv_append_partition(Partition& _return) +void ThriftHiveMetastoreClient::recv_get_partition_names_ps(std::vector & _return) { int32_t rseqid = 0; @@ -24030,12 +27368,12 @@ void ThriftHiveMetastoreClient::recv_append_partition(Partition& _return) iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - if (fname.compare("append_partition") != 0) { + if (fname.compare("get_partition_names_ps") != 0) { iprot_->skip(::apache::thrift::protocol::T_STRUCT); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - ThriftHiveMetastore_append_partition_presult result; + ThriftHiveMetastore_get_partition_names_ps_presult result; result.success = &_return; result.read(iprot_); iprot_->readMessageEnd(); @@ -24051,25 +27389,25 @@ void ThriftHiveMetastoreClient::recv_append_partition(Partition& _return) 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"); + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_partition_names_ps failed: unknown result"); } -void ThriftHiveMetastoreClient::add_partitions_req(AddPartitionsResult& _return, const AddPartitionsRequest& request) +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_add_partitions_req(request); - recv_add_partitions_req(_return); + send_get_partitions_by_filter(db_name, tbl_name, filter, max_parts); + recv_get_partitions_by_filter(_return); } -void ThriftHiveMetastoreClient::send_add_partitions_req(const AddPartitionsRequest& request) +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("add_partitions_req", ::apache::thrift::protocol::T_CALL, cseqid); + oprot_->writeMessageBegin("get_partitions_by_filter", ::apache::thrift::protocol::T_CALL, cseqid); - ThriftHiveMetastore_add_partitions_req_pargs args; - args.request = &request; + 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(); @@ -24077,7 +27415,7 @@ void ThriftHiveMetastoreClient::send_add_partitions_req(const AddPartitionsReque oprot_->getTransport()->flush(); } -void ThriftHiveMetastoreClient::recv_add_partitions_req(AddPartitionsResult& _return) +void ThriftHiveMetastoreClient::recv_get_partitions_by_filter(std::vector & _return) { int32_t rseqid = 0; @@ -24097,12 +27435,12 @@ void ThriftHiveMetastoreClient::recv_add_partitions_req(AddPartitionsResult& _re iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - if (fname.compare("add_partitions_req") != 0) { + if (fname.compare("get_partitions_by_filter") != 0) { iprot_->skip(::apache::thrift::protocol::T_STRUCT); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - ThriftHiveMetastore_add_partitions_req_presult result; + ThriftHiveMetastore_get_partitions_by_filter_presult result; result.success = &_return; result.read(iprot_); iprot_->readMessageEnd(); @@ -24118,28 +27456,22 @@ void ThriftHiveMetastoreClient::recv_add_partitions_req(AddPartitionsResult& _re 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"); + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_partitions_by_filter 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) +void ThriftHiveMetastoreClient::get_partitions_by_expr(PartitionsByExprResult& _return, const PartitionsByExprRequest& req) { - send_append_partition_with_environment_context(db_name, tbl_name, part_vals, environment_context); - recv_append_partition_with_environment_context(_return); + send_get_partitions_by_expr(req); + recv_get_partitions_by_expr(_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) +void ThriftHiveMetastoreClient::send_get_partitions_by_expr(const PartitionsByExprRequest& req) { int32_t cseqid = 0; - oprot_->writeMessageBegin("append_partition_with_environment_context", ::apache::thrift::protocol::T_CALL, cseqid); + oprot_->writeMessageBegin("get_partitions_by_expr", ::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; + ThriftHiveMetastore_get_partitions_by_expr_pargs args; + args.req = &req; args.write(oprot_); oprot_->writeMessageEnd(); @@ -24147,7 +27479,7 @@ void ThriftHiveMetastoreClient::send_append_partition_with_environment_context(c oprot_->getTransport()->flush(); } -void ThriftHiveMetastoreClient::recv_append_partition_with_environment_context(Partition& _return) +void ThriftHiveMetastoreClient::recv_get_partitions_by_expr(PartitionsByExprResult& _return) { int32_t rseqid = 0; @@ -24167,12 +27499,12 @@ void ThriftHiveMetastoreClient::recv_append_partition_with_environment_context(P iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - if (fname.compare("append_partition_with_environment_context") != 0) { + if (fname.compare("get_partitions_by_expr") != 0) { iprot_->skip(::apache::thrift::protocol::T_STRUCT); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - ThriftHiveMetastore_append_partition_with_environment_context_presult result; + ThriftHiveMetastore_get_partitions_by_expr_presult result; result.success = &_return; result.read(iprot_); iprot_->readMessageEnd(); @@ -24188,27 +27520,24 @@ void ThriftHiveMetastoreClient::recv_append_partition_with_environment_context(P 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"); + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_partitions_by_expr 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) +void ThriftHiveMetastoreClient::get_partitions_by_names(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const std::vector & names) { - send_append_partition_by_name(db_name, tbl_name, part_name); - recv_append_partition_by_name(_return); + send_get_partitions_by_names(db_name, tbl_name, names); + recv_get_partitions_by_names(_return); } -void ThriftHiveMetastoreClient::send_append_partition_by_name(const std::string& db_name, const std::string& tbl_name, const std::string& part_name) +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("append_partition_by_name", ::apache::thrift::protocol::T_CALL, cseqid); + oprot_->writeMessageBegin("get_partitions_by_names", ::apache::thrift::protocol::T_CALL, cseqid); - ThriftHiveMetastore_append_partition_by_name_pargs args; + ThriftHiveMetastore_get_partitions_by_names_pargs args; args.db_name = &db_name; args.tbl_name = &tbl_name; - args.part_name = &part_name; + args.names = &names; args.write(oprot_); oprot_->writeMessageEnd(); @@ -24216,7 +27545,7 @@ void ThriftHiveMetastoreClient::send_append_partition_by_name(const std::string& oprot_->getTransport()->flush(); } -void ThriftHiveMetastoreClient::recv_append_partition_by_name(Partition& _return) +void ThriftHiveMetastoreClient::recv_get_partitions_by_names(std::vector & _return) { int32_t rseqid = 0; @@ -24236,12 +27565,12 @@ void ThriftHiveMetastoreClient::recv_append_partition_by_name(Partition& _return iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - if (fname.compare("append_partition_by_name") != 0) { + if (fname.compare("get_partitions_by_names") != 0) { iprot_->skip(::apache::thrift::protocol::T_STRUCT); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - ThriftHiveMetastore_append_partition_by_name_presult result; + ThriftHiveMetastore_get_partitions_by_names_presult result; result.success = &_return; result.read(iprot_); iprot_->readMessageEnd(); @@ -24257,28 +27586,24 @@ void ThriftHiveMetastoreClient::recv_append_partition_by_name(Partition& _return 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"); + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_partitions_by_names 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) +void ThriftHiveMetastoreClient::alter_partition(const std::string& db_name, const std::string& tbl_name, const Partition& new_part) { - 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); + send_alter_partition(db_name, tbl_name, new_part); + recv_alter_partition(); } -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) +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("append_partition_by_name_with_environment_context", ::apache::thrift::protocol::T_CALL, cseqid); + oprot_->writeMessageBegin("alter_partition", ::apache::thrift::protocol::T_CALL, cseqid); - ThriftHiveMetastore_append_partition_by_name_with_environment_context_pargs args; + ThriftHiveMetastore_alter_partition_pargs args; args.db_name = &db_name; args.tbl_name = &tbl_name; - args.part_name = &part_name; - args.environment_context = &environment_context; + args.new_part = &new_part; args.write(oprot_); oprot_->writeMessageEnd(); @@ -24286,7 +27611,7 @@ void ThriftHiveMetastoreClient::send_append_partition_by_name_with_environment_c oprot_->getTransport()->flush(); } -void ThriftHiveMetastoreClient::recv_append_partition_by_name_with_environment_context(Partition& _return) +void ThriftHiveMetastoreClient::recv_alter_partition() { int32_t rseqid = 0; @@ -24306,49 +27631,40 @@ void ThriftHiveMetastoreClient::recv_append_partition_by_name_with_environment_c iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - if (fname.compare("append_partition_by_name_with_environment_context") != 0) { + if (fname.compare("alter_partition") != 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; + ThriftHiveMetastore_alter_partition_presult result; 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"); + return; } -bool ThriftHiveMetastoreClient::drop_partition(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const bool deleteData) +void ThriftHiveMetastoreClient::alter_partitions(const std::string& db_name, const std::string& tbl_name, const std::vector & new_parts) { - send_drop_partition(db_name, tbl_name, part_vals, deleteData); - return recv_drop_partition(); + send_alter_partitions(db_name, tbl_name, new_parts); + recv_alter_partitions(); } -void ThriftHiveMetastoreClient::send_drop_partition(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const bool deleteData) +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("drop_partition", ::apache::thrift::protocol::T_CALL, cseqid); + oprot_->writeMessageBegin("alter_partitions", ::apache::thrift::protocol::T_CALL, cseqid); - ThriftHiveMetastore_drop_partition_pargs args; + ThriftHiveMetastore_alter_partitions_pargs args; args.db_name = &db_name; args.tbl_name = &tbl_name; - args.part_vals = &part_vals; - args.deleteData = &deleteData; + args.new_parts = &new_parts; args.write(oprot_); oprot_->writeMessageEnd(); @@ -24356,7 +27672,7 @@ void ThriftHiveMetastoreClient::send_drop_partition(const std::string& db_name, oprot_->getTransport()->flush(); } -bool ThriftHiveMetastoreClient::recv_drop_partition() +void ThriftHiveMetastoreClient::recv_alter_partitions() { int32_t rseqid = 0; @@ -24376,46 +27692,40 @@ bool ThriftHiveMetastoreClient::recv_drop_partition() iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - if (fname.compare("drop_partition") != 0) { + if (fname.compare("alter_partitions") != 0) { iprot_->skip(::apache::thrift::protocol::T_STRUCT); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - bool _return; - ThriftHiveMetastore_drop_partition_presult result; - result.success = &_return; + ThriftHiveMetastore_alter_partitions_presult result; 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"); + return; } -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) +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_drop_partition_with_environment_context(db_name, tbl_name, part_vals, deleteData, environment_context); - return recv_drop_partition_with_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_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 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("drop_partition_with_environment_context", ::apache::thrift::protocol::T_CALL, cseqid); + oprot_->writeMessageBegin("alter_partition_with_environment_context", ::apache::thrift::protocol::T_CALL, cseqid); - ThriftHiveMetastore_drop_partition_with_environment_context_pargs args; + ThriftHiveMetastore_alter_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.new_part = &new_part; args.environment_context = &environment_context; args.write(oprot_); @@ -24424,7 +27734,7 @@ void ThriftHiveMetastoreClient::send_drop_partition_with_environment_context(con oprot_->getTransport()->flush(); } -bool ThriftHiveMetastoreClient::recv_drop_partition_with_environment_context() +void ThriftHiveMetastoreClient::recv_alter_partition_with_environment_context() { int32_t rseqid = 0; @@ -24444,46 +27754,41 @@ bool ThriftHiveMetastoreClient::recv_drop_partition_with_environment_context() iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - if (fname.compare("drop_partition_with_environment_context") != 0) { + if (fname.compare("alter_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; + ThriftHiveMetastore_alter_partition_with_environment_context_presult result; 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"); + return; } -bool ThriftHiveMetastoreClient::drop_partition_by_name(const std::string& db_name, const std::string& tbl_name, const std::string& part_name, const bool deleteData) +void ThriftHiveMetastoreClient::rename_partition(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const Partition& new_part) { - send_drop_partition_by_name(db_name, tbl_name, part_name, deleteData); - return recv_drop_partition_by_name(); + send_rename_partition(db_name, tbl_name, part_vals, new_part); + recv_rename_partition(); } -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) +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("drop_partition_by_name", ::apache::thrift::protocol::T_CALL, cseqid); + oprot_->writeMessageBegin("rename_partition", ::apache::thrift::protocol::T_CALL, cseqid); - ThriftHiveMetastore_drop_partition_by_name_pargs args; + ThriftHiveMetastore_rename_partition_pargs args; args.db_name = &db_name; args.tbl_name = &tbl_name; - args.part_name = &part_name; - args.deleteData = &deleteData; + args.part_vals = &part_vals; + args.new_part = &new_part; args.write(oprot_); oprot_->writeMessageEnd(); @@ -24491,7 +27796,7 @@ void ThriftHiveMetastoreClient::send_drop_partition_by_name(const std::string& d oprot_->getTransport()->flush(); } -bool ThriftHiveMetastoreClient::recv_drop_partition_by_name() +void ThriftHiveMetastoreClient::recv_rename_partition() { int32_t rseqid = 0; @@ -24511,47 +27816,39 @@ bool ThriftHiveMetastoreClient::recv_drop_partition_by_name() iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - if (fname.compare("drop_partition_by_name") != 0) { + if (fname.compare("rename_partition") != 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; + ThriftHiveMetastore_rename_partition_presult result; 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"); + return; } -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) +bool ThriftHiveMetastoreClient::partition_name_has_valid_characters(const std::vector & part_vals, const bool throw_exception) { - 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(); + send_partition_name_has_valid_characters(part_vals, throw_exception); + return recv_partition_name_has_valid_characters(); } -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) +void ThriftHiveMetastoreClient::send_partition_name_has_valid_characters(const std::vector & part_vals, const bool throw_exception) { int32_t cseqid = 0; - oprot_->writeMessageBegin("drop_partition_by_name_with_environment_context", ::apache::thrift::protocol::T_CALL, cseqid); + oprot_->writeMessageBegin("partition_name_has_valid_characters", ::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; + ThriftHiveMetastore_partition_name_has_valid_characters_pargs args; + args.part_vals = &part_vals; + args.throw_exception = &throw_exception; args.write(oprot_); oprot_->writeMessageEnd(); @@ -24559,7 +27856,7 @@ void ThriftHiveMetastoreClient::send_drop_partition_by_name_with_environment_con oprot_->getTransport()->flush(); } -bool ThriftHiveMetastoreClient::recv_drop_partition_by_name_with_environment_context() +bool ThriftHiveMetastoreClient::recv_partition_name_has_valid_characters() { int32_t rseqid = 0; @@ -24579,13 +27876,13 @@ bool ThriftHiveMetastoreClient::recv_drop_partition_by_name_with_environment_con iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - if (fname.compare("drop_partition_by_name_with_environment_context") != 0) { + if (fname.compare("partition_name_has_valid_characters") != 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; + ThriftHiveMetastore_partition_name_has_valid_characters_presult result; result.success = &_return; result.read(iprot_); iprot_->readMessageEnd(); @@ -24597,25 +27894,23 @@ bool ThriftHiveMetastoreClient::recv_drop_partition_by_name_with_environment_con 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"); + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "partition_name_has_valid_characters failed: unknown result"); } -void ThriftHiveMetastoreClient::drop_partitions_req(DropPartitionsResult& _return, const DropPartitionsRequest& req) +void ThriftHiveMetastoreClient::get_config_value(std::string& _return, const std::string& name, const std::string& defaultValue) { - send_drop_partitions_req(req); - recv_drop_partitions_req(_return); + send_get_config_value(name, defaultValue); + recv_get_config_value(_return); } -void ThriftHiveMetastoreClient::send_drop_partitions_req(const DropPartitionsRequest& req) +void ThriftHiveMetastoreClient::send_get_config_value(const std::string& name, const std::string& defaultValue) { int32_t cseqid = 0; - oprot_->writeMessageBegin("drop_partitions_req", ::apache::thrift::protocol::T_CALL, cseqid); + oprot_->writeMessageBegin("get_config_value", ::apache::thrift::protocol::T_CALL, cseqid); - ThriftHiveMetastore_drop_partitions_req_pargs args; - args.req = &req; + ThriftHiveMetastore_get_config_value_pargs args; + args.name = &name; + args.defaultValue = &defaultValue; args.write(oprot_); oprot_->writeMessageEnd(); @@ -24623,7 +27918,7 @@ void ThriftHiveMetastoreClient::send_drop_partitions_req(const DropPartitionsReq oprot_->getTransport()->flush(); } -void ThriftHiveMetastoreClient::recv_drop_partitions_req(DropPartitionsResult& _return) +void ThriftHiveMetastoreClient::recv_get_config_value(std::string& _return) { int32_t rseqid = 0; @@ -24643,12 +27938,12 @@ void ThriftHiveMetastoreClient::recv_drop_partitions_req(DropPartitionsResult& _ iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - if (fname.compare("drop_partitions_req") != 0) { + if (fname.compare("get_config_value") != 0) { iprot_->skip(::apache::thrift::protocol::T_STRUCT); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - ThriftHiveMetastore_drop_partitions_req_presult result; + ThriftHiveMetastore_get_config_value_presult result; result.success = &_return; result.read(iprot_); iprot_->readMessageEnd(); @@ -24661,27 +27956,22 @@ void ThriftHiveMetastoreClient::recv_drop_partitions_req(DropPartitionsResult& _ 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"); + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_config_value failed: unknown result"); } -void ThriftHiveMetastoreClient::get_partition(Partition& _return, const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals) +void ThriftHiveMetastoreClient::partition_name_to_vals(std::vector & _return, const std::string& part_name) { - send_get_partition(db_name, tbl_name, part_vals); - recv_get_partition(_return); + send_partition_name_to_vals(part_name); + recv_partition_name_to_vals(_return); } -void ThriftHiveMetastoreClient::send_get_partition(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals) +void ThriftHiveMetastoreClient::send_partition_name_to_vals(const std::string& part_name) { int32_t cseqid = 0; - oprot_->writeMessageBegin("get_partition", ::apache::thrift::protocol::T_CALL, cseqid); + oprot_->writeMessageBegin("partition_name_to_vals", ::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; + ThriftHiveMetastore_partition_name_to_vals_pargs args; + args.part_name = &part_name; args.write(oprot_); oprot_->writeMessageEnd(); @@ -24689,7 +27979,7 @@ void ThriftHiveMetastoreClient::send_get_partition(const std::string& db_name, c oprot_->getTransport()->flush(); } -void ThriftHiveMetastoreClient::recv_get_partition(Partition& _return) +void ThriftHiveMetastoreClient::recv_partition_name_to_vals(std::vector & _return) { int32_t rseqid = 0; @@ -24709,12 +27999,12 @@ void ThriftHiveMetastoreClient::recv_get_partition(Partition& _return) iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - if (fname.compare("get_partition") != 0) { + if (fname.compare("partition_name_to_vals") != 0) { iprot_->skip(::apache::thrift::protocol::T_STRUCT); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - ThriftHiveMetastore_get_partition_presult result; + ThriftHiveMetastore_partition_name_to_vals_presult result; result.success = &_return; result.read(iprot_); iprot_->readMessageEnd(); @@ -24727,29 +28017,22 @@ void ThriftHiveMetastoreClient::recv_get_partition(Partition& _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"); + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "partition_name_to_vals 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) +void ThriftHiveMetastoreClient::partition_name_to_spec(std::map & _return, const std::string& part_name) { - send_exchange_partition(partitionSpecs, source_db, source_table_name, dest_db, dest_table_name); - recv_exchange_partition(_return); + send_partition_name_to_spec(part_name); + recv_partition_name_to_spec(_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) +void ThriftHiveMetastoreClient::send_partition_name_to_spec(const std::string& part_name) { int32_t cseqid = 0; - oprot_->writeMessageBegin("exchange_partition", ::apache::thrift::protocol::T_CALL, cseqid); + oprot_->writeMessageBegin("partition_name_to_spec", ::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; + ThriftHiveMetastore_partition_name_to_spec_pargs args; + args.part_name = &part_name; args.write(oprot_); oprot_->writeMessageEnd(); @@ -24757,7 +28040,7 @@ void ThriftHiveMetastoreClient::send_exchange_partition(const std::mapgetTransport()->flush(); } -void ThriftHiveMetastoreClient::recv_exchange_partition(Partition& _return) +void ThriftHiveMetastoreClient::recv_partition_name_to_spec(std::map & _return) { int32_t rseqid = 0; @@ -24777,12 +28060,12 @@ void ThriftHiveMetastoreClient::recv_exchange_partition(Partition& _return) iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - if (fname.compare("exchange_partition") != 0) { + if (fname.compare("partition_name_to_spec") != 0) { iprot_->skip(::apache::thrift::protocol::T_STRUCT); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - ThriftHiveMetastore_exchange_partition_presult result; + ThriftHiveMetastore_partition_name_to_spec_presult result; result.success = &_return; result.read(iprot_); iprot_->readMessageEnd(); @@ -24795,35 +28078,25 @@ void ThriftHiveMetastoreClient::recv_exchange_partition(Partition& _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"); + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "partition_name_to_spec 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) +void ThriftHiveMetastoreClient::markPartitionForEvent(const std::string& db_name, const std::string& tbl_name, const std::map & part_vals, const PartitionEventType::type eventType) { - send_get_partition_with_auth(db_name, tbl_name, part_vals, user_name, group_names); - recv_get_partition_with_auth(_return); + send_markPartitionForEvent(db_name, tbl_name, part_vals, eventType); + recv_markPartitionForEvent(); } -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) +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("get_partition_with_auth", ::apache::thrift::protocol::T_CALL, cseqid); + oprot_->writeMessageBegin("markPartitionForEvent", ::apache::thrift::protocol::T_CALL, cseqid); - ThriftHiveMetastore_get_partition_with_auth_pargs args; + ThriftHiveMetastore_markPartitionForEvent_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.eventType = &eventType; args.write(oprot_); oprot_->writeMessageEnd(); @@ -24831,7 +28104,7 @@ void ThriftHiveMetastoreClient::send_get_partition_with_auth(const std::string& oprot_->getTransport()->flush(); } -void ThriftHiveMetastoreClient::recv_get_partition_with_auth(Partition& _return) +void ThriftHiveMetastoreClient::recv_markPartitionForEvent() { int32_t rseqid = 0; @@ -24851,45 +28124,53 @@ void ThriftHiveMetastoreClient::recv_get_partition_with_auth(Partition& _return) iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - if (fname.compare("get_partition_with_auth") != 0) { + if (fname.compare("markPartitionForEvent") != 0) { iprot_->skip(::apache::thrift::protocol::T_STRUCT); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - ThriftHiveMetastore_get_partition_with_auth_presult result; - result.success = &_return; + ThriftHiveMetastore_markPartitionForEvent_presult result; 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"); + 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; } -void ThriftHiveMetastoreClient::get_partition_by_name(Partition& _return, const std::string& db_name, const std::string& tbl_name, const std::string& part_name) +bool ThriftHiveMetastoreClient::isPartitionMarkedForEvent(const std::string& db_name, const std::string& tbl_name, const std::map & part_vals, const PartitionEventType::type eventType) { - send_get_partition_by_name(db_name, tbl_name, part_name); - recv_get_partition_by_name(_return); + send_isPartitionMarkedForEvent(db_name, tbl_name, part_vals, eventType); + return recv_isPartitionMarkedForEvent(); } -void ThriftHiveMetastoreClient::send_get_partition_by_name(const std::string& db_name, const std::string& tbl_name, const std::string& part_name) +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("get_partition_by_name", ::apache::thrift::protocol::T_CALL, cseqid); + oprot_->writeMessageBegin("isPartitionMarkedForEvent", ::apache::thrift::protocol::T_CALL, cseqid); - ThriftHiveMetastore_get_partition_by_name_pargs args; + ThriftHiveMetastore_isPartitionMarkedForEvent_pargs args; args.db_name = &db_name; args.tbl_name = &tbl_name; - args.part_name = &part_name; + args.part_vals = &part_vals; + args.eventType = &eventType; args.write(oprot_); oprot_->writeMessageEnd(); @@ -24897,7 +28178,7 @@ void ThriftHiveMetastoreClient::send_get_partition_by_name(const std::string& db oprot_->getTransport()->flush(); } -void ThriftHiveMetastoreClient::recv_get_partition_by_name(Partition& _return) +bool ThriftHiveMetastoreClient::recv_isPartitionMarkedForEvent() { int32_t rseqid = 0; @@ -24917,20 +28198,20 @@ void ThriftHiveMetastoreClient::recv_get_partition_by_name(Partition& _return) iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - if (fname.compare("get_partition_by_name") != 0) { + if (fname.compare("isPartitionMarkedForEvent") != 0) { iprot_->skip(::apache::thrift::protocol::T_STRUCT); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - ThriftHiveMetastore_get_partition_by_name_presult result; + bool _return; + ThriftHiveMetastore_isPartitionMarkedForEvent_presult result; result.success = &_return; result.read(iprot_); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); if (result.__isset.success) { - // _return pointer has now been filled - return; + return _return; } if (result.__isset.o1) { throw result.o1; @@ -24938,24 +28219,35 @@ void ThriftHiveMetastoreClient::recv_get_partition_by_name(Partition& _return) if (result.__isset.o2) { throw result.o2; } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_partition_by_name failed: unknown result"); + 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::get_partitions(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const int16_t max_parts) +void ThriftHiveMetastoreClient::add_index(Index& _return, const Index& new_index, const Table& index_table) { - send_get_partitions(db_name, tbl_name, max_parts); - recv_get_partitions(_return); + send_add_index(new_index, index_table); + recv_add_index(_return); } -void ThriftHiveMetastoreClient::send_get_partitions(const std::string& db_name, const std::string& tbl_name, const int16_t max_parts) +void ThriftHiveMetastoreClient::send_add_index(const Index& new_index, const Table& index_table) { int32_t cseqid = 0; - oprot_->writeMessageBegin("get_partitions", ::apache::thrift::protocol::T_CALL, cseqid); + oprot_->writeMessageBegin("add_index", ::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; + ThriftHiveMetastore_add_index_pargs args; + args.new_index = &new_index; + args.index_table = &index_table; args.write(oprot_); oprot_->writeMessageEnd(); @@ -24963,7 +28255,7 @@ void ThriftHiveMetastoreClient::send_get_partitions(const std::string& db_name, oprot_->getTransport()->flush(); } -void ThriftHiveMetastoreClient::recv_get_partitions(std::vector & _return) +void ThriftHiveMetastoreClient::recv_add_index(Index& _return) { int32_t rseqid = 0; @@ -24983,12 +28275,12 @@ void ThriftHiveMetastoreClient::recv_get_partitions(std::vector & _re iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - if (fname.compare("get_partitions") != 0) { + if (fname.compare("add_index") != 0) { iprot_->skip(::apache::thrift::protocol::T_STRUCT); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - ThriftHiveMetastore_get_partitions_presult result; + ThriftHiveMetastore_add_index_presult result; result.success = &_return; result.read(iprot_); iprot_->readMessageEnd(); @@ -25004,26 +28296,28 @@ void ThriftHiveMetastoreClient::recv_get_partitions(std::vector & _re if (result.__isset.o2) { throw result.o2; } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_partitions failed: unknown result"); + if (result.__isset.o3) { + throw result.o3; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "add_index 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) +void ThriftHiveMetastoreClient::alter_index(const std::string& dbname, const std::string& base_tbl_name, const std::string& idx_name, const Index& new_idx) { - send_get_partitions_with_auth(db_name, tbl_name, max_parts, user_name, group_names); - recv_get_partitions_with_auth(_return); + send_alter_index(dbname, base_tbl_name, idx_name, new_idx); + recv_alter_index(); } -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) +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("get_partitions_with_auth", ::apache::thrift::protocol::T_CALL, cseqid); + oprot_->writeMessageBegin("alter_index", ::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; + 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(); @@ -25031,7 +28325,7 @@ void ThriftHiveMetastoreClient::send_get_partitions_with_auth(const std::string& oprot_->getTransport()->flush(); } -void ThriftHiveMetastoreClient::recv_get_partitions_with_auth(std::vector & _return) +void ThriftHiveMetastoreClient::recv_alter_index() { int32_t rseqid = 0; @@ -25051,45 +28345,41 @@ void ThriftHiveMetastoreClient::recv_get_partitions_with_auth(std::vectorreadMessageEnd(); iprot_->getTransport()->readEnd(); } - if (fname.compare("get_partitions_with_auth") != 0) { + if (fname.compare("alter_index") != 0) { iprot_->skip(::apache::thrift::protocol::T_STRUCT); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - ThriftHiveMetastore_get_partitions_with_auth_presult result; - result.success = &_return; + ThriftHiveMetastore_alter_index_presult result; 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"); + return; } -void ThriftHiveMetastoreClient::get_partition_names(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const int16_t max_parts) +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_get_partition_names(db_name, tbl_name, max_parts); - recv_get_partition_names(_return); + send_drop_index_by_name(db_name, tbl_name, index_name, deleteData); + return recv_drop_index_by_name(); } -void ThriftHiveMetastoreClient::send_get_partition_names(const std::string& db_name, const std::string& tbl_name, const int16_t max_parts) +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("get_partition_names", ::apache::thrift::protocol::T_CALL, cseqid); + oprot_->writeMessageBegin("drop_index_by_name", ::apache::thrift::protocol::T_CALL, cseqid); - ThriftHiveMetastore_get_partition_names_pargs args; + ThriftHiveMetastore_drop_index_by_name_pargs args; args.db_name = &db_name; args.tbl_name = &tbl_name; - args.max_parts = &max_parts; + args.index_name = &index_name; + args.deleteData = &deleteData; args.write(oprot_); oprot_->writeMessageEnd(); @@ -25097,7 +28387,7 @@ void ThriftHiveMetastoreClient::send_get_partition_names(const std::string& db_n oprot_->getTransport()->flush(); } -void ThriftHiveMetastoreClient::recv_get_partition_names(std::vector & _return) +bool ThriftHiveMetastoreClient::recv_drop_index_by_name() { int32_t rseqid = 0; @@ -25117,43 +28407,45 @@ void ThriftHiveMetastoreClient::recv_get_partition_names(std::vectorreadMessageEnd(); iprot_->getTransport()->readEnd(); } - if (fname.compare("get_partition_names") != 0) { + if (fname.compare("drop_index_by_name") != 0) { iprot_->skip(::apache::thrift::protocol::T_STRUCT); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - ThriftHiveMetastore_get_partition_names_presult result; + 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 pointer has now been filled - return; + 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_partition_names failed: unknown result"); + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "drop_index_by_name 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) +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_partitions_ps(db_name, tbl_name, part_vals, max_parts); - recv_get_partitions_ps(_return); + send_get_index_by_name(db_name, tbl_name, index_name); + recv_get_index_by_name(_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) +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_partitions_ps", ::apache::thrift::protocol::T_CALL, cseqid); + oprot_->writeMessageBegin("get_index_by_name", ::apache::thrift::protocol::T_CALL, cseqid); - ThriftHiveMetastore_get_partitions_ps_pargs args; + ThriftHiveMetastore_get_index_by_name_pargs args; args.db_name = &db_name; args.tbl_name = &tbl_name; - args.part_vals = &part_vals; - args.max_parts = &max_parts; + args.index_name = &index_name; args.write(oprot_); oprot_->writeMessageEnd(); @@ -25161,7 +28453,7 @@ void ThriftHiveMetastoreClient::send_get_partitions_ps(const std::string& db_nam oprot_->getTransport()->flush(); } -void ThriftHiveMetastoreClient::recv_get_partitions_ps(std::vector & _return) +void ThriftHiveMetastoreClient::recv_get_index_by_name(Index& _return) { int32_t rseqid = 0; @@ -25181,12 +28473,12 @@ void ThriftHiveMetastoreClient::recv_get_partitions_ps(std::vector & iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - if (fname.compare("get_partitions_ps") != 0) { + if (fname.compare("get_index_by_name") != 0) { iprot_->skip(::apache::thrift::protocol::T_STRUCT); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - ThriftHiveMetastore_get_partitions_ps_presult result; + ThriftHiveMetastore_get_index_by_name_presult result; result.success = &_return; result.read(iprot_); iprot_->readMessageEnd(); @@ -25202,27 +28494,24 @@ void ThriftHiveMetastoreClient::recv_get_partitions_ps(std::vector & if (result.__isset.o2) { throw result.o2; } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_partitions_ps failed: unknown result"); + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_index_by_name 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) +void ThriftHiveMetastoreClient::get_indexes(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const int16_t max_indexes) { - 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); + send_get_indexes(db_name, tbl_name, max_indexes); + recv_get_indexes(_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) +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_partitions_ps_with_auth", ::apache::thrift::protocol::T_CALL, cseqid); + oprot_->writeMessageBegin("get_indexes", ::apache::thrift::protocol::T_CALL, cseqid); - ThriftHiveMetastore_get_partitions_ps_with_auth_pargs args; + ThriftHiveMetastore_get_indexes_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.max_indexes = &max_indexes; args.write(oprot_); oprot_->writeMessageEnd(); @@ -25230,7 +28519,7 @@ void ThriftHiveMetastoreClient::send_get_partitions_ps_with_auth(const std::stri oprot_->getTransport()->flush(); } -void ThriftHiveMetastoreClient::recv_get_partitions_ps_with_auth(std::vector & _return) +void ThriftHiveMetastoreClient::recv_get_indexes(std::vector & _return) { int32_t rseqid = 0; @@ -25250,12 +28539,12 @@ void ThriftHiveMetastoreClient::recv_get_partitions_ps_with_auth(std::vectorreadMessageEnd(); iprot_->getTransport()->readEnd(); } - if (fname.compare("get_partitions_ps_with_auth") != 0) { + if (fname.compare("get_indexes") != 0) { iprot_->skip(::apache::thrift::protocol::T_STRUCT); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - ThriftHiveMetastore_get_partitions_ps_with_auth_presult result; + ThriftHiveMetastore_get_indexes_presult result; result.success = &_return; result.read(iprot_); iprot_->readMessageEnd(); @@ -25271,25 +28560,24 @@ void ThriftHiveMetastoreClient::recv_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) +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_partition_names_ps(db_name, tbl_name, part_vals, max_parts); - recv_get_partition_names_ps(_return); + send_get_index_names(db_name, tbl_name, max_indexes); + recv_get_index_names(_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) +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_partition_names_ps", ::apache::thrift::protocol::T_CALL, cseqid); + oprot_->writeMessageBegin("get_index_names", ::apache::thrift::protocol::T_CALL, cseqid); - ThriftHiveMetastore_get_partition_names_ps_pargs args; + ThriftHiveMetastore_get_index_names_pargs args; args.db_name = &db_name; args.tbl_name = &tbl_name; - args.part_vals = &part_vals; - args.max_parts = &max_parts; + args.max_indexes = &max_indexes; args.write(oprot_); oprot_->writeMessageEnd(); @@ -25297,7 +28585,7 @@ void ThriftHiveMetastoreClient::send_get_partition_names_ps(const std::string& d oprot_->getTransport()->flush(); } -void ThriftHiveMetastoreClient::recv_get_partition_names_ps(std::vector & _return) +void ThriftHiveMetastoreClient::recv_get_index_names(std::vector & _return) { int32_t rseqid = 0; @@ -25317,12 +28605,12 @@ void ThriftHiveMetastoreClient::recv_get_partition_names_ps(std::vectorreadMessageEnd(); iprot_->getTransport()->readEnd(); } - if (fname.compare("get_partition_names_ps") != 0) { + if (fname.compare("get_index_names") != 0) { iprot_->skip(::apache::thrift::protocol::T_STRUCT); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - ThriftHiveMetastore_get_partition_names_ps_presult result; + ThriftHiveMetastore_get_index_names_presult result; result.success = &_return; result.read(iprot_); iprot_->readMessageEnd(); @@ -25332,31 +28620,25 @@ void ThriftHiveMetastoreClient::recv_get_partition_names_ps(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const std::string& filter, const int16_t max_parts) +bool ThriftHiveMetastoreClient::update_table_column_statistics(const ColumnStatistics& stats_obj) { - send_get_partitions_by_filter(db_name, tbl_name, filter, max_parts); - recv_get_partitions_by_filter(_return); + send_update_table_column_statistics(stats_obj); + return recv_update_table_column_statistics(); } -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) +void ThriftHiveMetastoreClient::send_update_table_column_statistics(const ColumnStatistics& stats_obj) { int32_t cseqid = 0; - oprot_->writeMessageBegin("get_partitions_by_filter", ::apache::thrift::protocol::T_CALL, cseqid); + oprot_->writeMessageBegin("update_table_column_statistics", ::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; + ThriftHiveMetastore_update_table_column_statistics_pargs args; + args.stats_obj = &stats_obj; args.write(oprot_); oprot_->writeMessageEnd(); @@ -25364,7 +28646,7 @@ void ThriftHiveMetastoreClient::send_get_partitions_by_filter(const std::string& oprot_->getTransport()->flush(); } -void ThriftHiveMetastoreClient::recv_get_partitions_by_filter(std::vector & _return) +bool ThriftHiveMetastoreClient::recv_update_table_column_statistics() { int32_t rseqid = 0; @@ -25384,20 +28666,20 @@ void ThriftHiveMetastoreClient::recv_get_partitions_by_filter(std::vectorreadMessageEnd(); iprot_->getTransport()->readEnd(); } - if (fname.compare("get_partitions_by_filter") != 0) { + if (fname.compare("update_table_column_statistics") != 0) { iprot_->skip(::apache::thrift::protocol::T_STRUCT); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - ThriftHiveMetastore_get_partitions_by_filter_presult result; + 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 pointer has now been filled - return; + return _return; } if (result.__isset.o1) { throw result.o1; @@ -25405,22 +28687,28 @@ void ThriftHiveMetastoreClient::recv_get_partitions_by_filter(std::vectorwriteMessageBegin("get_partitions_by_expr", ::apache::thrift::protocol::T_CALL, cseqid); + oprot_->writeMessageBegin("update_partition_column_statistics", ::apache::thrift::protocol::T_CALL, cseqid); - ThriftHiveMetastore_get_partitions_by_expr_pargs args; - args.req = &req; + ThriftHiveMetastore_update_partition_column_statistics_pargs args; + args.stats_obj = &stats_obj; args.write(oprot_); oprot_->writeMessageEnd(); @@ -25428,7 +28716,7 @@ void ThriftHiveMetastoreClient::send_get_partitions_by_expr(const PartitionsByEx oprot_->getTransport()->flush(); } -void ThriftHiveMetastoreClient::recv_get_partitions_by_expr(PartitionsByExprResult& _return) +bool ThriftHiveMetastoreClient::recv_update_partition_column_statistics() { int32_t rseqid = 0; @@ -25448,20 +28736,20 @@ void ThriftHiveMetastoreClient::recv_get_partitions_by_expr(PartitionsByExprResu iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - if (fname.compare("get_partitions_by_expr") != 0) { + if (fname.compare("update_partition_column_statistics") != 0) { iprot_->skip(::apache::thrift::protocol::T_STRUCT); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - ThriftHiveMetastore_get_partitions_by_expr_presult result; + 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 pointer has now been filled - return; + return _return; } if (result.__isset.o1) { throw result.o1; @@ -25469,24 +28757,30 @@ void ThriftHiveMetastoreClient::recv_get_partitions_by_expr(PartitionsByExprResu if (result.__isset.o2) { throw result.o2; } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_partitions_by_expr failed: unknown result"); + 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_partitions_by_names(std::vector & _return, const std::string& db_name, const std::string& tbl_name, const std::vector & names) +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_partitions_by_names(db_name, tbl_name, names); - recv_get_partitions_by_names(_return); + send_get_table_column_statistics(db_name, tbl_name, col_name); + recv_get_table_column_statistics(_return); } -void ThriftHiveMetastoreClient::send_get_partitions_by_names(const std::string& db_name, const std::string& tbl_name, const std::vector & names) +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_partitions_by_names", ::apache::thrift::protocol::T_CALL, cseqid); + oprot_->writeMessageBegin("get_table_column_statistics", ::apache::thrift::protocol::T_CALL, cseqid); - ThriftHiveMetastore_get_partitions_by_names_pargs args; + ThriftHiveMetastore_get_table_column_statistics_pargs args; args.db_name = &db_name; args.tbl_name = &tbl_name; - args.names = &names; + args.col_name = &col_name; args.write(oprot_); oprot_->writeMessageEnd(); @@ -25494,7 +28788,7 @@ void ThriftHiveMetastoreClient::send_get_partitions_by_names(const std::string& oprot_->getTransport()->flush(); } -void ThriftHiveMetastoreClient::recv_get_partitions_by_names(std::vector & _return) +void ThriftHiveMetastoreClient::recv_get_table_column_statistics(ColumnStatistics& _return) { int32_t rseqid = 0; @@ -25514,12 +28808,12 @@ void ThriftHiveMetastoreClient::recv_get_partitions_by_names(std::vectorreadMessageEnd(); iprot_->getTransport()->readEnd(); } - if (fname.compare("get_partitions_by_names") != 0) { + if (fname.compare("get_table_column_statistics") != 0) { iprot_->skip(::apache::thrift::protocol::T_STRUCT); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - ThriftHiveMetastore_get_partitions_by_names_presult result; + ThriftHiveMetastore_get_table_column_statistics_presult result; result.success = &_return; result.read(iprot_); iprot_->readMessageEnd(); @@ -25535,24 +28829,31 @@ void ThriftHiveMetastoreClient::recv_get_partitions_by_names(std::vectorwriteMessageBegin("alter_partition", ::apache::thrift::protocol::T_CALL, cseqid); + oprot_->writeMessageBegin("get_partition_column_statistics", ::apache::thrift::protocol::T_CALL, cseqid); - ThriftHiveMetastore_alter_partition_pargs args; + ThriftHiveMetastore_get_partition_column_statistics_pargs args; args.db_name = &db_name; args.tbl_name = &tbl_name; - args.new_part = &new_part; + args.part_name = &part_name; + args.col_name = &col_name; args.write(oprot_); oprot_->writeMessageEnd(); @@ -25560,7 +28861,7 @@ void ThriftHiveMetastoreClient::send_alter_partition(const std::string& db_name, oprot_->getTransport()->flush(); } -void ThriftHiveMetastoreClient::recv_alter_partition() +void ThriftHiveMetastoreClient::recv_get_partition_column_statistics(ColumnStatistics& _return) { int32_t rseqid = 0; @@ -25580,40 +28881,49 @@ void ThriftHiveMetastoreClient::recv_alter_partition() iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - if (fname.compare("alter_partition") != 0) { + if (fname.compare("get_partition_column_statistics") != 0) { iprot_->skip(::apache::thrift::protocol::T_STRUCT); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - ThriftHiveMetastore_alter_partition_presult result; + 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; } - return; + 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::alter_partitions(const std::string& db_name, const std::string& tbl_name, const std::vector & new_parts) +void ThriftHiveMetastoreClient::get_table_statistics_req(TableStatsResult& _return, const TableStatsRequest& request) { - send_alter_partitions(db_name, tbl_name, new_parts); - recv_alter_partitions(); + send_get_table_statistics_req(request); + recv_get_table_statistics_req(_return); } -void ThriftHiveMetastoreClient::send_alter_partitions(const std::string& db_name, const std::string& tbl_name, const std::vector & new_parts) +void ThriftHiveMetastoreClient::send_get_table_statistics_req(const TableStatsRequest& request) { int32_t cseqid = 0; - oprot_->writeMessageBegin("alter_partitions", ::apache::thrift::protocol::T_CALL, cseqid); + oprot_->writeMessageBegin("get_table_statistics_req", ::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; + ThriftHiveMetastore_get_table_statistics_req_pargs args; + args.request = &request; args.write(oprot_); oprot_->writeMessageEnd(); @@ -25621,7 +28931,7 @@ void ThriftHiveMetastoreClient::send_alter_partitions(const std::string& db_name oprot_->getTransport()->flush(); } -void ThriftHiveMetastoreClient::recv_alter_partitions() +void ThriftHiveMetastoreClient::recv_get_table_statistics_req(TableStatsResult& _return) { int32_t rseqid = 0; @@ -25641,41 +28951,43 @@ void ThriftHiveMetastoreClient::recv_alter_partitions() iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - if (fname.compare("alter_partitions") != 0) { + if (fname.compare("get_table_statistics_req") != 0) { iprot_->skip(::apache::thrift::protocol::T_STRUCT); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - ThriftHiveMetastore_alter_partitions_presult result; + 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; } - return; + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_table_statistics_req failed: unknown result"); } -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) +void ThriftHiveMetastoreClient::get_partitions_statistics_req(PartitionsStatsResult& _return, const PartitionsStatsRequest& request) { - send_alter_partition_with_environment_context(db_name, tbl_name, new_part, environment_context); - recv_alter_partition_with_environment_context(); + send_get_partitions_statistics_req(request); + recv_get_partitions_statistics_req(_return); } -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) +void ThriftHiveMetastoreClient::send_get_partitions_statistics_req(const PartitionsStatsRequest& request) { int32_t cseqid = 0; - oprot_->writeMessageBegin("alter_partition_with_environment_context", ::apache::thrift::protocol::T_CALL, cseqid); + oprot_->writeMessageBegin("get_partitions_statistics_req", ::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; + ThriftHiveMetastore_get_partitions_statistics_req_pargs args; + args.request = &request; args.write(oprot_); oprot_->writeMessageEnd(); @@ -25683,7 +28995,7 @@ void ThriftHiveMetastoreClient::send_alter_partition_with_environment_context(co oprot_->getTransport()->flush(); } -void ThriftHiveMetastoreClient::recv_alter_partition_with_environment_context() +void ThriftHiveMetastoreClient::recv_get_partitions_statistics_req(PartitionsStatsResult& _return) { int32_t rseqid = 0; @@ -25703,41 +29015,46 @@ void ThriftHiveMetastoreClient::recv_alter_partition_with_environment_context() iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - if (fname.compare("alter_partition_with_environment_context") != 0) { + if (fname.compare("get_partitions_statistics_req") != 0) { iprot_->skip(::apache::thrift::protocol::T_STRUCT); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - ThriftHiveMetastore_alter_partition_with_environment_context_presult result; + 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; } - return; + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_partitions_statistics_req failed: unknown result"); } -void ThriftHiveMetastoreClient::rename_partition(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const Partition& new_part) +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_rename_partition(db_name, tbl_name, part_vals, new_part); - recv_rename_partition(); + send_delete_partition_column_statistics(db_name, tbl_name, part_name, col_name); + return recv_delete_partition_column_statistics(); } -void ThriftHiveMetastoreClient::send_rename_partition(const std::string& db_name, const std::string& tbl_name, const std::vector & part_vals, const Partition& new_part) +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("rename_partition", ::apache::thrift::protocol::T_CALL, cseqid); + oprot_->writeMessageBegin("delete_partition_column_statistics", ::apache::thrift::protocol::T_CALL, cseqid); - ThriftHiveMetastore_rename_partition_pargs args; + ThriftHiveMetastore_delete_partition_column_statistics_pargs args; args.db_name = &db_name; args.tbl_name = &tbl_name; - args.part_vals = &part_vals; - args.new_part = &new_part; + args.part_name = &part_name; + args.col_name = &col_name; args.write(oprot_); oprot_->writeMessageEnd(); @@ -25745,7 +29062,7 @@ void ThriftHiveMetastoreClient::send_rename_partition(const std::string& db_name oprot_->getTransport()->flush(); } -void ThriftHiveMetastoreClient::recv_rename_partition() +bool ThriftHiveMetastoreClient::recv_delete_partition_column_statistics() { int32_t rseqid = 0; @@ -25765,39 +29082,51 @@ void ThriftHiveMetastoreClient::recv_rename_partition() iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - if (fname.compare("rename_partition") != 0) { + if (fname.compare("delete_partition_column_statistics") != 0) { iprot_->skip(::apache::thrift::protocol::T_STRUCT); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - ThriftHiveMetastore_rename_partition_presult result; + 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; } - return; + 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::partition_name_has_valid_characters(const std::vector & part_vals, const bool throw_exception) +bool ThriftHiveMetastoreClient::delete_table_column_statistics(const std::string& db_name, const std::string& tbl_name, const std::string& col_name) { - send_partition_name_has_valid_characters(part_vals, throw_exception); - return recv_partition_name_has_valid_characters(); + send_delete_table_column_statistics(db_name, tbl_name, col_name); + return recv_delete_table_column_statistics(); } -void ThriftHiveMetastoreClient::send_partition_name_has_valid_characters(const std::vector & part_vals, const bool throw_exception) +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("partition_name_has_valid_characters", ::apache::thrift::protocol::T_CALL, cseqid); + oprot_->writeMessageBegin("delete_table_column_statistics", ::apache::thrift::protocol::T_CALL, cseqid); - ThriftHiveMetastore_partition_name_has_valid_characters_pargs args; - args.part_vals = &part_vals; - args.throw_exception = &throw_exception; + 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(); @@ -25805,7 +29134,7 @@ void ThriftHiveMetastoreClient::send_partition_name_has_valid_characters(const s oprot_->getTransport()->flush(); } -bool ThriftHiveMetastoreClient::recv_partition_name_has_valid_characters() +bool ThriftHiveMetastoreClient::recv_delete_table_column_statistics() { int32_t rseqid = 0; @@ -25825,13 +29154,13 @@ bool ThriftHiveMetastoreClient::recv_partition_name_has_valid_characters() iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - if (fname.compare("partition_name_has_valid_characters") != 0) { + if (fname.compare("delete_table_column_statistics") != 0) { iprot_->skip(::apache::thrift::protocol::T_STRUCT); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } bool _return; - ThriftHiveMetastore_partition_name_has_valid_characters_presult result; + ThriftHiveMetastore_delete_table_column_statistics_presult result; result.success = &_return; result.read(iprot_); iprot_->readMessageEnd(); @@ -25843,23 +29172,31 @@ bool ThriftHiveMetastoreClient::recv_partition_name_has_valid_characters() if (result.__isset.o1) { throw result.o1; } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "partition_name_has_valid_characters failed: unknown result"); + 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::get_config_value(std::string& _return, const std::string& name, const std::string& defaultValue) +void ThriftHiveMetastoreClient::create_function(const Function& func) { - send_get_config_value(name, defaultValue); - recv_get_config_value(_return); + send_create_function(func); + recv_create_function(); } -void ThriftHiveMetastoreClient::send_get_config_value(const std::string& name, const std::string& defaultValue) +void ThriftHiveMetastoreClient::send_create_function(const Function& func) { int32_t cseqid = 0; - oprot_->writeMessageBegin("get_config_value", ::apache::thrift::protocol::T_CALL, cseqid); + oprot_->writeMessageBegin("create_function", ::apache::thrift::protocol::T_CALL, cseqid); - ThriftHiveMetastore_get_config_value_pargs args; - args.name = &name; - args.defaultValue = &defaultValue; + ThriftHiveMetastore_create_function_pargs args; + args.func = &func; args.write(oprot_); oprot_->writeMessageEnd(); @@ -25867,7 +29204,7 @@ void ThriftHiveMetastoreClient::send_get_config_value(const std::string& name, c oprot_->getTransport()->flush(); } -void ThriftHiveMetastoreClient::recv_get_config_value(std::string& _return) +void ThriftHiveMetastoreClient::recv_create_function() { int32_t rseqid = 0; @@ -25887,40 +29224,45 @@ void ThriftHiveMetastoreClient::recv_get_config_value(std::string& _return) iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - if (fname.compare("get_config_value") != 0) { + if (fname.compare("create_function") != 0) { iprot_->skip(::apache::thrift::protocol::T_STRUCT); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - ThriftHiveMetastore_get_config_value_presult result; - result.success = &_return; + ThriftHiveMetastore_create_function_presult result; 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"); + if (result.__isset.o2) { + throw result.o2; + } + if (result.__isset.o3) { + throw result.o3; + } + if (result.__isset.o4) { + throw result.o4; + } + return; } -void ThriftHiveMetastoreClient::partition_name_to_vals(std::vector & _return, const std::string& part_name) +void ThriftHiveMetastoreClient::drop_function(const std::string& dbName, const std::string& funcName) { - send_partition_name_to_vals(part_name); - recv_partition_name_to_vals(_return); + send_drop_function(dbName, funcName); + recv_drop_function(); } -void ThriftHiveMetastoreClient::send_partition_name_to_vals(const std::string& part_name) +void ThriftHiveMetastoreClient::send_drop_function(const std::string& dbName, const std::string& funcName) { int32_t cseqid = 0; - oprot_->writeMessageBegin("partition_name_to_vals", ::apache::thrift::protocol::T_CALL, cseqid); + oprot_->writeMessageBegin("drop_function", ::apache::thrift::protocol::T_CALL, cseqid); - ThriftHiveMetastore_partition_name_to_vals_pargs args; - args.part_name = &part_name; + ThriftHiveMetastore_drop_function_pargs args; + args.dbName = &dbName; + args.funcName = &funcName; args.write(oprot_); oprot_->writeMessageEnd(); @@ -25928,7 +29270,7 @@ void ThriftHiveMetastoreClient::send_partition_name_to_vals(const std::string& p oprot_->getTransport()->flush(); } -void ThriftHiveMetastoreClient::recv_partition_name_to_vals(std::vector & _return) +void ThriftHiveMetastoreClient::recv_drop_function() { int32_t rseqid = 0; @@ -25948,40 +29290,40 @@ void ThriftHiveMetastoreClient::recv_partition_name_to_vals(std::vectorreadMessageEnd(); iprot_->getTransport()->readEnd(); } - if (fname.compare("partition_name_to_vals") != 0) { + if (fname.compare("drop_function") != 0) { iprot_->skip(::apache::thrift::protocol::T_STRUCT); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - ThriftHiveMetastore_partition_name_to_vals_presult result; - result.success = &_return; + ThriftHiveMetastore_drop_function_presult result; 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"); + if (result.__isset.o3) { + throw result.o3; + } + return; } -void ThriftHiveMetastoreClient::partition_name_to_spec(std::map & _return, const std::string& part_name) +void ThriftHiveMetastoreClient::alter_function(const std::string& dbName, const std::string& funcName, const Function& newFunc) { - send_partition_name_to_spec(part_name); - recv_partition_name_to_spec(_return); + send_alter_function(dbName, funcName, newFunc); + recv_alter_function(); } -void ThriftHiveMetastoreClient::send_partition_name_to_spec(const std::string& part_name) +void ThriftHiveMetastoreClient::send_alter_function(const std::string& dbName, const std::string& funcName, const Function& newFunc) { int32_t cseqid = 0; - oprot_->writeMessageBegin("partition_name_to_spec", ::apache::thrift::protocol::T_CALL, cseqid); + oprot_->writeMessageBegin("alter_function", ::apache::thrift::protocol::T_CALL, cseqid); - ThriftHiveMetastore_partition_name_to_spec_pargs args; - args.part_name = &part_name; + ThriftHiveMetastore_alter_function_pargs args; + args.dbName = &dbName; + args.funcName = &funcName; + args.newFunc = &newFunc; args.write(oprot_); oprot_->writeMessageEnd(); @@ -25989,7 +29331,7 @@ void ThriftHiveMetastoreClient::send_partition_name_to_spec(const std::string& p oprot_->getTransport()->flush(); } -void ThriftHiveMetastoreClient::recv_partition_name_to_spec(std::map & _return) +void ThriftHiveMetastoreClient::recv_alter_function() { int32_t rseqid = 0; @@ -26009,43 +29351,39 @@ void ThriftHiveMetastoreClient::recv_partition_name_to_spec(std::mapreadMessageEnd(); iprot_->getTransport()->readEnd(); } - if (fname.compare("partition_name_to_spec") != 0) { + if (fname.compare("alter_function") != 0) { iprot_->skip(::apache::thrift::protocol::T_STRUCT); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - ThriftHiveMetastore_partition_name_to_spec_presult result; - result.success = &_return; + ThriftHiveMetastore_alter_function_presult result; 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"); + if (result.__isset.o2) { + throw result.o2; + } + return; } -void ThriftHiveMetastoreClient::markPartitionForEvent(const std::string& db_name, const std::string& tbl_name, const std::map & part_vals, const PartitionEventType::type eventType) +void ThriftHiveMetastoreClient::get_functions(std::vector & _return, const std::string& dbName, const std::string& pattern) { - send_markPartitionForEvent(db_name, tbl_name, part_vals, eventType); - recv_markPartitionForEvent(); + send_get_functions(dbName, pattern); + recv_get_functions(_return); } -void ThriftHiveMetastoreClient::send_markPartitionForEvent(const std::string& db_name, const std::string& tbl_name, const std::map & part_vals, const PartitionEventType::type eventType) +void ThriftHiveMetastoreClient::send_get_functions(const std::string& dbName, const std::string& pattern) { int32_t cseqid = 0; - oprot_->writeMessageBegin("markPartitionForEvent", ::apache::thrift::protocol::T_CALL, cseqid); + oprot_->writeMessageBegin("get_functions", ::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; + ThriftHiveMetastore_get_functions_pargs args; + args.dbName = &dbName; + args.pattern = &pattern; args.write(oprot_); oprot_->writeMessageEnd(); @@ -26053,7 +29391,7 @@ void ThriftHiveMetastoreClient::send_markPartitionForEvent(const std::string& db oprot_->getTransport()->flush(); } -void ThriftHiveMetastoreClient::recv_markPartitionForEvent() +void ThriftHiveMetastoreClient::recv_get_functions(std::vector & _return) { int32_t rseqid = 0; @@ -26073,53 +29411,41 @@ void ThriftHiveMetastoreClient::recv_markPartitionForEvent() iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - if (fname.compare("markPartitionForEvent") != 0) { + if (fname.compare("get_functions") != 0) { iprot_->skip(::apache::thrift::protocol::T_STRUCT); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - ThriftHiveMetastore_markPartitionForEvent_presult result; + 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; } - 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; + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_functions failed: unknown result"); } -bool ThriftHiveMetastoreClient::isPartitionMarkedForEvent(const std::string& db_name, const std::string& tbl_name, const std::map & part_vals, const PartitionEventType::type eventType) +void ThriftHiveMetastoreClient::get_function(Function& _return, const std::string& dbName, const std::string& funcName) { - send_isPartitionMarkedForEvent(db_name, tbl_name, part_vals, eventType); - return recv_isPartitionMarkedForEvent(); + send_get_function(dbName, funcName); + recv_get_function(_return); } -void ThriftHiveMetastoreClient::send_isPartitionMarkedForEvent(const std::string& db_name, const std::string& tbl_name, const std::map & part_vals, const PartitionEventType::type eventType) +void ThriftHiveMetastoreClient::send_get_function(const std::string& dbName, const std::string& funcName) { int32_t cseqid = 0; - oprot_->writeMessageBegin("isPartitionMarkedForEvent", ::apache::thrift::protocol::T_CALL, cseqid); + oprot_->writeMessageBegin("get_function", ::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; + ThriftHiveMetastore_get_function_pargs args; + args.dbName = &dbName; + args.funcName = &funcName; args.write(oprot_); oprot_->writeMessageEnd(); @@ -26127,7 +29453,7 @@ void ThriftHiveMetastoreClient::send_isPartitionMarkedForEvent(const std::string oprot_->getTransport()->flush(); } -bool ThriftHiveMetastoreClient::recv_isPartitionMarkedForEvent() +void ThriftHiveMetastoreClient::recv_get_function(Function& _return) { int32_t rseqid = 0; @@ -26147,20 +29473,20 @@ bool ThriftHiveMetastoreClient::recv_isPartitionMarkedForEvent() iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - if (fname.compare("isPartitionMarkedForEvent") != 0) { + if (fname.compare("get_function") != 0) { iprot_->skip(::apache::thrift::protocol::T_STRUCT); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - bool _return; - ThriftHiveMetastore_isPartitionMarkedForEvent_presult result; + ThriftHiveMetastore_get_function_presult result; result.success = &_return; result.read(iprot_); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); if (result.__isset.success) { - return _return; + // _return pointer has now been filled + return; } if (result.__isset.o1) { throw result.o1; @@ -26168,35 +29494,22 @@ bool ThriftHiveMetastoreClient::recv_isPartitionMarkedForEvent() 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"); + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_function failed: unknown result"); } -void ThriftHiveMetastoreClient::add_index(Index& _return, const Index& new_index, const Table& index_table) +bool ThriftHiveMetastoreClient::create_role(const Role& role) { - send_add_index(new_index, index_table); - recv_add_index(_return); + send_create_role(role); + return recv_create_role(); } -void ThriftHiveMetastoreClient::send_add_index(const Index& new_index, const Table& index_table) +void ThriftHiveMetastoreClient::send_create_role(const Role& role) { int32_t cseqid = 0; - oprot_->writeMessageBegin("add_index", ::apache::thrift::protocol::T_CALL, cseqid); + oprot_->writeMessageBegin("create_role", ::apache::thrift::protocol::T_CALL, cseqid); - ThriftHiveMetastore_add_index_pargs args; - args.new_index = &new_index; - args.index_table = &index_table; + ThriftHiveMetastore_create_role_pargs args; + args.role = &role; args.write(oprot_); oprot_->writeMessageEnd(); @@ -26204,7 +29517,7 @@ void ThriftHiveMetastoreClient::send_add_index(const Index& new_index, const Tab oprot_->getTransport()->flush(); } -void ThriftHiveMetastoreClient::recv_add_index(Index& _return) +bool ThriftHiveMetastoreClient::recv_create_role() { int32_t rseqid = 0; @@ -26224,49 +29537,40 @@ void ThriftHiveMetastoreClient::recv_add_index(Index& _return) iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - if (fname.compare("add_index") != 0) { + if (fname.compare("create_role") != 0) { iprot_->skip(::apache::thrift::protocol::T_STRUCT); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - ThriftHiveMetastore_add_index_presult result; + bool _return; + ThriftHiveMetastore_create_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; + 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_index failed: unknown result"); + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "create_role 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) +bool ThriftHiveMetastoreClient::drop_role(const std::string& role_name) { - send_alter_index(dbname, base_tbl_name, idx_name, new_idx); - recv_alter_index(); + send_drop_role(role_name); + return recv_drop_role(); } -void ThriftHiveMetastoreClient::send_alter_index(const std::string& dbname, const std::string& base_tbl_name, const std::string& idx_name, const Index& new_idx) +void ThriftHiveMetastoreClient::send_drop_role(const std::string& role_name) { int32_t cseqid = 0; - oprot_->writeMessageBegin("alter_index", ::apache::thrift::protocol::T_CALL, cseqid); + oprot_->writeMessageBegin("drop_role", ::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; + ThriftHiveMetastore_drop_role_pargs args; + args.role_name = &role_name; args.write(oprot_); oprot_->writeMessageEnd(); @@ -26274,7 +29578,7 @@ void ThriftHiveMetastoreClient::send_alter_index(const std::string& dbname, cons oprot_->getTransport()->flush(); } -void ThriftHiveMetastoreClient::recv_alter_index() +bool ThriftHiveMetastoreClient::recv_drop_role() { int32_t rseqid = 0; @@ -26294,41 +29598,39 @@ void ThriftHiveMetastoreClient::recv_alter_index() iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - if (fname.compare("alter_index") != 0) { + if (fname.compare("drop_role") != 0) { iprot_->skip(::apache::thrift::protocol::T_STRUCT); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - ThriftHiveMetastore_alter_index_presult result; + 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; } - if (result.__isset.o2) { - throw result.o2; - } - return; + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "drop_role failed: unknown result"); } -bool ThriftHiveMetastoreClient::drop_index_by_name(const std::string& db_name, const std::string& tbl_name, const std::string& index_name, const bool deleteData) +void ThriftHiveMetastoreClient::get_role_names(std::vector & _return) { - send_drop_index_by_name(db_name, tbl_name, index_name, deleteData); - return recv_drop_index_by_name(); + send_get_role_names(); + recv_get_role_names(_return); } -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) +void ThriftHiveMetastoreClient::send_get_role_names() { int32_t cseqid = 0; - oprot_->writeMessageBegin("drop_index_by_name", ::apache::thrift::protocol::T_CALL, cseqid); + oprot_->writeMessageBegin("get_role_names", ::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; + ThriftHiveMetastore_get_role_names_pargs args; args.write(oprot_); oprot_->writeMessageEnd(); @@ -26336,7 +29638,7 @@ void ThriftHiveMetastoreClient::send_drop_index_by_name(const std::string& db_na oprot_->getTransport()->flush(); } -bool ThriftHiveMetastoreClient::recv_drop_index_by_name() +void ThriftHiveMetastoreClient::recv_get_role_names(std::vector & _return) { int32_t rseqid = 0; @@ -26356,45 +29658,45 @@ bool ThriftHiveMetastoreClient::recv_drop_index_by_name() iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - if (fname.compare("drop_index_by_name") != 0) { + if (fname.compare("get_role_names") != 0) { iprot_->skip(::apache::thrift::protocol::T_STRUCT); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - bool _return; - ThriftHiveMetastore_drop_index_by_name_presult result; + ThriftHiveMetastore_get_role_names_presult result; result.success = &_return; result.read(iprot_); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); if (result.__isset.success) { - return _return; + // _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_index_by_name failed: unknown result"); + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_role_names 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) +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_get_index_by_name(db_name, tbl_name, index_name); - recv_get_index_by_name(_return); + send_grant_role(role_name, principal_name, principal_type, grantor, grantorType, grant_option); + return recv_grant_role(); } -void ThriftHiveMetastoreClient::send_get_index_by_name(const std::string& db_name, const std::string& tbl_name, const std::string& index_name) +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("get_index_by_name", ::apache::thrift::protocol::T_CALL, cseqid); + oprot_->writeMessageBegin("grant_role", ::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; + 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(); @@ -26402,7 +29704,7 @@ void ThriftHiveMetastoreClient::send_get_index_by_name(const std::string& db_nam oprot_->getTransport()->flush(); } -void ThriftHiveMetastoreClient::recv_get_index_by_name(Index& _return) +bool ThriftHiveMetastoreClient::recv_grant_role() { int32_t rseqid = 0; @@ -26422,45 +29724,42 @@ void ThriftHiveMetastoreClient::recv_get_index_by_name(Index& _return) iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - if (fname.compare("get_index_by_name") != 0) { + if (fname.compare("grant_role") != 0) { iprot_->skip(::apache::thrift::protocol::T_STRUCT); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - ThriftHiveMetastore_get_index_by_name_presult result; + bool _return; + ThriftHiveMetastore_grant_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; + 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_index_by_name failed: unknown result"); + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "grant_role 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) +bool ThriftHiveMetastoreClient::revoke_role(const std::string& role_name, const std::string& principal_name, const PrincipalType::type principal_type) { - send_get_indexes(db_name, tbl_name, max_indexes); - recv_get_indexes(_return); + send_revoke_role(role_name, principal_name, principal_type); + return recv_revoke_role(); } -void ThriftHiveMetastoreClient::send_get_indexes(const std::string& db_name, const std::string& tbl_name, const int16_t max_indexes) +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("get_indexes", ::apache::thrift::protocol::T_CALL, cseqid); + oprot_->writeMessageBegin("revoke_role", ::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; + 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(); @@ -26468,7 +29767,7 @@ void ThriftHiveMetastoreClient::send_get_indexes(const std::string& db_name, con oprot_->getTransport()->flush(); } -void ThriftHiveMetastoreClient::recv_get_indexes(std::vector & _return) +bool ThriftHiveMetastoreClient::recv_revoke_role() { int32_t rseqid = 0; @@ -26488,45 +29787,41 @@ void ThriftHiveMetastoreClient::recv_get_indexes(std::vector & _return) iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - if (fname.compare("get_indexes") != 0) { + if (fname.compare("revoke_role") != 0) { iprot_->skip(::apache::thrift::protocol::T_STRUCT); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - ThriftHiveMetastore_get_indexes_presult result; + bool _return; + ThriftHiveMetastore_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; + 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_indexes failed: unknown result"); + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "revoke_role 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) +void ThriftHiveMetastoreClient::list_roles(std::vector & _return, const std::string& principal_name, const PrincipalType::type principal_type) { - send_get_index_names(db_name, tbl_name, max_indexes); - recv_get_index_names(_return); + send_list_roles(principal_name, principal_type); + recv_list_roles(_return); } -void ThriftHiveMetastoreClient::send_get_index_names(const std::string& db_name, const std::string& tbl_name, const int16_t max_indexes) +void ThriftHiveMetastoreClient::send_list_roles(const std::string& principal_name, const PrincipalType::type principal_type) { int32_t cseqid = 0; - oprot_->writeMessageBegin("get_index_names", ::apache::thrift::protocol::T_CALL, cseqid); + oprot_->writeMessageBegin("list_roles", ::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; + ThriftHiveMetastore_list_roles_pargs args; + args.principal_name = &principal_name; + args.principal_type = &principal_type; args.write(oprot_); oprot_->writeMessageEnd(); @@ -26534,7 +29829,7 @@ void ThriftHiveMetastoreClient::send_get_index_names(const std::string& db_name, oprot_->getTransport()->flush(); } -void ThriftHiveMetastoreClient::recv_get_index_names(std::vector & _return) +void ThriftHiveMetastoreClient::recv_list_roles(std::vector & _return) { int32_t rseqid = 0; @@ -26554,12 +29849,12 @@ void ThriftHiveMetastoreClient::recv_get_index_names(std::vector & iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - if (fname.compare("get_index_names") != 0) { + if (fname.compare("list_roles") != 0) { iprot_->skip(::apache::thrift::protocol::T_STRUCT); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - ThriftHiveMetastore_get_index_names_presult result; + ThriftHiveMetastore_list_roles_presult result; result.success = &_return; result.read(iprot_); iprot_->readMessageEnd(); @@ -26569,25 +29864,27 @@ void ThriftHiveMetastoreClient::recv_get_index_names(std::vector & // _return pointer has now been filled return; } - if (result.__isset.o2) { - throw result.o2; + if (result.__isset.o1) { + throw result.o1; } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_index_names failed: unknown result"); + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "list_roles failed: unknown result"); } -bool ThriftHiveMetastoreClient::update_table_column_statistics(const ColumnStatistics& stats_obj) +void ThriftHiveMetastoreClient::get_privilege_set(PrincipalPrivilegeSet& _return, const HiveObjectRef& hiveObject, const std::string& user_name, const std::vector & group_names) { - send_update_table_column_statistics(stats_obj); - return recv_update_table_column_statistics(); + send_get_privilege_set(hiveObject, user_name, group_names); + recv_get_privilege_set(_return); } -void ThriftHiveMetastoreClient::send_update_table_column_statistics(const ColumnStatistics& stats_obj) +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("update_table_column_statistics", ::apache::thrift::protocol::T_CALL, cseqid); + oprot_->writeMessageBegin("get_privilege_set", ::apache::thrift::protocol::T_CALL, cseqid); - ThriftHiveMetastore_update_table_column_statistics_pargs args; - args.stats_obj = &stats_obj; + ThriftHiveMetastore_get_privilege_set_pargs args; + args.hiveObject = &hiveObject; + args.user_name = &user_name; + args.group_names = &group_names; args.write(oprot_); oprot_->writeMessageEnd(); @@ -26595,7 +29892,7 @@ void ThriftHiveMetastoreClient::send_update_table_column_statistics(const Column oprot_->getTransport()->flush(); } -bool ThriftHiveMetastoreClient::recv_update_table_column_statistics() +void ThriftHiveMetastoreClient::recv_get_privilege_set(PrincipalPrivilegeSet& _return) { int32_t rseqid = 0; @@ -26615,49 +29912,42 @@ bool ThriftHiveMetastoreClient::recv_update_table_column_statistics() iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - if (fname.compare("update_table_column_statistics") != 0) { + if (fname.compare("get_privilege_set") != 0) { iprot_->skip(::apache::thrift::protocol::T_STRUCT); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - bool _return; - ThriftHiveMetastore_update_table_column_statistics_presult result; + ThriftHiveMetastore_get_privilege_set_presult result; result.success = &_return; result.read(iprot_); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); if (result.__isset.success) { - return _return; + // _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, "update_table_column_statistics failed: unknown result"); + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_privilege_set failed: unknown result"); } -bool ThriftHiveMetastoreClient::update_partition_column_statistics(const ColumnStatistics& stats_obj) +void ThriftHiveMetastoreClient::list_privileges(std::vector & _return, const std::string& principal_name, const PrincipalType::type principal_type, const HiveObjectRef& hiveObject) { - send_update_partition_column_statistics(stats_obj); - return recv_update_partition_column_statistics(); + send_list_privileges(principal_name, principal_type, hiveObject); + recv_list_privileges(_return); } -void ThriftHiveMetastoreClient::send_update_partition_column_statistics(const ColumnStatistics& stats_obj) +void ThriftHiveMetastoreClient::send_list_privileges(const std::string& principal_name, const PrincipalType::type principal_type, const HiveObjectRef& hiveObject) { int32_t cseqid = 0; - oprot_->writeMessageBegin("update_partition_column_statistics", ::apache::thrift::protocol::T_CALL, cseqid); + oprot_->writeMessageBegin("list_privileges", ::apache::thrift::protocol::T_CALL, cseqid); - ThriftHiveMetastore_update_partition_column_statistics_pargs args; - args.stats_obj = &stats_obj; + ThriftHiveMetastore_list_privileges_pargs args; + args.principal_name = &principal_name; + args.principal_type = &principal_type; + args.hiveObject = &hiveObject; args.write(oprot_); oprot_->writeMessageEnd(); @@ -26665,7 +29955,7 @@ void ThriftHiveMetastoreClient::send_update_partition_column_statistics(const Co oprot_->getTransport()->flush(); } -bool ThriftHiveMetastoreClient::recv_update_partition_column_statistics() +void ThriftHiveMetastoreClient::recv_list_privileges(std::vector & _return) { int32_t rseqid = 0; @@ -26685,51 +29975,40 @@ bool ThriftHiveMetastoreClient::recv_update_partition_column_statistics() iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - if (fname.compare("update_partition_column_statistics") != 0) { + if (fname.compare("list_privileges") != 0) { iprot_->skip(::apache::thrift::protocol::T_STRUCT); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - bool _return; - ThriftHiveMetastore_update_partition_column_statistics_presult result; + ThriftHiveMetastore_list_privileges_presult result; result.success = &_return; result.read(iprot_); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); if (result.__isset.success) { - return _return; + // _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, "update_partition_column_statistics failed: unknown result"); + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "list_privileges 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) +bool ThriftHiveMetastoreClient::grant_privileges(const PrivilegeBag& privileges) { - send_get_table_column_statistics(db_name, tbl_name, col_name); - recv_get_table_column_statistics(_return); + send_grant_privileges(privileges); + return recv_grant_privileges(); } -void ThriftHiveMetastoreClient::send_get_table_column_statistics(const std::string& db_name, const std::string& tbl_name, const std::string& col_name) +void ThriftHiveMetastoreClient::send_grant_privileges(const PrivilegeBag& privileges) { int32_t cseqid = 0; - oprot_->writeMessageBegin("get_table_column_statistics", ::apache::thrift::protocol::T_CALL, cseqid); + oprot_->writeMessageBegin("grant_privileges", ::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; + ThriftHiveMetastore_grant_privileges_pargs args; + args.privileges = &privileges; args.write(oprot_); oprot_->writeMessageEnd(); @@ -26737,7 +30016,7 @@ void ThriftHiveMetastoreClient::send_get_table_column_statistics(const std::stri oprot_->getTransport()->flush(); } -void ThriftHiveMetastoreClient::recv_get_table_column_statistics(ColumnStatistics& _return) +bool ThriftHiveMetastoreClient::recv_grant_privileges() { int32_t rseqid = 0; @@ -26757,52 +30036,40 @@ void ThriftHiveMetastoreClient::recv_get_table_column_statistics(ColumnStatistic iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - if (fname.compare("get_table_column_statistics") != 0) { + if (fname.compare("grant_privileges") != 0) { iprot_->skip(::apache::thrift::protocol::T_STRUCT); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - ThriftHiveMetastore_get_table_column_statistics_presult result; + bool _return; + ThriftHiveMetastore_grant_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; + 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, "get_table_column_statistics failed: unknown result"); + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "grant_privileges 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) +bool ThriftHiveMetastoreClient::revoke_privileges(const PrivilegeBag& privileges) { - send_get_partition_column_statistics(db_name, tbl_name, part_name, col_name); - recv_get_partition_column_statistics(_return); + send_revoke_privileges(privileges); + return recv_revoke_privileges(); } -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) +void ThriftHiveMetastoreClient::send_revoke_privileges(const PrivilegeBag& privileges) { int32_t cseqid = 0; - oprot_->writeMessageBegin("get_partition_column_statistics", ::apache::thrift::protocol::T_CALL, cseqid); + oprot_->writeMessageBegin("revoke_privileges", ::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; + ThriftHiveMetastore_revoke_privileges_pargs args; + args.privileges = &privileges; args.write(oprot_); oprot_->writeMessageEnd(); @@ -26810,7 +30077,7 @@ void ThriftHiveMetastoreClient::send_get_partition_column_statistics(const std:: oprot_->getTransport()->flush(); } -void ThriftHiveMetastoreClient::recv_get_partition_column_statistics(ColumnStatistics& _return) +bool ThriftHiveMetastoreClient::recv_revoke_privileges() { int32_t rseqid = 0; @@ -26830,49 +30097,41 @@ void ThriftHiveMetastoreClient::recv_get_partition_column_statistics(ColumnStati iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - if (fname.compare("get_partition_column_statistics") != 0) { + if (fname.compare("revoke_privileges") != 0) { iprot_->skip(::apache::thrift::protocol::T_STRUCT); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - ThriftHiveMetastore_get_partition_column_statistics_presult result; + bool _return; + ThriftHiveMetastore_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; + 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, "get_partition_column_statistics failed: unknown result"); + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "revoke_privileges failed: unknown result"); } -void ThriftHiveMetastoreClient::get_table_statistics_req(TableStatsResult& _return, const TableStatsRequest& request) +void ThriftHiveMetastoreClient::set_ugi(std::vector & _return, const std::string& user_name, const std::vector & group_names) { - send_get_table_statistics_req(request); - recv_get_table_statistics_req(_return); + send_set_ugi(user_name, group_names); + recv_set_ugi(_return); } -void ThriftHiveMetastoreClient::send_get_table_statistics_req(const TableStatsRequest& request) +void ThriftHiveMetastoreClient::send_set_ugi(const std::string& user_name, const std::vector & group_names) { int32_t cseqid = 0; - oprot_->writeMessageBegin("get_table_statistics_req", ::apache::thrift::protocol::T_CALL, cseqid); + oprot_->writeMessageBegin("set_ugi", ::apache::thrift::protocol::T_CALL, cseqid); - ThriftHiveMetastore_get_table_statistics_req_pargs args; - args.request = &request; + ThriftHiveMetastore_set_ugi_pargs args; + args.user_name = &user_name; + args.group_names = &group_names; args.write(oprot_); oprot_->writeMessageEnd(); @@ -26880,7 +30139,7 @@ void ThriftHiveMetastoreClient::send_get_table_statistics_req(const TableStatsRe oprot_->getTransport()->flush(); } -void ThriftHiveMetastoreClient::recv_get_table_statistics_req(TableStatsResult& _return) +void ThriftHiveMetastoreClient::recv_set_ugi(std::vector & _return) { int32_t rseqid = 0; @@ -26900,12 +30159,12 @@ void ThriftHiveMetastoreClient::recv_get_table_statistics_req(TableStatsResult& iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - if (fname.compare("get_table_statistics_req") != 0) { + if (fname.compare("set_ugi") != 0) { iprot_->skip(::apache::thrift::protocol::T_STRUCT); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - ThriftHiveMetastore_get_table_statistics_req_presult result; + ThriftHiveMetastore_set_ugi_presult result; result.success = &_return; result.read(iprot_); iprot_->readMessageEnd(); @@ -26918,25 +30177,23 @@ void ThriftHiveMetastoreClient::recv_get_table_statistics_req(TableStatsResult& 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"); + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "set_ugi failed: unknown result"); } -void ThriftHiveMetastoreClient::get_partitions_statistics_req(PartitionsStatsResult& _return, const PartitionsStatsRequest& request) +void ThriftHiveMetastoreClient::get_delegation_token(std::string& _return, const std::string& token_owner, const std::string& renewer_kerberos_principal_name) { - send_get_partitions_statistics_req(request); - recv_get_partitions_statistics_req(_return); + send_get_delegation_token(token_owner, renewer_kerberos_principal_name); + recv_get_delegation_token(_return); } -void ThriftHiveMetastoreClient::send_get_partitions_statistics_req(const PartitionsStatsRequest& request) +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_partitions_statistics_req", ::apache::thrift::protocol::T_CALL, cseqid); + oprot_->writeMessageBegin("get_delegation_token", ::apache::thrift::protocol::T_CALL, cseqid); - ThriftHiveMetastore_get_partitions_statistics_req_pargs args; - args.request = &request; + 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(); @@ -26944,7 +30201,7 @@ void ThriftHiveMetastoreClient::send_get_partitions_statistics_req(const Partiti oprot_->getTransport()->flush(); } -void ThriftHiveMetastoreClient::recv_get_partitions_statistics_req(PartitionsStatsResult& _return) +void ThriftHiveMetastoreClient::recv_get_delegation_token(std::string& _return) { int32_t rseqid = 0; @@ -26964,12 +30221,12 @@ void ThriftHiveMetastoreClient::recv_get_partitions_statistics_req(PartitionsSta iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - if (fname.compare("get_partitions_statistics_req") != 0) { + if (fname.compare("get_delegation_token") != 0) { iprot_->skip(::apache::thrift::protocol::T_STRUCT); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - ThriftHiveMetastore_get_partitions_statistics_req_presult result; + ThriftHiveMetastore_get_delegation_token_presult result; result.success = &_return; result.read(iprot_); iprot_->readMessageEnd(); @@ -26982,28 +30239,22 @@ void ThriftHiveMetastoreClient::recv_get_partitions_statistics_req(PartitionsSta 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"); + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_delegation_token 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) +int64_t ThriftHiveMetastoreClient::renew_delegation_token(const std::string& token_str_form) { - send_delete_partition_column_statistics(db_name, tbl_name, part_name, col_name); - return recv_delete_partition_column_statistics(); + send_renew_delegation_token(token_str_form); + return recv_renew_delegation_token(); } -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) +void ThriftHiveMetastoreClient::send_renew_delegation_token(const std::string& token_str_form) { int32_t cseqid = 0; - oprot_->writeMessageBegin("delete_partition_column_statistics", ::apache::thrift::protocol::T_CALL, cseqid); + oprot_->writeMessageBegin("renew_delegation_token", ::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; + ThriftHiveMetastore_renew_delegation_token_pargs args; + args.token_str_form = &token_str_form; args.write(oprot_); oprot_->writeMessageEnd(); @@ -27011,7 +30262,7 @@ void ThriftHiveMetastoreClient::send_delete_partition_column_statistics(const st oprot_->getTransport()->flush(); } -bool ThriftHiveMetastoreClient::recv_delete_partition_column_statistics() +int64_t ThriftHiveMetastoreClient::recv_renew_delegation_token() { int32_t rseqid = 0; @@ -27031,13 +30282,13 @@ bool ThriftHiveMetastoreClient::recv_delete_partition_column_statistics() iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - if (fname.compare("delete_partition_column_statistics") != 0) { + if (fname.compare("renew_delegation_token") != 0) { iprot_->skip(::apache::thrift::protocol::T_STRUCT); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - bool _return; - ThriftHiveMetastore_delete_partition_column_statistics_presult result; + int64_t _return; + ThriftHiveMetastore_renew_delegation_token_presult result; result.success = &_return; result.read(iprot_); iprot_->readMessageEnd(); @@ -27049,33 +30300,22 @@ bool ThriftHiveMetastoreClient::recv_delete_partition_column_statistics() 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"); + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "renew_delegation_token failed: unknown result"); } -bool ThriftHiveMetastoreClient::delete_table_column_statistics(const std::string& db_name, const std::string& tbl_name, const std::string& col_name) +void ThriftHiveMetastoreClient::cancel_delegation_token(const std::string& token_str_form) { - send_delete_table_column_statistics(db_name, tbl_name, col_name); - return recv_delete_table_column_statistics(); + send_cancel_delegation_token(token_str_form); + recv_cancel_delegation_token(); } -void ThriftHiveMetastoreClient::send_delete_table_column_statistics(const std::string& db_name, const std::string& tbl_name, const std::string& col_name) +void ThriftHiveMetastoreClient::send_cancel_delegation_token(const std::string& token_str_form) { int32_t cseqid = 0; - oprot_->writeMessageBegin("delete_table_column_statistics", ::apache::thrift::protocol::T_CALL, cseqid); + oprot_->writeMessageBegin("cancel_delegation_token", ::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; + ThriftHiveMetastore_cancel_delegation_token_pargs args; + args.token_str_form = &token_str_form; args.write(oprot_); oprot_->writeMessageEnd(); @@ -27083,7 +30323,7 @@ void ThriftHiveMetastoreClient::send_delete_table_column_statistics(const std::s oprot_->getTransport()->flush(); } -bool ThriftHiveMetastoreClient::recv_delete_table_column_statistics() +void ThriftHiveMetastoreClient::recv_cancel_delegation_token() { int32_t rseqid = 0; @@ -27103,49 +30343,34 @@ bool ThriftHiveMetastoreClient::recv_delete_table_column_statistics() iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - if (fname.compare("delete_table_column_statistics") != 0) { + if (fname.compare("cancel_delegation_token") != 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; + ThriftHiveMetastore_cancel_delegation_token_presult result; 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"); + return; } -void ThriftHiveMetastoreClient::create_function(const Function& func) +void ThriftHiveMetastoreClient::get_open_txns(GetOpenTxnsResponse& _return) { - send_create_function(func); - recv_create_function(); + send_get_open_txns(); + recv_get_open_txns(_return); } -void ThriftHiveMetastoreClient::send_create_function(const Function& func) +void ThriftHiveMetastoreClient::send_get_open_txns() { int32_t cseqid = 0; - oprot_->writeMessageBegin("create_function", ::apache::thrift::protocol::T_CALL, cseqid); + oprot_->writeMessageBegin("get_open_txns", ::apache::thrift::protocol::T_CALL, cseqid); - ThriftHiveMetastore_create_function_pargs args; - args.func = &func; + ThriftHiveMetastore_get_open_txns_pargs args; args.write(oprot_); oprot_->writeMessageEnd(); @@ -27153,7 +30378,7 @@ void ThriftHiveMetastoreClient::send_create_function(const Function& func) oprot_->getTransport()->flush(); } -void ThriftHiveMetastoreClient::recv_create_function() +void ThriftHiveMetastoreClient::recv_get_open_txns(GetOpenTxnsResponse& _return) { int32_t rseqid = 0; @@ -27173,45 +30398,36 @@ void ThriftHiveMetastoreClient::recv_create_function() iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - if (fname.compare("create_function") != 0) { + if (fname.compare("get_open_txns") != 0) { iprot_->skip(::apache::thrift::protocol::T_STRUCT); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - ThriftHiveMetastore_create_function_presult result; + ThriftHiveMetastore_get_open_txns_presult result; + result.success = &_return; 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.success) { + // _return pointer has now been filled + return; } - return; + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_open_txns failed: unknown result"); } -void ThriftHiveMetastoreClient::drop_function(const std::string& dbName, const std::string& funcName) +void ThriftHiveMetastoreClient::get_open_txns_info(GetOpenTxnsInfoResponse& _return) { - send_drop_function(dbName, funcName); - recv_drop_function(); + send_get_open_txns_info(); + recv_get_open_txns_info(_return); } -void ThriftHiveMetastoreClient::send_drop_function(const std::string& dbName, const std::string& funcName) +void ThriftHiveMetastoreClient::send_get_open_txns_info() { int32_t cseqid = 0; - oprot_->writeMessageBegin("drop_function", ::apache::thrift::protocol::T_CALL, cseqid); + oprot_->writeMessageBegin("get_open_txns_info", ::apache::thrift::protocol::T_CALL, cseqid); - ThriftHiveMetastore_drop_function_pargs args; - args.dbName = &dbName; - args.funcName = &funcName; + ThriftHiveMetastore_get_open_txns_info_pargs args; args.write(oprot_); oprot_->writeMessageEnd(); @@ -27219,7 +30435,7 @@ void ThriftHiveMetastoreClient::send_drop_function(const std::string& dbName, co oprot_->getTransport()->flush(); } -void ThriftHiveMetastoreClient::recv_drop_function() +void ThriftHiveMetastoreClient::recv_get_open_txns_info(GetOpenTxnsInfoResponse& _return) { int32_t rseqid = 0; @@ -27239,40 +30455,37 @@ void ThriftHiveMetastoreClient::recv_drop_function() iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - if (fname.compare("drop_function") != 0) { + if (fname.compare("get_open_txns_info") != 0) { iprot_->skip(::apache::thrift::protocol::T_STRUCT); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - ThriftHiveMetastore_drop_function_presult result; + ThriftHiveMetastore_get_open_txns_info_presult result; + result.success = &_return; result.read(iprot_); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); - if (result.__isset.o1) { - throw result.o1; - } - if (result.__isset.o3) { - throw result.o3; + if (result.__isset.success) { + // _return pointer has now been filled + return; } - return; + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_open_txns_info failed: unknown result"); } -void ThriftHiveMetastoreClient::alter_function(const std::string& dbName, const std::string& funcName, const Function& newFunc) +void ThriftHiveMetastoreClient::open_txns(OpenTxnsResponse& _return, const OpenTxnRequest& rqst) { - send_alter_function(dbName, funcName, newFunc); - recv_alter_function(); + send_open_txns(rqst); + recv_open_txns(_return); } -void ThriftHiveMetastoreClient::send_alter_function(const std::string& dbName, const std::string& funcName, const Function& newFunc) +void ThriftHiveMetastoreClient::send_open_txns(const OpenTxnRequest& rqst) { int32_t cseqid = 0; - oprot_->writeMessageBegin("alter_function", ::apache::thrift::protocol::T_CALL, cseqid); + oprot_->writeMessageBegin("open_txns", ::apache::thrift::protocol::T_CALL, cseqid); - ThriftHiveMetastore_alter_function_pargs args; - args.dbName = &dbName; - args.funcName = &funcName; - args.newFunc = &newFunc; + ThriftHiveMetastore_open_txns_pargs args; + args.rqst = &rqst; args.write(oprot_); oprot_->writeMessageEnd(); @@ -27280,7 +30493,7 @@ void ThriftHiveMetastoreClient::send_alter_function(const std::string& dbName, c oprot_->getTransport()->flush(); } -void ThriftHiveMetastoreClient::recv_alter_function() +void ThriftHiveMetastoreClient::recv_open_txns(OpenTxnsResponse& _return) { int32_t rseqid = 0; @@ -27300,39 +30513,37 @@ void ThriftHiveMetastoreClient::recv_alter_function() iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - if (fname.compare("alter_function") != 0) { + if (fname.compare("open_txns") != 0) { iprot_->skip(::apache::thrift::protocol::T_STRUCT); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - ThriftHiveMetastore_alter_function_presult result; + ThriftHiveMetastore_open_txns_presult result; + result.success = &_return; 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.success) { + // _return pointer has now been filled + return; } - return; + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "open_txns failed: unknown result"); } -void ThriftHiveMetastoreClient::get_functions(std::vector & _return, const std::string& dbName, const std::string& pattern) +void ThriftHiveMetastoreClient::abort_txn(const AbortTxnRequest& rqst) { - send_get_functions(dbName, pattern); - recv_get_functions(_return); + send_abort_txn(rqst); + recv_abort_txn(); } -void ThriftHiveMetastoreClient::send_get_functions(const std::string& dbName, const std::string& pattern) +void ThriftHiveMetastoreClient::send_abort_txn(const AbortTxnRequest& rqst) { int32_t cseqid = 0; - oprot_->writeMessageBegin("get_functions", ::apache::thrift::protocol::T_CALL, cseqid); + oprot_->writeMessageBegin("abort_txn", ::apache::thrift::protocol::T_CALL, cseqid); - ThriftHiveMetastore_get_functions_pargs args; - args.dbName = &dbName; - args.pattern = &pattern; + ThriftHiveMetastore_abort_txn_pargs args; + args.rqst = &rqst; args.write(oprot_); oprot_->writeMessageEnd(); @@ -27340,7 +30551,7 @@ void ThriftHiveMetastoreClient::send_get_functions(const std::string& dbName, co oprot_->getTransport()->flush(); } -void ThriftHiveMetastoreClient::recv_get_functions(std::vector & _return) +void ThriftHiveMetastoreClient::recv_abort_txn() { int32_t rseqid = 0; @@ -27360,41 +30571,35 @@ void ThriftHiveMetastoreClient::recv_get_functions(std::vector & _r iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - if (fname.compare("get_functions") != 0) { + if (fname.compare("abort_txn") != 0) { iprot_->skip(::apache::thrift::protocol::T_STRUCT); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - ThriftHiveMetastore_get_functions_presult result; - result.success = &_return; + ThriftHiveMetastore_abort_txn_presult result; 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"); + return; } -void ThriftHiveMetastoreClient::get_function(Function& _return, const std::string& dbName, const std::string& funcName) +void ThriftHiveMetastoreClient::commit_txn(const CommitTxnRequest& rqst) { - send_get_function(dbName, funcName); - recv_get_function(_return); + send_commit_txn(rqst); + recv_commit_txn(); } -void ThriftHiveMetastoreClient::send_get_function(const std::string& dbName, const std::string& funcName) +void ThriftHiveMetastoreClient::send_commit_txn(const CommitTxnRequest& rqst) { int32_t cseqid = 0; - oprot_->writeMessageBegin("get_function", ::apache::thrift::protocol::T_CALL, cseqid); + oprot_->writeMessageBegin("commit_txn", ::apache::thrift::protocol::T_CALL, cseqid); - ThriftHiveMetastore_get_function_pargs args; - args.dbName = &dbName; - args.funcName = &funcName; + ThriftHiveMetastore_commit_txn_pargs args; + args.rqst = &rqst; args.write(oprot_); oprot_->writeMessageEnd(); @@ -27402,7 +30607,7 @@ void ThriftHiveMetastoreClient::send_get_function(const std::string& dbName, con oprot_->getTransport()->flush(); } -void ThriftHiveMetastoreClient::recv_get_function(Function& _return) +void ThriftHiveMetastoreClient::recv_commit_txn() { int32_t rseqid = 0; @@ -27422,43 +30627,38 @@ void ThriftHiveMetastoreClient::recv_get_function(Function& _return) iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - if (fname.compare("get_function") != 0) { + if (fname.compare("commit_txn") != 0) { iprot_->skip(::apache::thrift::protocol::T_STRUCT); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - ThriftHiveMetastore_get_function_presult result; - result.success = &_return; + ThriftHiveMetastore_commit_txn_presult result; 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"); + return; } -bool ThriftHiveMetastoreClient::create_role(const Role& role) +void ThriftHiveMetastoreClient::lock(LockResponse& _return, const LockRequest& rqst) { - send_create_role(role); - return recv_create_role(); + send_lock(rqst); + recv_lock(_return); } -void ThriftHiveMetastoreClient::send_create_role(const Role& role) +void ThriftHiveMetastoreClient::send_lock(const LockRequest& rqst) { int32_t cseqid = 0; - oprot_->writeMessageBegin("create_role", ::apache::thrift::protocol::T_CALL, cseqid); + oprot_->writeMessageBegin("lock", ::apache::thrift::protocol::T_CALL, cseqid); - ThriftHiveMetastore_create_role_pargs args; - args.role = &role; + ThriftHiveMetastore_lock_pargs args; + args.rqst = &rqst; args.write(oprot_); oprot_->writeMessageEnd(); @@ -27466,7 +30666,7 @@ void ThriftHiveMetastoreClient::send_create_role(const Role& role) oprot_->getTransport()->flush(); } -bool ThriftHiveMetastoreClient::recv_create_role() +void ThriftHiveMetastoreClient::recv_lock(LockResponse& _return) { int32_t rseqid = 0; @@ -27486,40 +30686,43 @@ bool ThriftHiveMetastoreClient::recv_create_role() iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - if (fname.compare("create_role") != 0) { + if (fname.compare("lock") != 0) { iprot_->skip(::apache::thrift::protocol::T_STRUCT); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - bool _return; - ThriftHiveMetastore_create_role_presult result; + ThriftHiveMetastore_lock_presult result; result.success = &_return; result.read(iprot_); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); if (result.__isset.success) { - return _return; + // _return pointer has now been filled + return; } if (result.__isset.o1) { throw result.o1; } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "create_role failed: unknown result"); + if (result.__isset.o2) { + throw result.o2; + } + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "lock failed: unknown result"); } -bool ThriftHiveMetastoreClient::drop_role(const std::string& role_name) +void ThriftHiveMetastoreClient::check_lock(LockResponse& _return, const CheckLockRequest& rqst) { - send_drop_role(role_name); - return recv_drop_role(); + send_check_lock(rqst); + recv_check_lock(_return); } -void ThriftHiveMetastoreClient::send_drop_role(const std::string& role_name) +void ThriftHiveMetastoreClient::send_check_lock(const CheckLockRequest& rqst) { int32_t cseqid = 0; - oprot_->writeMessageBegin("drop_role", ::apache::thrift::protocol::T_CALL, cseqid); + oprot_->writeMessageBegin("check_lock", ::apache::thrift::protocol::T_CALL, cseqid); - ThriftHiveMetastore_drop_role_pargs args; - args.role_name = &role_name; + ThriftHiveMetastore_check_lock_pargs args; + args.rqst = &rqst; args.write(oprot_); oprot_->writeMessageEnd(); @@ -27527,7 +30730,7 @@ void ThriftHiveMetastoreClient::send_drop_role(const std::string& role_name) oprot_->getTransport()->flush(); } -bool ThriftHiveMetastoreClient::recv_drop_role() +void ThriftHiveMetastoreClient::recv_check_lock(LockResponse& _return) { int32_t rseqid = 0; @@ -27547,39 +30750,46 @@ bool ThriftHiveMetastoreClient::recv_drop_role() iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - if (fname.compare("drop_role") != 0) { + if (fname.compare("check_lock") != 0) { iprot_->skip(::apache::thrift::protocol::T_STRUCT); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - bool _return; - ThriftHiveMetastore_drop_role_presult result; + ThriftHiveMetastore_check_lock_presult result; result.success = &_return; result.read(iprot_); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); if (result.__isset.success) { - return _return; + // _return pointer has now been filled + return; } if (result.__isset.o1) { throw result.o1; } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "drop_role failed: unknown result"); + 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::get_role_names(std::vector & _return) +void ThriftHiveMetastoreClient::unlock(const UnlockRequest& rqst) { - send_get_role_names(); - recv_get_role_names(_return); + send_unlock(rqst); + recv_unlock(); } -void ThriftHiveMetastoreClient::send_get_role_names() +void ThriftHiveMetastoreClient::send_unlock(const UnlockRequest& rqst) { int32_t cseqid = 0; - oprot_->writeMessageBegin("get_role_names", ::apache::thrift::protocol::T_CALL, cseqid); + oprot_->writeMessageBegin("unlock", ::apache::thrift::protocol::T_CALL, cseqid); - ThriftHiveMetastore_get_role_names_pargs args; + ThriftHiveMetastore_unlock_pargs args; + args.rqst = &rqst; args.write(oprot_); oprot_->writeMessageEnd(); @@ -27587,7 +30797,7 @@ void ThriftHiveMetastoreClient::send_get_role_names() oprot_->getTransport()->flush(); } -void ThriftHiveMetastoreClient::recv_get_role_names(std::vector & _return) +void ThriftHiveMetastoreClient::recv_unlock() { int32_t rseqid = 0; @@ -27607,45 +30817,38 @@ void ThriftHiveMetastoreClient::recv_get_role_names(std::vector & _ iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - if (fname.compare("get_role_names") != 0) { + if (fname.compare("unlock") != 0) { iprot_->skip(::apache::thrift::protocol::T_STRUCT); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - ThriftHiveMetastore_get_role_names_presult result; - result.success = &_return; + ThriftHiveMetastore_unlock_presult result; 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"); + if (result.__isset.o2) { + throw result.o2; + } + return; } -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) +void ThriftHiveMetastoreClient::show_locks(ShowLocksResponse& _return, const ShowLocksRequest& rqst) { - send_grant_role(role_name, principal_name, principal_type, grantor, grantorType, grant_option); - return recv_grant_role(); + send_show_locks(rqst); + recv_show_locks(_return); } -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) +void ThriftHiveMetastoreClient::send_show_locks(const ShowLocksRequest& rqst) { int32_t cseqid = 0; - oprot_->writeMessageBegin("grant_role", ::apache::thrift::protocol::T_CALL, cseqid); + oprot_->writeMessageBegin("show_locks", ::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; + ThriftHiveMetastore_show_locks_pargs args; + args.rqst = &rqst; args.write(oprot_); oprot_->writeMessageEnd(); @@ -27653,7 +30856,7 @@ void ThriftHiveMetastoreClient::send_grant_role(const std::string& role_name, co oprot_->getTransport()->flush(); } -bool ThriftHiveMetastoreClient::recv_grant_role() +void ThriftHiveMetastoreClient::recv_show_locks(ShowLocksResponse& _return) { int32_t rseqid = 0; @@ -27673,42 +30876,37 @@ bool ThriftHiveMetastoreClient::recv_grant_role() iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - if (fname.compare("grant_role") != 0) { + if (fname.compare("show_locks") != 0) { iprot_->skip(::apache::thrift::protocol::T_STRUCT); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - bool _return; - ThriftHiveMetastore_grant_role_presult result; + ThriftHiveMetastore_show_locks_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; + // _return pointer has now been filled + return; } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "grant_role failed: unknown result"); + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "show_locks failed: unknown result"); } -bool ThriftHiveMetastoreClient::revoke_role(const std::string& role_name, const std::string& principal_name, const PrincipalType::type principal_type) +void ThriftHiveMetastoreClient::heartbeat(const HeartbeatRequest& ids) { - send_revoke_role(role_name, principal_name, principal_type); - return recv_revoke_role(); + send_heartbeat(ids); + recv_heartbeat(); } -void ThriftHiveMetastoreClient::send_revoke_role(const std::string& role_name, const std::string& principal_name, const PrincipalType::type principal_type) +void ThriftHiveMetastoreClient::send_heartbeat(const HeartbeatRequest& ids) { int32_t cseqid = 0; - oprot_->writeMessageBegin("revoke_role", ::apache::thrift::protocol::T_CALL, cseqid); + oprot_->writeMessageBegin("heartbeat", ::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; + ThriftHiveMetastore_heartbeat_pargs args; + args.ids = &ids; args.write(oprot_); oprot_->writeMessageEnd(); @@ -27716,7 +30914,7 @@ void ThriftHiveMetastoreClient::send_revoke_role(const std::string& role_name, c oprot_->getTransport()->flush(); } -bool ThriftHiveMetastoreClient::recv_revoke_role() +void ThriftHiveMetastoreClient::recv_heartbeat() { int32_t rseqid = 0; @@ -27736,41 +30934,41 @@ bool ThriftHiveMetastoreClient::recv_revoke_role() iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - if (fname.compare("revoke_role") != 0) { + if (fname.compare("heartbeat") != 0) { iprot_->skip(::apache::thrift::protocol::T_STRUCT); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - bool _return; - ThriftHiveMetastore_revoke_role_presult result; - result.success = &_return; + ThriftHiveMetastore_heartbeat_presult result; 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"); + if (result.__isset.o2) { + throw result.o2; + } + if (result.__isset.o3) { + throw result.o3; + } + return; } -void ThriftHiveMetastoreClient::list_roles(std::vector & _return, const std::string& principal_name, const PrincipalType::type principal_type) +void ThriftHiveMetastoreClient::compact(const CompactionRequest& rqst) { - send_list_roles(principal_name, principal_type); - recv_list_roles(_return); + send_compact(rqst); + recv_compact(); } -void ThriftHiveMetastoreClient::send_list_roles(const std::string& principal_name, const PrincipalType::type principal_type) +void ThriftHiveMetastoreClient::send_compact(const CompactionRequest& rqst) { int32_t cseqid = 0; - oprot_->writeMessageBegin("list_roles", ::apache::thrift::protocol::T_CALL, cseqid); + oprot_->writeMessageBegin("compact", ::apache::thrift::protocol::T_CALL, cseqid); - ThriftHiveMetastore_list_roles_pargs args; - args.principal_name = &principal_name; - args.principal_type = &principal_type; + ThriftHiveMetastore_compact_pargs args; + args.rqst = &rqst; args.write(oprot_); oprot_->writeMessageEnd(); @@ -27778,7 +30976,7 @@ void ThriftHiveMetastoreClient::send_list_roles(const std::string& principal_nam oprot_->getTransport()->flush(); } -void ThriftHiveMetastoreClient::recv_list_roles(std::vector & _return) +void ThriftHiveMetastoreClient::recv_compact() { int32_t rseqid = 0; @@ -27798,42 +30996,32 @@ void ThriftHiveMetastoreClient::recv_list_roles(std::vector & _return) iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - if (fname.compare("list_roles") != 0) { + if (fname.compare("compact") != 0) { iprot_->skip(::apache::thrift::protocol::T_STRUCT); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - ThriftHiveMetastore_list_roles_presult result; - result.success = &_return; + ThriftHiveMetastore_compact_presult result; 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"); + return; } -void ThriftHiveMetastoreClient::get_privilege_set(PrincipalPrivilegeSet& _return, const HiveObjectRef& hiveObject, const std::string& user_name, const std::vector & group_names) +void ThriftHiveMetastoreClient::show_compact(ShowCompactResponse& _return, const ShowCompactRequest& rqst) { - send_get_privilege_set(hiveObject, user_name, group_names); - recv_get_privilege_set(_return); + send_show_compact(rqst); + recv_show_compact(_return); } -void ThriftHiveMetastoreClient::send_get_privilege_set(const HiveObjectRef& hiveObject, const std::string& user_name, const std::vector & group_names) +void ThriftHiveMetastoreClient::send_show_compact(const ShowCompactRequest& rqst) { int32_t cseqid = 0; - oprot_->writeMessageBegin("get_privilege_set", ::apache::thrift::protocol::T_CALL, cseqid); + oprot_->writeMessageBegin("show_compact", ::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; + ThriftHiveMetastore_show_compact_pargs args; + args.rqst = &rqst; args.write(oprot_); oprot_->writeMessageEnd(); @@ -27841,7 +31029,7 @@ void ThriftHiveMetastoreClient::send_get_privilege_set(const HiveObjectRef& hive oprot_->getTransport()->flush(); } -void ThriftHiveMetastoreClient::recv_get_privilege_set(PrincipalPrivilegeSet& _return) +void ThriftHiveMetastoreClient::recv_show_compact(ShowCompactResponse& _return) { int32_t rseqid = 0; @@ -27861,12 +31049,12 @@ void ThriftHiveMetastoreClient::recv_get_privilege_set(PrincipalPrivilegeSet& _r iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - if (fname.compare("get_privilege_set") != 0) { + if (fname.compare("show_compact") != 0) { iprot_->skip(::apache::thrift::protocol::T_STRUCT); iprot_->readMessageEnd(); iprot_->getTransport()->readEnd(); } - ThriftHiveMetastore_get_privilege_set_presult result; + ThriftHiveMetastore_show_compact_presult result; result.success = &_return; result.read(iprot_); iprot_->readMessageEnd(); @@ -27876,472 +31064,766 @@ void ThriftHiveMetastoreClient::recv_get_privilege_set(PrincipalPrivilegeSet& _r // _return pointer has now been filled return; } - if (result.__isset.o1) { - throw result.o1; + throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "show_compact 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()) { + return ::facebook::fb303::FacebookServiceProcessor::dispatchCall(iprot, oprot, fname, seqid, callContext); } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "get_privilege_set failed: unknown result"); + (this->*(pfn->second))(seqid, iprot, oprot, callContext); + return true; } -void ThriftHiveMetastoreClient::list_privileges(std::vector & _return, const std::string& principal_name, const PrincipalType::type principal_type, const HiveObjectRef& hiveObject) +void ThriftHiveMetastoreProcessor::process_create_database(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) { - send_list_privileges(principal_name, principal_type, hiveObject); - recv_list_privileges(_return); + 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 ThriftHiveMetastoreClient::send_list_privileges(const std::string& principal_name, const PrincipalType::type principal_type, const HiveObjectRef& hiveObject) +void ThriftHiveMetastoreProcessor::process_get_database(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) { - int32_t cseqid = 0; - oprot_->writeMessageBegin("list_privileges", ::apache::thrift::protocol::T_CALL, cseqid); + 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"); - ThriftHiveMetastore_list_privileges_pargs args; - args.principal_name = &principal_name; - args.principal_type = &principal_type; - args.hiveObject = &hiveObject; - args.write(oprot_); + 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_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); + 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 ThriftHiveMetastoreClient::recv_list_privileges(std::vector & _return) +void ThriftHiveMetastoreProcessor::process_get_databases(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) { - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_databases", callContext); } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_databases"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_databases"); } - if (fname.compare("list_privileges") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); + + 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_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 + 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 (result.__isset.o1) { - throw result.o1; + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_databases"); } - 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(); + 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 ThriftHiveMetastoreClient::send_grant_privileges(const PrivilegeBag& privileges) +void ThriftHiveMetastoreProcessor::process_get_all_databases(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) { - int32_t cseqid = 0; - oprot_->writeMessageBegin("grant_privileges", ::apache::thrift::protocol::T_CALL, cseqid); + 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"); - ThriftHiveMetastore_grant_privileges_pargs args; - args.privileges = &privileges; - args.write(oprot_); + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_all_databases"); + } - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} + ThriftHiveMetastore_get_all_databases_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); -bool ThriftHiveMetastoreClient::recv_grant_privileges() -{ + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_all_databases", bytes); + } - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; + 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"); + } - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - 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(); + ::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; } - 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; + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_all_databases"); } - 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(); + 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 ThriftHiveMetastoreClient::send_revoke_privileges(const PrivilegeBag& privileges) +void ThriftHiveMetastoreProcessor::process_alter_database(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) { - int32_t cseqid = 0; - oprot_->writeMessageBegin("revoke_privileges", ::apache::thrift::protocol::T_CALL, cseqid); + 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"); - ThriftHiveMetastore_revoke_privileges_pargs args; - args.privileges = &privileges; - args.write(oprot_); + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.alter_database"); + } - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} + ThriftHiveMetastore_alter_database_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); -bool ThriftHiveMetastoreClient::recv_revoke_privileges() -{ + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.alter_database", bytes); + } - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; + 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"); + } - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - 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(); + ::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; } - 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 (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.alter_database"); } - if (result.__isset.o1) { - throw result.o1; + + 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); } - throw ::apache::thrift::TApplicationException(::apache::thrift::TApplicationException::MISSING_RESULT, "revoke_privileges failed: unknown result"); } -void ThriftHiveMetastoreClient::set_ugi(std::vector & _return, const std::string& user_name, const std::vector & group_names) +void ThriftHiveMetastoreProcessor::process_get_type(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) { - send_set_ugi(user_name, group_names); - recv_set_ugi(_return); -} + 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"); -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); + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_type"); + } - ThriftHiveMetastore_set_ugi_pargs args; - args.user_name = &user_name; - args.group_names = &group_names; - args.write(oprot_); + ThriftHiveMetastore_get_type_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_type", bytes); + } -void ThriftHiveMetastoreClient::recv_set_ugi(std::vector & _return) -{ + 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"); + } - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; + ::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; + } - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_type"); } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); + + 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); } - if (fname.compare("set_ugi") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); +} + +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); } - ThriftHiveMetastore_set_ugi_presult result; - result.success = &_return; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.create_type"); - if (result.__isset.success) { - // _return pointer has now been filled - return; + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.create_type"); } - if (result.__isset.o1) { - throw result.o1; + + 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); } - 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); -} + 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"); + } -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); + ::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; + } - ThriftHiveMetastore_get_delegation_token_pargs args; - args.token_owner = &token_owner; - args.renewer_kerberos_principal_name = &renewer_kerberos_principal_name; - args.write(oprot_); + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.create_type"); + } - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); + 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 ThriftHiveMetastoreClient::recv_get_delegation_token(std::string& _return) +void ThriftHiveMetastoreProcessor::process_drop_type(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) { - - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; + void* ctx = NULL; + if (this->eventHandler_.get() != NULL) { + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.drop_type", callContext); } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.drop_type"); + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.drop_type"); } - if (fname.compare("get_delegation_token") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); + + 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_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 + 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 (result.__isset.o1) { - throw result.o1; + + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.drop_type"); } - 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(); + 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 ThriftHiveMetastoreClient::send_renew_delegation_token(const std::string& token_str_form) +void ThriftHiveMetastoreProcessor::process_get_type_all(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) { - int32_t cseqid = 0; - oprot_->writeMessageBegin("renew_delegation_token", ::apache::thrift::protocol::T_CALL, cseqid); + 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"); - ThriftHiveMetastore_renew_delegation_token_pargs args; - args.token_str_form = &token_str_form; - args.write(oprot_); + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_type_all"); + } - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); -} + ThriftHiveMetastore_get_type_all_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); -int64_t ThriftHiveMetastoreClient::recv_renew_delegation_token() -{ + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_type_all", bytes); + } - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; + 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"); + } - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - 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(); + ::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; } - 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 (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_type_all"); } - if (result.__isset.o1) { - throw result.o1; + + 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); } - 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) +void ThriftHiveMetastoreProcessor::process_get_fields(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) { - send_cancel_delegation_token(token_str_form); - recv_cancel_delegation_token(); -} + 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"); -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); + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_fields"); + } - ThriftHiveMetastore_cancel_delegation_token_pargs args; - args.token_str_form = &token_str_form; - args.write(oprot_); + ThriftHiveMetastore_get_fields_args args; + args.read(iprot); + iprot->readMessageEnd(); + uint32_t bytes = iprot->getTransport()->readEnd(); - oprot_->writeMessageEnd(); - oprot_->getTransport()->writeEnd(); - oprot_->getTransport()->flush(); + 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 ThriftHiveMetastoreClient::recv_cancel_delegation_token() +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"); - int32_t rseqid = 0; - std::string fname; - ::apache::thrift::protocol::TMessageType mtype; - - iprot_->readMessageBegin(fname, mtype, rseqid); - if (mtype == ::apache::thrift::protocol::T_EXCEPTION) { - ::apache::thrift::TApplicationException x; - x.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - throw x; + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_schema"); } - if (mtype != ::apache::thrift::protocol::T_REPLY) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); + + 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); } - if (fname.compare("cancel_delegation_token") != 0) { - iprot_->skip(::apache::thrift::protocol::T_STRUCT); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); + + 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; } - ThriftHiveMetastore_cancel_delegation_token_presult result; - result.read(iprot_); - iprot_->readMessageEnd(); - iprot_->getTransport()->readEnd(); - if (result.__isset.o1) { - throw result.o1; + if (this->eventHandler_.get() != NULL) { + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_schema"); } - return; -} -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()) { - return ::facebook::fb303::FacebookServiceProcessor::dispatchCall(iprot, oprot, fname, seqid, callContext); + 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); } - (this->*(pfn->second))(seqid, iprot, oprot, callContext); - return true; } -void ThriftHiveMetastoreProcessor::process_create_database(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +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_database", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.create_table", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.create_database"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.create_table"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.create_database"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.create_table"); } - ThriftHiveMetastore_create_database_args args; + 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_database", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.create_table", bytes); } - ThriftHiveMetastore_create_database_result result; + ThriftHiveMetastore_create_table_result result; try { - iface_->create_database(args.database); + iface_->create_table(args.tbl); } catch (AlreadyExistsException &o1) { result.o1 = o1; result.__isset.o1 = true; @@ -28351,13 +31833,16 @@ void ThriftHiveMetastoreProcessor::process_create_database(int32_t seqid, ::apac } 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_database"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.create_table"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("create_database", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("create_table", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -28366,58 +31851,63 @@ void ThriftHiveMetastoreProcessor::process_create_database(int32_t seqid, ::apac } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.create_database"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.create_table"); } - oprot->writeMessageBegin("create_database", ::apache::thrift::protocol::T_REPLY, seqid); + 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_database", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.create_table", bytes); } } -void ThriftHiveMetastoreProcessor::process_get_database(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +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.get_database", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.create_table_with_environment_context", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_database"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.create_table_with_environment_context"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_database"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.create_table_with_environment_context"); } - ThriftHiveMetastore_get_database_args args; + 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.get_database", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.create_table_with_environment_context", bytes); } - ThriftHiveMetastore_get_database_result result; + ThriftHiveMetastore_create_table_with_environment_context_result result; try { - iface_->get_database(result.success, args.name); - result.__isset.success = true; - } catch (NoSuchObjectException &o1) { + iface_->create_table_with_environment_context(args.tbl, args.environment_context); + } catch (AlreadyExistsException &o1) { result.o1 = o1; result.__isset.o1 = true; - } catch (MetaException &o2) { + } 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.get_database"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.create_table_with_environment_context"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("get_database", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("create_table_with_environment_context", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -28426,60 +31916,57 @@ void ThriftHiveMetastoreProcessor::process_get_database(int32_t seqid, ::apache: } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_database"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.create_table_with_environment_context"); } - oprot->writeMessageBegin("get_database", ::apache::thrift::protocol::T_REPLY, seqid); + 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.get_database", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.create_table_with_environment_context", bytes); } } -void ThriftHiveMetastoreProcessor::process_drop_database(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +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_database", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.drop_table", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.drop_database"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.drop_table"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.drop_database"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.drop_table"); } - ThriftHiveMetastore_drop_database_args args; + 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_database", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.drop_table", bytes); } - ThriftHiveMetastore_drop_database_result result; + ThriftHiveMetastore_drop_table_result result; try { - iface_->drop_database(args.name, args.deleteData, args.cascade); + iface_->drop_table(args.dbname, args.name, args.deleteData); } 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"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.drop_table"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("drop_database", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("drop_table", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -28488,55 +31975,57 @@ void ThriftHiveMetastoreProcessor::process_drop_database(int32_t seqid, ::apache } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.drop_database"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.drop_table"); } - oprot->writeMessageBegin("drop_database", ::apache::thrift::protocol::T_REPLY, seqid); + 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_database", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.drop_table", bytes); } } -void ThriftHiveMetastoreProcessor::process_get_databases(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +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.get_databases", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.drop_table_with_environment_context", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_databases"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.drop_table_with_environment_context"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_databases"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.drop_table_with_environment_context"); } - ThriftHiveMetastore_get_databases_args args; + 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.get_databases", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.drop_table_with_environment_context", bytes); } - ThriftHiveMetastore_get_databases_result result; + ThriftHiveMetastore_drop_table_with_environment_context_result result; try { - iface_->get_databases(result.success, args.pattern); - result.__isset.success = true; - } catch (MetaException &o1) { + 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.get_databases"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.drop_table_with_environment_context"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("get_databases", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("drop_table_with_environment_context", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -28545,55 +32034,55 @@ void ThriftHiveMetastoreProcessor::process_get_databases(int32_t seqid, ::apache } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_databases"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.drop_table_with_environment_context"); } - oprot->writeMessageBegin("get_databases", ::apache::thrift::protocol::T_REPLY, seqid); + 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.get_databases", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.drop_table_with_environment_context", bytes); } } -void ThriftHiveMetastoreProcessor::process_get_all_databases(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +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_all_databases", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_tables", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_all_databases"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_tables"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_all_databases"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_tables"); } - ThriftHiveMetastore_get_all_databases_args args; + 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_all_databases", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_tables", bytes); } - ThriftHiveMetastore_get_all_databases_result result; + ThriftHiveMetastore_get_tables_result result; try { - iface_->get_all_databases(result.success); + 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_all_databases"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_tables"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("get_all_databases", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("get_tables", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -28602,57 +32091,55 @@ void ThriftHiveMetastoreProcessor::process_get_all_databases(int32_t seqid, ::ap } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_all_databases"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_tables"); } - oprot->writeMessageBegin("get_all_databases", ::apache::thrift::protocol::T_REPLY, seqid); + 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_all_databases", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_tables", bytes); } } -void ThriftHiveMetastoreProcessor::process_alter_database(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +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.alter_database", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_all_tables", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.alter_database"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_all_tables"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.alter_database"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_all_tables"); } - ThriftHiveMetastore_alter_database_args args; + 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.alter_database", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_all_tables", bytes); } - ThriftHiveMetastore_alter_database_result result; + ThriftHiveMetastore_get_all_tables_result result; try { - iface_->alter_database(args.dbname, args.db); + iface_->get_all_tables(result.success, args.db_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.alter_database"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_all_tables"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("alter_database", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("get_all_tables", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -28661,44 +32148,44 @@ void ThriftHiveMetastoreProcessor::process_alter_database(int32_t seqid, ::apach } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.alter_database"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_all_tables"); } - oprot->writeMessageBegin("alter_database", ::apache::thrift::protocol::T_REPLY, seqid); + 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.alter_database", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_all_tables", bytes); } } -void ThriftHiveMetastoreProcessor::process_get_type(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +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_type", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_table", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_type"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_table"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_type"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_table"); } - ThriftHiveMetastore_get_type_args args; + 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_type", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_table", bytes); } - ThriftHiveMetastore_get_type_result result; + ThriftHiveMetastore_get_table_result result; try { - iface_->get_type(result.success, args.name); + iface_->get_table(result.success, args.dbname, args.tbl_name); result.__isset.success = true; } catch (MetaException &o1) { result.o1 = o1; @@ -28708,11 +32195,11 @@ void ThriftHiveMetastoreProcessor::process_get_type(int32_t seqid, ::apache::thr result.__isset.o2 = true; } catch (const std::exception& e) { if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_type"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_table"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("get_type", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("get_table", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -28721,61 +32208,61 @@ void ThriftHiveMetastoreProcessor::process_get_type(int32_t seqid, ::apache::thr } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_type"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_table"); } - oprot->writeMessageBegin("get_type", ::apache::thrift::protocol::T_REPLY, seqid); + 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_type", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_table", bytes); } } -void ThriftHiveMetastoreProcessor::process_create_type(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +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.create_type", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_table_objects_by_name", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.create_type"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_table_objects_by_name"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.create_type"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_table_objects_by_name"); } - ThriftHiveMetastore_create_type_args args; + 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.create_type", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_table_objects_by_name", bytes); } - ThriftHiveMetastore_create_type_result result; + ThriftHiveMetastore_get_table_objects_by_name_result result; try { - result.success = iface_->create_type(args.type); + iface_->get_table_objects_by_name(result.success, args.dbname, args.tbl_names); result.__isset.success = true; - } catch (AlreadyExistsException &o1) { + } catch (MetaException &o1) { result.o1 = o1; result.__isset.o1 = true; - } catch (InvalidObjectException &o2) { + } catch (InvalidOperationException &o2) { result.o2 = o2; result.__isset.o2 = true; - } catch (MetaException &o3) { + } 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.create_type"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_table_objects_by_name"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("create_type", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("get_table_objects_by_name", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -28784,58 +32271,61 @@ void ThriftHiveMetastoreProcessor::process_create_type(int32_t seqid, ::apache:: } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.create_type"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_table_objects_by_name"); } - oprot->writeMessageBegin("create_type", ::apache::thrift::protocol::T_REPLY, seqid); + 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.create_type", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_table_objects_by_name", bytes); } } -void ThriftHiveMetastoreProcessor::process_drop_type(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +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.drop_type", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_table_names_by_filter", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.drop_type"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_table_names_by_filter"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.drop_type"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_table_names_by_filter"); } - ThriftHiveMetastore_drop_type_args args; + 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.drop_type", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_table_names_by_filter", bytes); } - ThriftHiveMetastore_drop_type_result result; + ThriftHiveMetastore_get_table_names_by_filter_result result; try { - result.success = iface_->drop_type(args.type); + 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 (NoSuchObjectException &o2) { + } 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.drop_type"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_table_names_by_filter"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("drop_type", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("get_table_names_by_filter", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -28844,55 +32334,57 @@ void ThriftHiveMetastoreProcessor::process_drop_type(int32_t seqid, ::apache::th } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.drop_type"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_table_names_by_filter"); } - oprot->writeMessageBegin("drop_type", ::apache::thrift::protocol::T_REPLY, seqid); + 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.drop_type", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_table_names_by_filter", bytes); } } -void ThriftHiveMetastoreProcessor::process_get_type_all(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +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.get_type_all", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.alter_table", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_type_all"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.alter_table"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_type_all"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.alter_table"); } - ThriftHiveMetastore_get_type_all_args args; + 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.get_type_all", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.alter_table", bytes); } - ThriftHiveMetastore_get_type_all_result result; + ThriftHiveMetastore_alter_table_result result; try { - iface_->get_type_all(result.success, args.name); - result.__isset.success = true; + 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.get_type_all"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.alter_table"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("get_type_all", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("alter_table", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -28901,61 +32393,57 @@ void ThriftHiveMetastoreProcessor::process_get_type_all(int32_t seqid, ::apache: } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_type_all"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.alter_table"); } - oprot->writeMessageBegin("get_type_all", ::apache::thrift::protocol::T_REPLY, seqid); + 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.get_type_all", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.alter_table", bytes); } } -void ThriftHiveMetastoreProcessor::process_get_fields(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +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.get_fields", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.alter_table_with_environment_context", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_fields"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.alter_table_with_environment_context"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_fields"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.alter_table_with_environment_context"); } - ThriftHiveMetastore_get_fields_args args; + 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.get_fields", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.alter_table_with_environment_context", bytes); } - ThriftHiveMetastore_get_fields_result result; + ThriftHiveMetastore_alter_table_with_environment_context_result result; try { - iface_->get_fields(result.success, args.db_name, args.table_name); - result.__isset.success = true; - } catch (MetaException &o1) { + 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 (UnknownTableException &o2) { + } catch (MetaException &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"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.alter_table_with_environment_context"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("get_fields", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("alter_table_with_environment_context", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -28964,61 +32452,61 @@ void ThriftHiveMetastoreProcessor::process_get_fields(int32_t seqid, ::apache::t } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_fields"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.alter_table_with_environment_context"); } - oprot->writeMessageBegin("get_fields", ::apache::thrift::protocol::T_REPLY, seqid); + 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.get_fields", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.alter_table_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 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.get_schema", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.add_partition", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_schema"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.add_partition"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_schema"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.add_partition"); } - ThriftHiveMetastore_get_schema_args args; + 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.get_schema", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.add_partition", bytes); } - ThriftHiveMetastore_get_schema_result result; + ThriftHiveMetastore_add_partition_result result; try { - iface_->get_schema(result.success, args.db_name, args.table_name); + iface_->add_partition(result.success, args.new_part); result.__isset.success = true; - } catch (MetaException &o1) { + } catch (InvalidObjectException &o1) { result.o1 = o1; result.__isset.o1 = true; - } catch (UnknownTableException &o2) { + } catch (AlreadyExistsException &o2) { result.o2 = o2; result.__isset.o2 = true; - } catch (UnknownDBException &o3) { + } 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.get_schema"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.add_partition"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("get_schema", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("add_partition", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -29027,63 +32515,61 @@ void ThriftHiveMetastoreProcessor::process_get_schema(int32_t seqid, ::apache::t } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_schema"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.add_partition"); } - oprot->writeMessageBegin("get_schema", ::apache::thrift::protocol::T_REPLY, seqid); + 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.get_schema", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.add_partition", bytes); } } -void ThriftHiveMetastoreProcessor::process_create_table(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +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.create_table", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.add_partition_with_environment_context", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.create_table"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.add_partition_with_environment_context"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.create_table"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.add_partition_with_environment_context"); } - ThriftHiveMetastore_create_table_args args; + 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.create_table", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.add_partition_with_environment_context", bytes); } - ThriftHiveMetastore_create_table_result result; + ThriftHiveMetastore_add_partition_with_environment_context_result result; try { - iface_->create_table(args.tbl); - } catch (AlreadyExistsException &o1) { + 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 (InvalidObjectException &o2) { + } catch (AlreadyExistsException &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"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.add_partition_with_environment_context"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("create_table", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("add_partition_with_environment_context", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -29092,63 +32578,61 @@ void ThriftHiveMetastoreProcessor::process_create_table(int32_t seqid, ::apache: } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.create_table"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.add_partition_with_environment_context"); } - oprot->writeMessageBegin("create_table", ::apache::thrift::protocol::T_REPLY, seqid); + 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.create_table", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.add_partition_with_environment_context", 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 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.create_table_with_environment_context", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.add_partitions", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.create_table_with_environment_context"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.add_partitions"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.create_table_with_environment_context"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.add_partitions"); } - ThriftHiveMetastore_create_table_with_environment_context_args args; + 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.create_table_with_environment_context", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.add_partitions", bytes); } - ThriftHiveMetastore_create_table_with_environment_context_result result; + ThriftHiveMetastore_add_partitions_result result; try { - iface_->create_table_with_environment_context(args.tbl, args.environment_context); - } catch (AlreadyExistsException &o1) { + result.success = iface_->add_partitions(args.new_parts); + result.__isset.success = true; + } catch (InvalidObjectException &o1) { result.o1 = o1; result.__isset.o1 = true; - } catch (InvalidObjectException &o2) { + } catch (AlreadyExistsException &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"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.add_partitions"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("create_table_with_environment_context", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("add_partitions", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -29157,57 +32641,61 @@ void ThriftHiveMetastoreProcessor::process_create_table_with_environment_context } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.create_table_with_environment_context"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.add_partitions"); } - oprot->writeMessageBegin("create_table_with_environment_context", ::apache::thrift::protocol::T_REPLY, seqid); + 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.create_table_with_environment_context", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.add_partitions", bytes); } } -void ThriftHiveMetastoreProcessor::process_drop_table(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +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.drop_table", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.append_partition", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.drop_table"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.append_partition"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.drop_table"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.append_partition"); } - ThriftHiveMetastore_drop_table_args args; + 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.drop_table", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.append_partition", bytes); } - ThriftHiveMetastore_drop_table_result result; + ThriftHiveMetastore_append_partition_result result; try { - iface_->drop_table(args.dbname, args.name, args.deleteData); - } catch (NoSuchObjectException &o1) { + 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.drop_table"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.append_partition"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("drop_table", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("append_partition", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -29216,57 +32704,61 @@ void ThriftHiveMetastoreProcessor::process_drop_table(int32_t seqid, ::apache::t } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.drop_table"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.append_partition"); } - oprot->writeMessageBegin("drop_table", ::apache::thrift::protocol::T_REPLY, seqid); + 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.drop_table", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.append_partition", 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 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.drop_table_with_environment_context", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.add_partitions_req", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.drop_table_with_environment_context"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.add_partitions_req"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.drop_table_with_environment_context"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.add_partitions_req"); } - ThriftHiveMetastore_drop_table_with_environment_context_args args; + 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.drop_table_with_environment_context", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.add_partitions_req", bytes); } - ThriftHiveMetastore_drop_table_with_environment_context_result result; + ThriftHiveMetastore_add_partitions_req_result result; try { - iface_->drop_table_with_environment_context(args.dbname, args.name, args.deleteData, args.environment_context); - } catch (NoSuchObjectException &o1) { + 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.drop_table_with_environment_context"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.add_partitions_req"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("drop_table_with_environment_context", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("add_partitions_req", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -29275,55 +32767,61 @@ void ThriftHiveMetastoreProcessor::process_drop_table_with_environment_context(i } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.drop_table_with_environment_context"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.add_partitions_req"); } - oprot->writeMessageBegin("drop_table_with_environment_context", ::apache::thrift::protocol::T_REPLY, seqid); + 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.drop_table_with_environment_context", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.add_partitions_req", bytes); } } -void ThriftHiveMetastoreProcessor::process_get_tables(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +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.get_tables", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.append_partition_with_environment_context", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_tables"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.append_partition_with_environment_context"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_tables"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.append_partition_with_environment_context"); } - ThriftHiveMetastore_get_tables_args args; + 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.get_tables", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.append_partition_with_environment_context", bytes); } - ThriftHiveMetastore_get_tables_result result; + ThriftHiveMetastore_append_partition_with_environment_context_result result; try { - iface_->get_tables(result.success, args.db_name, args.pattern); + 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 (MetaException &o1) { + } 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.get_tables"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.append_partition_with_environment_context"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("get_tables", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("append_partition_with_environment_context", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -29332,55 +32830,61 @@ void ThriftHiveMetastoreProcessor::process_get_tables(int32_t seqid, ::apache::t } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_tables"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.append_partition_with_environment_context"); } - oprot->writeMessageBegin("get_tables", ::apache::thrift::protocol::T_REPLY, seqid); + 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.get_tables", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.append_partition_with_environment_context", bytes); } } -void ThriftHiveMetastoreProcessor::process_get_all_tables(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +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.get_all_tables", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.append_partition_by_name", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_all_tables"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.append_partition_by_name"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_all_tables"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.append_partition_by_name"); } - ThriftHiveMetastore_get_all_tables_args args; + 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.get_all_tables", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.append_partition_by_name", bytes); } - ThriftHiveMetastore_get_all_tables_result result; + ThriftHiveMetastore_append_partition_by_name_result result; try { - iface_->get_all_tables(result.success, args.db_name); + iface_->append_partition_by_name(result.success, args.db_name, args.tbl_name, args.part_name); result.__isset.success = true; - } catch (MetaException &o1) { + } 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.get_all_tables"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.append_partition_by_name"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("get_all_tables", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("append_partition_by_name", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -29389,58 +32893,61 @@ void ThriftHiveMetastoreProcessor::process_get_all_tables(int32_t seqid, ::apach } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_all_tables"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.append_partition_by_name"); } - oprot->writeMessageBegin("get_all_tables", ::apache::thrift::protocol::T_REPLY, seqid); + 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.get_all_tables", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.append_partition_by_name", bytes); } } -void ThriftHiveMetastoreProcessor::process_get_table(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +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.get_table", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.append_partition_by_name_with_environment_context", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_table"); + ::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.get_table"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.append_partition_by_name_with_environment_context"); } - ThriftHiveMetastore_get_table_args args; + 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.get_table", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.append_partition_by_name_with_environment_context", bytes); } - ThriftHiveMetastore_get_table_result result; + ThriftHiveMetastore_append_partition_by_name_with_environment_context_result result; try { - iface_->get_table(result.success, args.dbname, args.tbl_name); + 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 (MetaException &o1) { + } catch (InvalidObjectException &o1) { result.o1 = o1; result.__isset.o1 = true; - } catch (NoSuchObjectException &o2) { + } 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.get_table"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.append_partition_by_name_with_environment_context"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("get_table", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("append_partition_by_name_with_environment_context", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -29449,61 +32956,58 @@ void ThriftHiveMetastoreProcessor::process_get_table(int32_t seqid, ::apache::th } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_table"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.append_partition_by_name_with_environment_context"); } - oprot->writeMessageBegin("get_table", ::apache::thrift::protocol::T_REPLY, seqid); + 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.get_table", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.append_partition_by_name_with_environment_context", 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 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.get_table_objects_by_name", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.drop_partition", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_table_objects_by_name"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.drop_partition"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_table_objects_by_name"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.drop_partition"); } - ThriftHiveMetastore_get_table_objects_by_name_args args; + 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.get_table_objects_by_name", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.drop_partition", bytes); } - ThriftHiveMetastore_get_table_objects_by_name_result result; + ThriftHiveMetastore_drop_partition_result result; try { - iface_->get_table_objects_by_name(result.success, args.dbname, args.tbl_names); + result.success = iface_->drop_partition(args.db_name, args.tbl_name, args.part_vals, args.deleteData); result.__isset.success = true; - } catch (MetaException &o1) { + } catch (NoSuchObjectException &o1) { result.o1 = o1; result.__isset.o1 = true; - } catch (InvalidOperationException &o2) { + } catch (MetaException &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"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.drop_partition"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("get_table_objects_by_name", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("drop_partition", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -29512,61 +33016,58 @@ void ThriftHiveMetastoreProcessor::process_get_table_objects_by_name(int32_t seq } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_table_objects_by_name"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.drop_partition"); } - oprot->writeMessageBegin("get_table_objects_by_name", ::apache::thrift::protocol::T_REPLY, seqid); + 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.get_table_objects_by_name", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.drop_partition", 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 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.get_table_names_by_filter", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.drop_partition_with_environment_context", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_table_names_by_filter"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.drop_partition_with_environment_context"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_table_names_by_filter"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.drop_partition_with_environment_context"); } - ThriftHiveMetastore_get_table_names_by_filter_args args; + 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.get_table_names_by_filter", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.drop_partition_with_environment_context", bytes); } - ThriftHiveMetastore_get_table_names_by_filter_result result; + ThriftHiveMetastore_drop_partition_with_environment_context_result result; try { - iface_->get_table_names_by_filter(result.success, args.dbname, args.filter, args.max_tables); + 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 (MetaException &o1) { + } catch (NoSuchObjectException &o1) { result.o1 = o1; result.__isset.o1 = true; - } catch (InvalidOperationException &o2) { + } catch (MetaException &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"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.drop_partition_with_environment_context"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("get_table_names_by_filter", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("drop_partition_with_environment_context", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -29575,45 +33076,46 @@ void ThriftHiveMetastoreProcessor::process_get_table_names_by_filter(int32_t seq } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_table_names_by_filter"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.drop_partition_with_environment_context"); } - oprot->writeMessageBegin("get_table_names_by_filter", ::apache::thrift::protocol::T_REPLY, seqid); + 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.get_table_names_by_filter", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.drop_partition_with_environment_context", bytes); } } -void ThriftHiveMetastoreProcessor::process_alter_table(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +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.alter_table", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.drop_partition_by_name", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.alter_table"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.drop_partition_by_name"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.alter_table"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.drop_partition_by_name"); } - ThriftHiveMetastore_alter_table_args args; + 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.alter_table", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.drop_partition_by_name", bytes); } - ThriftHiveMetastore_alter_table_result result; + ThriftHiveMetastore_drop_partition_by_name_result result; try { - iface_->alter_table(args.dbname, args.tbl_name, args.new_tbl); - } catch (InvalidOperationException &o1) { + 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) { @@ -29621,11 +33123,11 @@ void ThriftHiveMetastoreProcessor::process_alter_table(int32_t seqid, ::apache:: result.__isset.o2 = true; } catch (const std::exception& e) { if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.alter_table"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.drop_partition_by_name"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("alter_table", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("drop_partition_by_name", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -29634,45 +33136,46 @@ void ThriftHiveMetastoreProcessor::process_alter_table(int32_t seqid, ::apache:: } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.alter_table"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.drop_partition_by_name"); } - oprot->writeMessageBegin("alter_table", ::apache::thrift::protocol::T_REPLY, seqid); + 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.alter_table", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.drop_partition_by_name", 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 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.alter_table_with_environment_context", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.drop_partition_by_name_with_environment_context", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.alter_table_with_environment_context"); + ::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.alter_table_with_environment_context"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.drop_partition_by_name_with_environment_context"); } - ThriftHiveMetastore_alter_table_with_environment_context_args args; + 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.alter_table_with_environment_context", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.drop_partition_by_name_with_environment_context", bytes); } - ThriftHiveMetastore_alter_table_with_environment_context_result result; + ThriftHiveMetastore_drop_partition_by_name_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.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) { @@ -29680,11 +33183,11 @@ void ThriftHiveMetastoreProcessor::process_alter_table_with_environment_context( result.__isset.o2 = true; } catch (const std::exception& e) { if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.alter_table_with_environment_context"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.drop_partition_by_name_with_environment_context"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("alter_table_with_environment_context", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("drop_partition_by_name_with_environment_context", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -29693,61 +33196,58 @@ void ThriftHiveMetastoreProcessor::process_alter_table_with_environment_context( } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.alter_table_with_environment_context"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.drop_partition_by_name_with_environment_context"); } - oprot->writeMessageBegin("alter_table_with_environment_context", ::apache::thrift::protocol::T_REPLY, seqid); + 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.alter_table_with_environment_context", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.drop_partition_by_name_with_environment_context", bytes); } } -void ThriftHiveMetastoreProcessor::process_add_partition(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +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.add_partition", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.drop_partitions_req", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.add_partition"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.drop_partitions_req"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.add_partition"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.drop_partitions_req"); } - ThriftHiveMetastore_add_partition_args args; + 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.add_partition", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.drop_partitions_req", bytes); } - ThriftHiveMetastore_add_partition_result result; + ThriftHiveMetastore_drop_partitions_req_result result; try { - iface_->add_partition(result.success, args.new_part); + iface_->drop_partitions_req(result.success, args.req); result.__isset.success = true; - } catch (InvalidObjectException &o1) { + } catch (NoSuchObjectException &o1) { result.o1 = o1; result.__isset.o1 = true; - } catch (AlreadyExistsException &o2) { + } catch (MetaException &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"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.drop_partitions_req"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("add_partition", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("drop_partitions_req", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -29756,61 +33256,58 @@ void ThriftHiveMetastoreProcessor::process_add_partition(int32_t seqid, ::apache } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.add_partition"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.drop_partitions_req"); } - oprot->writeMessageBegin("add_partition", ::apache::thrift::protocol::T_REPLY, seqid); + 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.add_partition", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.drop_partitions_req", 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 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.add_partition_with_environment_context", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_partition", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.add_partition_with_environment_context"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_partition"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.add_partition_with_environment_context"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_partition"); } - ThriftHiveMetastore_add_partition_with_environment_context_args args; + 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.add_partition_with_environment_context", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_partition", bytes); } - ThriftHiveMetastore_add_partition_with_environment_context_result result; + ThriftHiveMetastore_get_partition_result result; try { - iface_->add_partition_with_environment_context(result.success, args.new_part, args.environment_context); + iface_->get_partition(result.success, args.db_name, args.tbl_name, args.part_vals); result.__isset.success = true; - } catch (InvalidObjectException &o1) { + } catch (MetaException &o1) { result.o1 = o1; result.__isset.o1 = true; - } catch (AlreadyExistsException &o2) { + } catch (NoSuchObjectException &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"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_partition"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("add_partition_with_environment_context", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("get_partition", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -29819,61 +33316,64 @@ void ThriftHiveMetastoreProcessor::process_add_partition_with_environment_contex } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.add_partition_with_environment_context"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_partition"); } - oprot->writeMessageBegin("add_partition_with_environment_context", ::apache::thrift::protocol::T_REPLY, seqid); + 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.add_partition_with_environment_context", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_partition", bytes); } } -void ThriftHiveMetastoreProcessor::process_add_partitions(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +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.add_partitions", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.exchange_partition", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.add_partitions"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.exchange_partition"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.add_partitions"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.exchange_partition"); } - ThriftHiveMetastore_add_partitions_args args; + 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.add_partitions", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.exchange_partition", bytes); } - ThriftHiveMetastore_add_partitions_result result; + ThriftHiveMetastore_exchange_partition_result result; try { - result.success = iface_->add_partitions(args.new_parts); + 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 (InvalidObjectException &o1) { + } catch (MetaException &o1) { result.o1 = o1; result.__isset.o1 = true; - } catch (AlreadyExistsException &o2) { + } catch (NoSuchObjectException &o2) { result.o2 = o2; result.__isset.o2 = true; - } catch (MetaException &o3) { + } 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.add_partitions"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.exchange_partition"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("add_partitions", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("exchange_partition", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -29882,61 +33382,58 @@ void ThriftHiveMetastoreProcessor::process_add_partitions(int32_t seqid, ::apach } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.add_partitions"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.exchange_partition"); } - oprot->writeMessageBegin("add_partitions", ::apache::thrift::protocol::T_REPLY, seqid); + 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.add_partitions", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.exchange_partition", bytes); } } -void ThriftHiveMetastoreProcessor::process_append_partition(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +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.append_partition", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_partition_with_auth", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.append_partition"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_partition_with_auth"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.append_partition"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_partition_with_auth"); } - ThriftHiveMetastore_append_partition_args args; + 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.append_partition", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_partition_with_auth", bytes); } - ThriftHiveMetastore_append_partition_result result; + ThriftHiveMetastore_get_partition_with_auth_result result; try { - iface_->append_partition(result.success, args.db_name, args.tbl_name, args.part_vals); + 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 (InvalidObjectException &o1) { + } catch (MetaException &o1) { result.o1 = o1; result.__isset.o1 = true; - } catch (AlreadyExistsException &o2) { + } catch (NoSuchObjectException &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"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_partition_with_auth"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("append_partition", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("get_partition_with_auth", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -29945,61 +33442,58 @@ void ThriftHiveMetastoreProcessor::process_append_partition(int32_t seqid, ::apa } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.append_partition"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_partition_with_auth"); } - oprot->writeMessageBegin("append_partition", ::apache::thrift::protocol::T_REPLY, seqid); + 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.append_partition", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_partition_with_auth", bytes); } } -void ThriftHiveMetastoreProcessor::process_add_partitions_req(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +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.add_partitions_req", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_partition_by_name", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.add_partitions_req"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_partition_by_name"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.add_partitions_req"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_partition_by_name"); } - ThriftHiveMetastore_add_partitions_req_args args; + 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.add_partitions_req", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_partition_by_name", bytes); } - ThriftHiveMetastore_add_partitions_req_result result; + ThriftHiveMetastore_get_partition_by_name_result result; try { - iface_->add_partitions_req(result.success, args.request); + iface_->get_partition_by_name(result.success, args.db_name, args.tbl_name, args.part_name); result.__isset.success = true; - } catch (InvalidObjectException &o1) { + } catch (MetaException &o1) { result.o1 = o1; result.__isset.o1 = true; - } catch (AlreadyExistsException &o2) { + } catch (NoSuchObjectException &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"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_partition_by_name"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("add_partitions_req", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("get_partition_by_name", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -30008,61 +33502,58 @@ void ThriftHiveMetastoreProcessor::process_add_partitions_req(int32_t seqid, ::a } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.add_partitions_req"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_partition_by_name"); } - oprot->writeMessageBegin("add_partitions_req", ::apache::thrift::protocol::T_REPLY, seqid); + 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.add_partitions_req", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_partition_by_name", 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 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.append_partition_with_environment_context", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_partitions", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.append_partition_with_environment_context"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_partitions"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.append_partition_with_environment_context"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_partitions"); } - ThriftHiveMetastore_append_partition_with_environment_context_args args; + 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.append_partition_with_environment_context", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_partitions", bytes); } - ThriftHiveMetastore_append_partition_with_environment_context_result result; + ThriftHiveMetastore_get_partitions_result result; try { - iface_->append_partition_with_environment_context(result.success, args.db_name, args.tbl_name, args.part_vals, args.environment_context); + iface_->get_partitions(result.success, args.db_name, args.tbl_name, args.max_parts); result.__isset.success = true; - } catch (InvalidObjectException &o1) { + } catch (NoSuchObjectException &o1) { result.o1 = o1; result.__isset.o1 = true; - } catch (AlreadyExistsException &o2) { + } catch (MetaException &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"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_partitions"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("append_partition_with_environment_context", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("get_partitions", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -30071,61 +33562,58 @@ void ThriftHiveMetastoreProcessor::process_append_partition_with_environment_con } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.append_partition_with_environment_context"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_partitions"); } - oprot->writeMessageBegin("append_partition_with_environment_context", ::apache::thrift::protocol::T_REPLY, seqid); + 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.append_partition_with_environment_context", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_partitions", bytes); } } -void ThriftHiveMetastoreProcessor::process_append_partition_by_name(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +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.append_partition_by_name", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_partitions_with_auth", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.append_partition_by_name"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_partitions_with_auth"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.append_partition_by_name"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_partitions_with_auth"); } - ThriftHiveMetastore_append_partition_by_name_args args; + 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.append_partition_by_name", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_partitions_with_auth", bytes); } - ThriftHiveMetastore_append_partition_by_name_result result; + ThriftHiveMetastore_get_partitions_with_auth_result result; try { - iface_->append_partition_by_name(result.success, args.db_name, args.tbl_name, args.part_name); + 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 (InvalidObjectException &o1) { + } catch (NoSuchObjectException &o1) { result.o1 = o1; result.__isset.o1 = true; - } catch (AlreadyExistsException &o2) { + } catch (MetaException &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"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_partitions_with_auth"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("append_partition_by_name", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("get_partitions_with_auth", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -30134,61 +33622,55 @@ void ThriftHiveMetastoreProcessor::process_append_partition_by_name(int32_t seqi } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.append_partition_by_name"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_partitions_with_auth"); } - oprot->writeMessageBegin("append_partition_by_name", ::apache::thrift::protocol::T_REPLY, seqid); + 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.append_partition_by_name", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_partitions_with_auth", 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 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.append_partition_by_name_with_environment_context", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_partition_names", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.append_partition_by_name_with_environment_context"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_partition_names"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.append_partition_by_name_with_environment_context"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_partition_names"); } - ThriftHiveMetastore_append_partition_by_name_with_environment_context_args args; + 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.append_partition_by_name_with_environment_context", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_partition_names", bytes); } - ThriftHiveMetastore_append_partition_by_name_with_environment_context_result result; + ThriftHiveMetastore_get_partition_names_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); + iface_->get_partition_names(result.success, args.db_name, args.tbl_name, args.max_parts); result.__isset.success = true; - } catch (InvalidObjectException &o1) { - result.o1 = o1; - result.__isset.o1 = true; - } catch (AlreadyExistsException &o2) { + } catch (MetaException &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"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_partition_names"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("append_partition_by_name_with_environment_context", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("get_partition_names", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -30197,58 +33679,58 @@ void ThriftHiveMetastoreProcessor::process_append_partition_by_name_with_environ } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.append_partition_by_name_with_environment_context"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_partition_names"); } - oprot->writeMessageBegin("append_partition_by_name_with_environment_context", ::apache::thrift::protocol::T_REPLY, seqid); + 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.append_partition_by_name_with_environment_context", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_partition_names", bytes); } } -void ThriftHiveMetastoreProcessor::process_drop_partition(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +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.drop_partition", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_partitions_ps", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.drop_partition"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_partitions_ps"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.drop_partition"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_partitions_ps"); } - ThriftHiveMetastore_drop_partition_args args; + 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.drop_partition", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_partitions_ps", bytes); } - ThriftHiveMetastore_drop_partition_result result; + ThriftHiveMetastore_get_partitions_ps_result result; try { - result.success = iface_->drop_partition(args.db_name, args.tbl_name, args.part_vals, args.deleteData); + iface_->get_partitions_ps(result.success, args.db_name, args.tbl_name, args.part_vals, args.max_parts); result.__isset.success = true; - } catch (NoSuchObjectException &o1) { + } catch (MetaException &o1) { result.o1 = o1; result.__isset.o1 = true; - } catch (MetaException &o2) { + } 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_partition"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_partitions_ps"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("drop_partition", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("get_partitions_ps", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -30257,44 +33739,44 @@ void ThriftHiveMetastoreProcessor::process_drop_partition(int32_t seqid, ::apach } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.drop_partition"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_partitions_ps"); } - oprot->writeMessageBegin("drop_partition", ::apache::thrift::protocol::T_REPLY, seqid); + 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.drop_partition", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_partitions_ps", 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 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.drop_partition_with_environment_context", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_partitions_ps_with_auth", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.drop_partition_with_environment_context"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_partitions_ps_with_auth"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.drop_partition_with_environment_context"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_partitions_ps_with_auth"); } - ThriftHiveMetastore_drop_partition_with_environment_context_args args; + 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.drop_partition_with_environment_context", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_partitions_ps_with_auth", bytes); } - ThriftHiveMetastore_drop_partition_with_environment_context_result result; + ThriftHiveMetastore_get_partitions_ps_with_auth_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); + 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; @@ -30304,11 +33786,11 @@ void ThriftHiveMetastoreProcessor::process_drop_partition_with_environment_conte result.__isset.o2 = true; } catch (const std::exception& e) { if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.drop_partition_with_environment_context"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_partitions_ps_with_auth"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("drop_partition_with_environment_context", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("get_partitions_ps_with_auth", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -30317,58 +33799,58 @@ void ThriftHiveMetastoreProcessor::process_drop_partition_with_environment_conte } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.drop_partition_with_environment_context"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_partitions_ps_with_auth"); } - oprot->writeMessageBegin("drop_partition_with_environment_context", ::apache::thrift::protocol::T_REPLY, seqid); + 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.drop_partition_with_environment_context", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_partitions_ps_with_auth", bytes); } } -void ThriftHiveMetastoreProcessor::process_drop_partition_by_name(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +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.drop_partition_by_name", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_partition_names_ps", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.drop_partition_by_name"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_partition_names_ps"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.drop_partition_by_name"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_partition_names_ps"); } - ThriftHiveMetastore_drop_partition_by_name_args args; + 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.drop_partition_by_name", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_partition_names_ps", bytes); } - ThriftHiveMetastore_drop_partition_by_name_result result; + ThriftHiveMetastore_get_partition_names_ps_result result; try { - result.success = iface_->drop_partition_by_name(args.db_name, args.tbl_name, args.part_name, args.deleteData); + iface_->get_partition_names_ps(result.success, args.db_name, args.tbl_name, args.part_vals, args.max_parts); result.__isset.success = true; - } catch (NoSuchObjectException &o1) { + } catch (MetaException &o1) { result.o1 = o1; result.__isset.o1 = true; - } catch (MetaException &o2) { + } 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_partition_by_name"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_partition_names_ps"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("drop_partition_by_name", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("get_partition_names_ps", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -30377,58 +33859,58 @@ void ThriftHiveMetastoreProcessor::process_drop_partition_by_name(int32_t seqid, } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.drop_partition_by_name"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_partition_names_ps"); } - oprot->writeMessageBegin("drop_partition_by_name", ::apache::thrift::protocol::T_REPLY, seqid); + 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.drop_partition_by_name", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_partition_names_ps", 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 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.drop_partition_by_name_with_environment_context", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_partitions_by_filter", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.drop_partition_by_name_with_environment_context"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_partitions_by_filter"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.drop_partition_by_name_with_environment_context"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_partitions_by_filter"); } - ThriftHiveMetastore_drop_partition_by_name_with_environment_context_args args; + 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.drop_partition_by_name_with_environment_context", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_partitions_by_filter", bytes); } - ThriftHiveMetastore_drop_partition_by_name_with_environment_context_result result; + ThriftHiveMetastore_get_partitions_by_filter_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); + iface_->get_partitions_by_filter(result.success, args.db_name, args.tbl_name, args.filter, args.max_parts); result.__isset.success = true; - } catch (NoSuchObjectException &o1) { + } catch (MetaException &o1) { result.o1 = o1; result.__isset.o1 = true; - } catch (MetaException &o2) { + } 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_partition_by_name_with_environment_context"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_partitions_by_filter"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("drop_partition_by_name_with_environment_context", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("get_partitions_by_filter", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -30437,58 +33919,58 @@ void ThriftHiveMetastoreProcessor::process_drop_partition_by_name_with_environme } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.drop_partition_by_name_with_environment_context"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_partitions_by_filter"); } - oprot->writeMessageBegin("drop_partition_by_name_with_environment_context", ::apache::thrift::protocol::T_REPLY, seqid); + 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.drop_partition_by_name_with_environment_context", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_partitions_by_filter", bytes); } } -void ThriftHiveMetastoreProcessor::process_drop_partitions_req(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +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.drop_partitions_req", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_partitions_by_expr", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.drop_partitions_req"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_partitions_by_expr"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.drop_partitions_req"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_partitions_by_expr"); } - ThriftHiveMetastore_drop_partitions_req_args args; + 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.drop_partitions_req", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_partitions_by_expr", bytes); } - ThriftHiveMetastore_drop_partitions_req_result result; + ThriftHiveMetastore_get_partitions_by_expr_result result; try { - iface_->drop_partitions_req(result.success, args.req); + iface_->get_partitions_by_expr(result.success, args.req); result.__isset.success = true; - } catch (NoSuchObjectException &o1) { + } catch (MetaException &o1) { result.o1 = o1; result.__isset.o1 = true; - } catch (MetaException &o2) { + } 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_partitions_req"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_partitions_by_expr"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("drop_partitions_req", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("get_partitions_by_expr", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -30497,44 +33979,44 @@ void ThriftHiveMetastoreProcessor::process_drop_partitions_req(int32_t seqid, :: } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.drop_partitions_req"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_partitions_by_expr"); } - oprot->writeMessageBegin("drop_partitions_req", ::apache::thrift::protocol::T_REPLY, seqid); + 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.drop_partitions_req", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_partitions_by_expr", bytes); } } -void ThriftHiveMetastoreProcessor::process_get_partition(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +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_partition", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_partitions_by_names", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_partition"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_partitions_by_names"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_partition"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_partitions_by_names"); } - ThriftHiveMetastore_get_partition_args args; + 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_partition", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_partitions_by_names", bytes); } - ThriftHiveMetastore_get_partition_result result; + ThriftHiveMetastore_get_partitions_by_names_result result; try { - iface_->get_partition(result.success, args.db_name, args.tbl_name, args.part_vals); + 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; @@ -30544,11 +34026,11 @@ void ThriftHiveMetastoreProcessor::process_get_partition(int32_t seqid, ::apache result.__isset.o2 = true; } catch (const std::exception& e) { if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_partition"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_partitions_by_names"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("get_partition", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("get_partitions_by_names", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -30557,64 +34039,57 @@ void ThriftHiveMetastoreProcessor::process_get_partition(int32_t seqid, ::apache } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_partition"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_partitions_by_names"); } - oprot->writeMessageBegin("get_partition", ::apache::thrift::protocol::T_REPLY, seqid); + 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_partition", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_partitions_by_names", bytes); } } -void ThriftHiveMetastoreProcessor::process_exchange_partition(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +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.exchange_partition", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.alter_partition", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.exchange_partition"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.alter_partition"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.exchange_partition"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.alter_partition"); } - ThriftHiveMetastore_exchange_partition_args args; + 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.exchange_partition", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.alter_partition", bytes); } - ThriftHiveMetastore_exchange_partition_result result; + ThriftHiveMetastore_alter_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) { + iface_->alter_partition(args.db_name, args.tbl_name, args.new_part); + } catch (InvalidOperationException &o1) { result.o1 = o1; result.__isset.o1 = true; - } catch (NoSuchObjectException &o2) { + } 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.exchange_partition"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.alter_partition"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("exchange_partition", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("alter_partition", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -30623,58 +34098,57 @@ void ThriftHiveMetastoreProcessor::process_exchange_partition(int32_t seqid, ::a } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.exchange_partition"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.alter_partition"); } - oprot->writeMessageBegin("exchange_partition", ::apache::thrift::protocol::T_REPLY, seqid); + 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.exchange_partition", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.alter_partition", bytes); } } -void ThriftHiveMetastoreProcessor::process_get_partition_with_auth(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +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.get_partition_with_auth", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.alter_partitions", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_partition_with_auth"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.alter_partitions"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_partition_with_auth"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.alter_partitions"); } - ThriftHiveMetastore_get_partition_with_auth_args args; + 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.get_partition_with_auth", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.alter_partitions", bytes); } - ThriftHiveMetastore_get_partition_with_auth_result result; + ThriftHiveMetastore_alter_partitions_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) { + iface_->alter_partitions(args.db_name, args.tbl_name, args.new_parts); + } catch (InvalidOperationException &o1) { result.o1 = o1; result.__isset.o1 = true; - } catch (NoSuchObjectException &o2) { + } 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_with_auth"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.alter_partitions"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("get_partition_with_auth", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("alter_partitions", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -30683,58 +34157,57 @@ void ThriftHiveMetastoreProcessor::process_get_partition_with_auth(int32_t seqid } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_partition_with_auth"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.alter_partitions"); } - oprot->writeMessageBegin("get_partition_with_auth", ::apache::thrift::protocol::T_REPLY, seqid); + 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.get_partition_with_auth", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.alter_partitions", bytes); } } -void ThriftHiveMetastoreProcessor::process_get_partition_by_name(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +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.get_partition_by_name", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.alter_partition_with_environment_context", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_partition_by_name"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.alter_partition_with_environment_context"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_partition_by_name"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.alter_partition_with_environment_context"); } - ThriftHiveMetastore_get_partition_by_name_args args; + 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.get_partition_by_name", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.alter_partition_with_environment_context", bytes); } - ThriftHiveMetastore_get_partition_by_name_result result; + ThriftHiveMetastore_alter_partition_with_environment_context_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) { + 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 (NoSuchObjectException &o2) { + } 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_by_name"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.alter_partition_with_environment_context"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("get_partition_by_name", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("alter_partition_with_environment_context", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -30743,46 +34216,45 @@ void ThriftHiveMetastoreProcessor::process_get_partition_by_name(int32_t seqid, } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_partition_by_name"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.alter_partition_with_environment_context"); } - oprot->writeMessageBegin("get_partition_by_name", ::apache::thrift::protocol::T_REPLY, seqid); + 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.get_partition_by_name", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.alter_partition_with_environment_context", bytes); } } -void ThriftHiveMetastoreProcessor::process_get_partitions(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +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.get_partitions", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.rename_partition", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_partitions"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.rename_partition"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_partitions"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.rename_partition"); } - ThriftHiveMetastore_get_partitions_args args; + 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.get_partitions", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.rename_partition", bytes); } - ThriftHiveMetastore_get_partitions_result result; + ThriftHiveMetastore_rename_partition_result result; try { - iface_->get_partitions(result.success, args.db_name, args.tbl_name, args.max_parts); - result.__isset.success = true; - } catch (NoSuchObjectException &o1) { + 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) { @@ -30790,11 +34262,11 @@ void ThriftHiveMetastoreProcessor::process_get_partitions(int32_t seqid, ::apach result.__isset.o2 = true; } catch (const std::exception& e) { if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_partitions"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.rename_partition"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("get_partitions", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("rename_partition", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -30803,58 +34275,55 @@ void ThriftHiveMetastoreProcessor::process_get_partitions(int32_t seqid, ::apach } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_partitions"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.rename_partition"); } - oprot->writeMessageBegin("get_partitions", ::apache::thrift::protocol::T_REPLY, seqid); + 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.get_partitions", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.rename_partition", bytes); } } -void ThriftHiveMetastoreProcessor::process_get_partitions_with_auth(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +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.get_partitions_with_auth", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.partition_name_has_valid_characters", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_partitions_with_auth"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.partition_name_has_valid_characters"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_partitions_with_auth"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.partition_name_has_valid_characters"); } - ThriftHiveMetastore_get_partitions_with_auth_args args; + 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.get_partitions_with_auth", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.partition_name_has_valid_characters", bytes); } - ThriftHiveMetastore_get_partitions_with_auth_result result; + ThriftHiveMetastore_partition_name_has_valid_characters_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.success = iface_->partition_name_has_valid_characters(args.part_vals, args.throw_exception); result.__isset.success = true; - } catch (NoSuchObjectException &o1) { + } catch (MetaException &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"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.partition_name_has_valid_characters"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("get_partitions_with_auth", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("partition_name_has_valid_characters", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -30863,55 +34332,55 @@ void ThriftHiveMetastoreProcessor::process_get_partitions_with_auth(int32_t seqi } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_partitions_with_auth"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.partition_name_has_valid_characters"); } - oprot->writeMessageBegin("get_partitions_with_auth", ::apache::thrift::protocol::T_REPLY, seqid); + 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.get_partitions_with_auth", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.partition_name_has_valid_characters", bytes); } } -void ThriftHiveMetastoreProcessor::process_get_partition_names(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +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_partition_names", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_config_value", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_partition_names"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_config_value"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_partition_names"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_config_value"); } - ThriftHiveMetastore_get_partition_names_args args; + 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_partition_names", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_config_value", bytes); } - ThriftHiveMetastore_get_partition_names_result result; + ThriftHiveMetastore_get_config_value_result result; try { - iface_->get_partition_names(result.success, args.db_name, args.tbl_name, args.max_parts); + iface_->get_config_value(result.success, args.name, args.defaultValue); result.__isset.success = true; - } catch (MetaException &o2) { - result.o2 = o2; - result.__isset.o2 = 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_partition_names"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_config_value"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("get_partition_names", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("get_config_value", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -30920,58 +34389,55 @@ void ThriftHiveMetastoreProcessor::process_get_partition_names(int32_t seqid, :: } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_partition_names"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_config_value"); } - oprot->writeMessageBegin("get_partition_names", ::apache::thrift::protocol::T_REPLY, seqid); + 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_partition_names", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_config_value", bytes); } } -void ThriftHiveMetastoreProcessor::process_get_partitions_ps(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +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.get_partitions_ps", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.partition_name_to_vals", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_partitions_ps"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.partition_name_to_vals"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_partitions_ps"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.partition_name_to_vals"); } - ThriftHiveMetastore_get_partitions_ps_args args; + 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.get_partitions_ps", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.partition_name_to_vals", bytes); } - ThriftHiveMetastore_get_partitions_ps_result result; + ThriftHiveMetastore_partition_name_to_vals_result result; try { - iface_->get_partitions_ps(result.success, args.db_name, args.tbl_name, args.part_vals, args.max_parts); + 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 (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"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.partition_name_to_vals"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("get_partitions_ps", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("partition_name_to_vals", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -30980,58 +34446,55 @@ void ThriftHiveMetastoreProcessor::process_get_partitions_ps(int32_t seqid, ::ap } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_partitions_ps"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.partition_name_to_vals"); } - oprot->writeMessageBegin("get_partitions_ps", ::apache::thrift::protocol::T_REPLY, seqid); + 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.get_partitions_ps", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.partition_name_to_vals", 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 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.get_partitions_ps_with_auth", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.partition_name_to_spec", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_partitions_ps_with_auth"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.partition_name_to_spec"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_partitions_ps_with_auth"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.partition_name_to_spec"); } - ThriftHiveMetastore_get_partitions_ps_with_auth_args args; + 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.get_partitions_ps_with_auth", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.partition_name_to_spec", bytes); } - ThriftHiveMetastore_get_partitions_ps_with_auth_result result; + ThriftHiveMetastore_partition_name_to_spec_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); + iface_->partition_name_to_spec(result.success, args.part_name); result.__isset.success = true; - } catch (NoSuchObjectException &o1) { + } catch (MetaException &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"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.partition_name_to_spec"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("get_partitions_ps_with_auth", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("partition_name_to_spec", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -31040,58 +34503,69 @@ void ThriftHiveMetastoreProcessor::process_get_partitions_ps_with_auth(int32_t s } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_partitions_ps_with_auth"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.partition_name_to_spec"); } - oprot->writeMessageBegin("get_partitions_ps_with_auth", ::apache::thrift::protocol::T_REPLY, seqid); + 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.get_partitions_ps_with_auth", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.partition_name_to_spec", bytes); } } -void ThriftHiveMetastoreProcessor::process_get_partition_names_ps(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +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.get_partition_names_ps", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.markPartitionForEvent", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_partition_names_ps"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.markPartitionForEvent"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_partition_names_ps"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.markPartitionForEvent"); } - ThriftHiveMetastore_get_partition_names_ps_args args; + 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.get_partition_names_ps", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.markPartitionForEvent", bytes); } - ThriftHiveMetastore_get_partition_names_ps_result result; + ThriftHiveMetastore_markPartitionForEvent_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; + 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.get_partition_names_ps"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.markPartitionForEvent"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("get_partition_names_ps", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("markPartitionForEvent", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -31100,44 +34574,44 @@ void ThriftHiveMetastoreProcessor::process_get_partition_names_ps(int32_t seqid, } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_partition_names_ps"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.markPartitionForEvent"); } - oprot->writeMessageBegin("get_partition_names_ps", ::apache::thrift::protocol::T_REPLY, seqid); + 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.get_partition_names_ps", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.markPartitionForEvent", bytes); } } -void ThriftHiveMetastoreProcessor::process_get_partitions_by_filter(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +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.get_partitions_by_filter", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.isPartitionMarkedForEvent", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_partitions_by_filter"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.isPartitionMarkedForEvent"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_partitions_by_filter"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.isPartitionMarkedForEvent"); } - ThriftHiveMetastore_get_partitions_by_filter_args args; + 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.get_partitions_by_filter", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.isPartitionMarkedForEvent", bytes); } - ThriftHiveMetastore_get_partitions_by_filter_result result; + ThriftHiveMetastore_isPartitionMarkedForEvent_result result; try { - iface_->get_partitions_by_filter(result.success, args.db_name, args.tbl_name, args.filter, args.max_parts); + 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; @@ -31145,13 +34619,25 @@ void ThriftHiveMetastoreProcessor::process_get_partitions_by_filter(int32_t seqi } 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.get_partitions_by_filter"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.isPartitionMarkedForEvent"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("get_partitions_by_filter", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("isPartitionMarkedForEvent", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -31160,58 +34646,61 @@ void ThriftHiveMetastoreProcessor::process_get_partitions_by_filter(int32_t seqi } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_partitions_by_filter"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.isPartitionMarkedForEvent"); } - oprot->writeMessageBegin("get_partitions_by_filter", ::apache::thrift::protocol::T_REPLY, seqid); + 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.get_partitions_by_filter", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.isPartitionMarkedForEvent", bytes); } } -void ThriftHiveMetastoreProcessor::process_get_partitions_by_expr(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +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.get_partitions_by_expr", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.add_index", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_partitions_by_expr"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.add_index"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_partitions_by_expr"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.add_index"); } - ThriftHiveMetastore_get_partitions_by_expr_args args; + 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.get_partitions_by_expr", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.add_index", bytes); } - ThriftHiveMetastore_get_partitions_by_expr_result result; + ThriftHiveMetastore_add_index_result result; try { - iface_->get_partitions_by_expr(result.success, args.req); + iface_->add_index(result.success, args.new_index, args.index_table); result.__isset.success = true; - } catch (MetaException &o1) { + } catch (InvalidObjectException &o1) { result.o1 = o1; result.__isset.o1 = true; - } catch (NoSuchObjectException &o2) { + } 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.get_partitions_by_expr"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.add_index"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("get_partitions_by_expr", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("add_index", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -31220,58 +34709,57 @@ void ThriftHiveMetastoreProcessor::process_get_partitions_by_expr(int32_t seqid, } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_partitions_by_expr"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.add_index"); } - oprot->writeMessageBegin("get_partitions_by_expr", ::apache::thrift::protocol::T_REPLY, seqid); + 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.get_partitions_by_expr", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.add_index", bytes); } } -void ThriftHiveMetastoreProcessor::process_get_partitions_by_names(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +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.get_partitions_by_names", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.alter_index", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_partitions_by_names"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.alter_index"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_partitions_by_names"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.alter_index"); } - ThriftHiveMetastore_get_partitions_by_names_args args; + 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.get_partitions_by_names", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.alter_index", bytes); } - ThriftHiveMetastore_get_partitions_by_names_result result; + ThriftHiveMetastore_alter_index_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) { + 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 (NoSuchObjectException &o2) { + } 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_by_names"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.alter_index"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("get_partitions_by_names", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("alter_index", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -31280,45 +34768,46 @@ void ThriftHiveMetastoreProcessor::process_get_partitions_by_names(int32_t seqid } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_partitions_by_names"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.alter_index"); } - oprot->writeMessageBegin("get_partitions_by_names", ::apache::thrift::protocol::T_REPLY, seqid); + 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.get_partitions_by_names", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.alter_index", bytes); } } -void ThriftHiveMetastoreProcessor::process_alter_partition(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +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.alter_partition", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.drop_index_by_name", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.alter_partition"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.drop_index_by_name"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.alter_partition"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.drop_index_by_name"); } - ThriftHiveMetastore_alter_partition_args args; + 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.alter_partition", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.drop_index_by_name", bytes); } - ThriftHiveMetastore_alter_partition_result result; + ThriftHiveMetastore_drop_index_by_name_result result; try { - iface_->alter_partition(args.db_name, args.tbl_name, args.new_part); - } catch (InvalidOperationException &o1) { + 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) { @@ -31326,11 +34815,11 @@ void ThriftHiveMetastoreProcessor::process_alter_partition(int32_t seqid, ::apac result.__isset.o2 = true; } catch (const std::exception& e) { if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.alter_partition"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.drop_index_by_name"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("alter_partition", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("drop_index_by_name", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -31339,57 +34828,58 @@ void ThriftHiveMetastoreProcessor::process_alter_partition(int32_t seqid, ::apac } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.alter_partition"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.drop_index_by_name"); } - oprot->writeMessageBegin("alter_partition", ::apache::thrift::protocol::T_REPLY, seqid); + 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.alter_partition", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.drop_index_by_name", bytes); } } -void ThriftHiveMetastoreProcessor::process_alter_partitions(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +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.alter_partitions", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_index_by_name", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.alter_partitions"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_index_by_name"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.alter_partitions"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_index_by_name"); } - ThriftHiveMetastore_alter_partitions_args args; + 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.alter_partitions", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_index_by_name", bytes); } - ThriftHiveMetastore_alter_partitions_result result; + ThriftHiveMetastore_get_index_by_name_result result; try { - iface_->alter_partitions(args.db_name, args.tbl_name, args.new_parts); - } catch (InvalidOperationException &o1) { + 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 (MetaException &o2) { + } 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_partitions"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_index_by_name"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("alter_partitions", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("get_index_by_name", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -31398,45 +34888,46 @@ void ThriftHiveMetastoreProcessor::process_alter_partitions(int32_t seqid, ::apa } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.alter_partitions"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_index_by_name"); } - oprot->writeMessageBegin("alter_partitions", ::apache::thrift::protocol::T_REPLY, seqid); + 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.alter_partitions", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_index_by_name", 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 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.alter_partition_with_environment_context", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_indexes", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.alter_partition_with_environment_context"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_indexes"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.alter_partition_with_environment_context"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_indexes"); } - ThriftHiveMetastore_alter_partition_with_environment_context_args args; + 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.alter_partition_with_environment_context", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_indexes", bytes); } - ThriftHiveMetastore_alter_partition_with_environment_context_result result; + ThriftHiveMetastore_get_indexes_result result; try { - iface_->alter_partition_with_environment_context(args.db_name, args.tbl_name, args.new_part, args.environment_context); - } catch (InvalidOperationException &o1) { + 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) { @@ -31444,11 +34935,11 @@ void ThriftHiveMetastoreProcessor::process_alter_partition_with_environment_cont result.__isset.o2 = true; } catch (const std::exception& e) { if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.alter_partition_with_environment_context"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_indexes"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("alter_partition_with_environment_context", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("get_indexes", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -31457,57 +34948,55 @@ void ThriftHiveMetastoreProcessor::process_alter_partition_with_environment_cont } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.alter_partition_with_environment_context"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_indexes"); } - oprot->writeMessageBegin("alter_partition_with_environment_context", ::apache::thrift::protocol::T_REPLY, seqid); + 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.alter_partition_with_environment_context", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_indexes", bytes); } } -void ThriftHiveMetastoreProcessor::process_rename_partition(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +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.rename_partition", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_index_names", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.rename_partition"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_index_names"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.rename_partition"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_index_names"); } - ThriftHiveMetastore_rename_partition_args args; + 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.rename_partition", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_index_names", bytes); } - ThriftHiveMetastore_rename_partition_result result; + ThriftHiveMetastore_get_index_names_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; + 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.rename_partition"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_index_names"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("rename_partition", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("get_index_names", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -31516,55 +35005,64 @@ void ThriftHiveMetastoreProcessor::process_rename_partition(int32_t seqid, ::apa } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.rename_partition"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_index_names"); } - oprot->writeMessageBegin("rename_partition", ::apache::thrift::protocol::T_REPLY, seqid); + 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.rename_partition", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_index_names", 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 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.partition_name_has_valid_characters", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.update_table_column_statistics", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.partition_name_has_valid_characters"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.update_table_column_statistics"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.partition_name_has_valid_characters"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.update_table_column_statistics"); } - ThriftHiveMetastore_partition_name_has_valid_characters_args args; + 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.partition_name_has_valid_characters", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.update_table_column_statistics", bytes); } - ThriftHiveMetastore_partition_name_has_valid_characters_result result; + ThriftHiveMetastore_update_table_column_statistics_result result; try { - result.success = iface_->partition_name_has_valid_characters(args.part_vals, args.throw_exception); + result.success = iface_->update_table_column_statistics(args.stats_obj); result.__isset.success = true; - } catch (MetaException &o1) { + } 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.partition_name_has_valid_characters"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.update_table_column_statistics"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("partition_name_has_valid_characters", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("update_table_column_statistics", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -31573,55 +35071,64 @@ void ThriftHiveMetastoreProcessor::process_partition_name_has_valid_characters(i } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.partition_name_has_valid_characters"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.update_table_column_statistics"); } - oprot->writeMessageBegin("partition_name_has_valid_characters", ::apache::thrift::protocol::T_REPLY, seqid); + 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.partition_name_has_valid_characters", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.update_table_column_statistics", bytes); } } -void ThriftHiveMetastoreProcessor::process_get_config_value(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +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.get_config_value", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.update_partition_column_statistics", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_config_value"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.update_partition_column_statistics"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_config_value"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.update_partition_column_statistics"); } - ThriftHiveMetastore_get_config_value_args args; + 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.get_config_value", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.update_partition_column_statistics", bytes); } - ThriftHiveMetastore_get_config_value_result result; + ThriftHiveMetastore_update_partition_column_statistics_result result; try { - iface_->get_config_value(result.success, args.name, args.defaultValue); + result.success = iface_->update_partition_column_statistics(args.stats_obj); result.__isset.success = true; - } catch (ConfigValSecurityException &o1) { + } 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.get_config_value"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.update_partition_column_statistics"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("get_config_value", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("update_partition_column_statistics", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -31630,55 +35137,64 @@ void ThriftHiveMetastoreProcessor::process_get_config_value(int32_t seqid, ::apa } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_config_value"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.update_partition_column_statistics"); } - oprot->writeMessageBegin("get_config_value", ::apache::thrift::protocol::T_REPLY, seqid); + 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.get_config_value", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.update_partition_column_statistics", bytes); } } -void ThriftHiveMetastoreProcessor::process_partition_name_to_vals(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +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.partition_name_to_vals", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_table_column_statistics", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.partition_name_to_vals"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_table_column_statistics"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.partition_name_to_vals"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_table_column_statistics"); } - ThriftHiveMetastore_partition_name_to_vals_args args; + 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.partition_name_to_vals", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_table_column_statistics", bytes); } - ThriftHiveMetastore_partition_name_to_vals_result result; + ThriftHiveMetastore_get_table_column_statistics_result result; try { - iface_->partition_name_to_vals(result.success, args.part_name); + iface_->get_table_column_statistics(result.success, args.db_name, args.tbl_name, args.col_name); result.__isset.success = true; - } catch (MetaException &o1) { + } 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.partition_name_to_vals"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_table_column_statistics"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("partition_name_to_vals", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("get_table_column_statistics", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -31687,55 +35203,64 @@ void ThriftHiveMetastoreProcessor::process_partition_name_to_vals(int32_t seqid, } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.partition_name_to_vals"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_table_column_statistics"); } - oprot->writeMessageBegin("partition_name_to_vals", ::apache::thrift::protocol::T_REPLY, seqid); + 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.partition_name_to_vals", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_table_column_statistics", bytes); } } -void ThriftHiveMetastoreProcessor::process_partition_name_to_spec(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +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.partition_name_to_spec", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_partition_column_statistics", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.partition_name_to_spec"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_partition_column_statistics"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.partition_name_to_spec"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_partition_column_statistics"); } - ThriftHiveMetastore_partition_name_to_spec_args args; + 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.partition_name_to_spec", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_partition_column_statistics", bytes); } - ThriftHiveMetastore_partition_name_to_spec_result result; + ThriftHiveMetastore_get_partition_column_statistics_result result; try { - iface_->partition_name_to_spec(result.success, args.part_name); + iface_->get_partition_column_statistics(result.success, args.db_name, args.tbl_name, args.part_name, args.col_name); result.__isset.success = true; - } catch (MetaException &o1) { + } 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.partition_name_to_spec"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_partition_column_statistics"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("partition_name_to_spec", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("get_partition_column_statistics", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -31744,69 +35269,58 @@ void ThriftHiveMetastoreProcessor::process_partition_name_to_spec(int32_t seqid, } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.partition_name_to_spec"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_partition_column_statistics"); } - oprot->writeMessageBegin("partition_name_to_spec", ::apache::thrift::protocol::T_REPLY, seqid); + 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.partition_name_to_spec", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_partition_column_statistics", bytes); } } -void ThriftHiveMetastoreProcessor::process_markPartitionForEvent(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +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.markPartitionForEvent", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_table_statistics_req", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.markPartitionForEvent"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_table_statistics_req"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.markPartitionForEvent"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_table_statistics_req"); } - ThriftHiveMetastore_markPartitionForEvent_args args; + 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.markPartitionForEvent", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_table_statistics_req", bytes); } - ThriftHiveMetastore_markPartitionForEvent_result result; + ThriftHiveMetastore_get_table_statistics_req_result result; try { - iface_->markPartitionForEvent(args.db_name, args.tbl_name, args.part_vals, args.eventType); - } catch (MetaException &o1) { + iface_->get_table_statistics_req(result.success, args.request); + result.__isset.success = true; + } catch (NoSuchObjectException &o1) { result.o1 = o1; result.__isset.o1 = true; - } catch (NoSuchObjectException &o2) { + } catch (MetaException &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"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_table_statistics_req"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("markPartitionForEvent", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("get_table_statistics_req", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -31815,70 +35329,58 @@ void ThriftHiveMetastoreProcessor::process_markPartitionForEvent(int32_t seqid, } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.markPartitionForEvent"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_table_statistics_req"); } - oprot->writeMessageBegin("markPartitionForEvent", ::apache::thrift::protocol::T_REPLY, seqid); + 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.markPartitionForEvent", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_table_statistics_req", bytes); } } -void ThriftHiveMetastoreProcessor::process_isPartitionMarkedForEvent(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +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.isPartitionMarkedForEvent", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_partitions_statistics_req", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.isPartitionMarkedForEvent"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_partitions_statistics_req"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.isPartitionMarkedForEvent"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_partitions_statistics_req"); } - ThriftHiveMetastore_isPartitionMarkedForEvent_args args; + 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.isPartitionMarkedForEvent", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_partitions_statistics_req", bytes); } - ThriftHiveMetastore_isPartitionMarkedForEvent_result result; + ThriftHiveMetastore_get_partitions_statistics_req_result result; try { - result.success = iface_->isPartitionMarkedForEvent(args.db_name, args.tbl_name, args.part_vals, args.eventType); + iface_->get_partitions_statistics_req(result.success, args.request); result.__isset.success = true; - } catch (MetaException &o1) { + } catch (NoSuchObjectException &o1) { result.o1 = o1; result.__isset.o1 = true; - } catch (NoSuchObjectException &o2) { + } catch (MetaException &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"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_partitions_statistics_req"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("isPartitionMarkedForEvent", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("get_partitions_statistics_req", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -31887,61 +35389,64 @@ void ThriftHiveMetastoreProcessor::process_isPartitionMarkedForEvent(int32_t seq } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.isPartitionMarkedForEvent"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_partitions_statistics_req"); } - oprot->writeMessageBegin("isPartitionMarkedForEvent", ::apache::thrift::protocol::T_REPLY, seqid); + 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.isPartitionMarkedForEvent", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_partitions_statistics_req", bytes); } } -void ThriftHiveMetastoreProcessor::process_add_index(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +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.add_index", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.delete_partition_column_statistics", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.add_index"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.delete_partition_column_statistics"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.add_index"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.delete_partition_column_statistics"); } - ThriftHiveMetastore_add_index_args args; + 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.add_index", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.delete_partition_column_statistics", bytes); } - ThriftHiveMetastore_add_index_result result; + ThriftHiveMetastore_delete_partition_column_statistics_result result; try { - iface_->add_index(result.success, args.new_index, args.index_table); + result.success = iface_->delete_partition_column_statistics(args.db_name, args.tbl_name, args.part_name, args.col_name); result.__isset.success = true; - } catch (InvalidObjectException &o1) { + } catch (NoSuchObjectException &o1) { result.o1 = o1; result.__isset.o1 = true; - } catch (AlreadyExistsException &o2) { + } catch (MetaException &o2) { result.o2 = o2; result.__isset.o2 = true; - } catch (MetaException &o3) { + } 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.add_index"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.delete_partition_column_statistics"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("add_index", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("delete_partition_column_statistics", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -31950,57 +35455,64 @@ void ThriftHiveMetastoreProcessor::process_add_index(int32_t seqid, ::apache::th } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.add_index"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.delete_partition_column_statistics"); } - oprot->writeMessageBegin("add_index", ::apache::thrift::protocol::T_REPLY, seqid); + 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.add_index", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.delete_partition_column_statistics", bytes); } } -void ThriftHiveMetastoreProcessor::process_alter_index(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +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.alter_index", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.delete_table_column_statistics", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.alter_index"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.delete_table_column_statistics"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.alter_index"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.delete_table_column_statistics"); } - ThriftHiveMetastore_alter_index_args args; + 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.alter_index", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.delete_table_column_statistics", bytes); } - ThriftHiveMetastore_alter_index_result result; + ThriftHiveMetastore_delete_table_column_statistics_result result; try { - iface_->alter_index(args.dbname, args.base_tbl_name, args.idx_name, args.new_idx); - } catch (InvalidOperationException &o1) { + 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.alter_index"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.delete_table_column_statistics"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("alter_index", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("delete_table_column_statistics", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -32009,58 +35521,63 @@ void ThriftHiveMetastoreProcessor::process_alter_index(int32_t seqid, ::apache:: } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.alter_index"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.delete_table_column_statistics"); } - oprot->writeMessageBegin("alter_index", ::apache::thrift::protocol::T_REPLY, seqid); + 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.alter_index", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.delete_table_column_statistics", bytes); } } -void ThriftHiveMetastoreProcessor::process_drop_index_by_name(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +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.drop_index_by_name", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.create_function", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.drop_index_by_name"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.create_function"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.drop_index_by_name"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.create_function"); } - ThriftHiveMetastore_drop_index_by_name_args args; + 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.drop_index_by_name", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.create_function", bytes); } - ThriftHiveMetastore_drop_index_by_name_result result; + ThriftHiveMetastore_create_function_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) { + iface_->create_function(args.func); + } catch (AlreadyExistsException &o1) { result.o1 = o1; result.__isset.o1 = true; - } catch (MetaException &o2) { + } 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.drop_index_by_name"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.create_function"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("drop_index_by_name", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("create_function", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -32069,58 +35586,57 @@ void ThriftHiveMetastoreProcessor::process_drop_index_by_name(int32_t seqid, ::a } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.drop_index_by_name"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.create_function"); } - oprot->writeMessageBegin("drop_index_by_name", ::apache::thrift::protocol::T_REPLY, seqid); + 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.drop_index_by_name", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.create_function", bytes); } } -void ThriftHiveMetastoreProcessor::process_get_index_by_name(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +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.get_index_by_name", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.drop_function", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_index_by_name"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.drop_function"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_index_by_name"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.drop_function"); } - ThriftHiveMetastore_get_index_by_name_args args; + 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.get_index_by_name", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.drop_function", bytes); } - ThriftHiveMetastore_get_index_by_name_result result; + ThriftHiveMetastore_drop_function_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) { + iface_->drop_function(args.dbName, args.funcName); + } catch (NoSuchObjectException &o1) { result.o1 = o1; result.__isset.o1 = true; - } catch (NoSuchObjectException &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.get_index_by_name"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.drop_function"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("get_index_by_name", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("drop_function", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -32129,46 +35645,45 @@ void ThriftHiveMetastoreProcessor::process_get_index_by_name(int32_t seqid, ::ap } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_index_by_name"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.drop_function"); } - oprot->writeMessageBegin("get_index_by_name", ::apache::thrift::protocol::T_REPLY, seqid); + 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.get_index_by_name", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.drop_function", bytes); } } -void ThriftHiveMetastoreProcessor::process_get_indexes(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +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.get_indexes", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.alter_function", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_indexes"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.alter_function"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_indexes"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.alter_function"); } - ThriftHiveMetastore_get_indexes_args args; + 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.get_indexes", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.alter_function", bytes); } - ThriftHiveMetastore_get_indexes_result result; + ThriftHiveMetastore_alter_function_result result; try { - iface_->get_indexes(result.success, args.db_name, args.tbl_name, args.max_indexes); - result.__isset.success = true; - } catch (NoSuchObjectException &o1) { + iface_->alter_function(args.dbName, args.funcName, args.newFunc); + } catch (InvalidOperationException &o1) { result.o1 = o1; result.__isset.o1 = true; } catch (MetaException &o2) { @@ -32176,11 +35691,11 @@ void ThriftHiveMetastoreProcessor::process_get_indexes(int32_t seqid, ::apache:: result.__isset.o2 = true; } catch (const std::exception& e) { if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_indexes"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.alter_function"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("get_indexes", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("alter_function", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -32189,55 +35704,55 @@ void ThriftHiveMetastoreProcessor::process_get_indexes(int32_t seqid, ::apache:: } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_indexes"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.alter_function"); } - oprot->writeMessageBegin("get_indexes", ::apache::thrift::protocol::T_REPLY, seqid); + 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.get_indexes", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.alter_function", bytes); } } -void ThriftHiveMetastoreProcessor::process_get_index_names(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +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_index_names", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_functions", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_index_names"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_functions"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_index_names"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_functions"); } - ThriftHiveMetastore_get_index_names_args args; + 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_index_names", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_functions", bytes); } - ThriftHiveMetastore_get_index_names_result result; + ThriftHiveMetastore_get_functions_result result; try { - iface_->get_index_names(result.success, args.db_name, args.tbl_name, args.max_indexes); + iface_->get_functions(result.success, args.dbName, args.pattern); result.__isset.success = true; - } catch (MetaException &o2) { - result.o2 = o2; - result.__isset.o2 = 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_index_names"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_functions"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("get_index_names", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("get_functions", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -32246,64 +35761,58 @@ void ThriftHiveMetastoreProcessor::process_get_index_names(int32_t seqid, ::apac } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_index_names"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_functions"); } - oprot->writeMessageBegin("get_index_names", ::apache::thrift::protocol::T_REPLY, seqid); + 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_index_names", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_functions", bytes); } } -void ThriftHiveMetastoreProcessor::process_update_table_column_statistics(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +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.update_table_column_statistics", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_function", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.update_table_column_statistics"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_function"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.update_table_column_statistics"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_function"); } - ThriftHiveMetastore_update_table_column_statistics_args args; + 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.update_table_column_statistics", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_function", bytes); } - ThriftHiveMetastore_update_table_column_statistics_result result; + ThriftHiveMetastore_get_function_result result; try { - result.success = iface_->update_table_column_statistics(args.stats_obj); + iface_->get_function(result.success, args.dbName, args.funcName); result.__isset.success = true; - } catch (NoSuchObjectException &o1) { + } catch (MetaException &o1) { result.o1 = o1; result.__isset.o1 = true; - } catch (InvalidObjectException &o2) { + } catch (NoSuchObjectException &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"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_function"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("update_table_column_statistics", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("get_function", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -32312,64 +35821,55 @@ void ThriftHiveMetastoreProcessor::process_update_table_column_statistics(int32_ } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.update_table_column_statistics"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_function"); } - oprot->writeMessageBegin("update_table_column_statistics", ::apache::thrift::protocol::T_REPLY, seqid); + 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.update_table_column_statistics", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_function", bytes); } } -void ThriftHiveMetastoreProcessor::process_update_partition_column_statistics(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +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.update_partition_column_statistics", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.create_role", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.update_partition_column_statistics"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.create_role"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.update_partition_column_statistics"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.create_role"); } - ThriftHiveMetastore_update_partition_column_statistics_args args; + 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.update_partition_column_statistics", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.create_role", bytes); } - ThriftHiveMetastore_update_partition_column_statistics_result result; + ThriftHiveMetastore_create_role_result result; try { - result.success = iface_->update_partition_column_statistics(args.stats_obj); + result.success = iface_->create_role(args.role); result.__isset.success = true; - } catch (NoSuchObjectException &o1) { + } catch (MetaException &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"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.create_role"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("update_partition_column_statistics", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("create_role", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -32378,64 +35878,55 @@ void ThriftHiveMetastoreProcessor::process_update_partition_column_statistics(in } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.update_partition_column_statistics"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.create_role"); } - oprot->writeMessageBegin("update_partition_column_statistics", ::apache::thrift::protocol::T_REPLY, seqid); + 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.update_partition_column_statistics", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.create_role", bytes); } } -void ThriftHiveMetastoreProcessor::process_get_table_column_statistics(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +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.get_table_column_statistics", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.drop_role", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_table_column_statistics"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.drop_role"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_table_column_statistics"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.drop_role"); } - ThriftHiveMetastore_get_table_column_statistics_args args; + 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.get_table_column_statistics", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.drop_role", bytes); } - ThriftHiveMetastore_get_table_column_statistics_result result; + ThriftHiveMetastore_drop_role_result result; try { - iface_->get_table_column_statistics(result.success, args.db_name, args.tbl_name, args.col_name); + result.success = iface_->drop_role(args.role_name); result.__isset.success = true; - } catch (NoSuchObjectException &o1) { + } catch (MetaException &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"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.drop_role"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("get_table_column_statistics", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("drop_role", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -32444,64 +35935,55 @@ void ThriftHiveMetastoreProcessor::process_get_table_column_statistics(int32_t s } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_table_column_statistics"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.drop_role"); } - oprot->writeMessageBegin("get_table_column_statistics", ::apache::thrift::protocol::T_REPLY, seqid); + 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.get_table_column_statistics", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.drop_role", bytes); } } -void ThriftHiveMetastoreProcessor::process_get_partition_column_statistics(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +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_partition_column_statistics", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_role_names", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_partition_column_statistics"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_role_names"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_partition_column_statistics"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_role_names"); } - ThriftHiveMetastore_get_partition_column_statistics_args args; + 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_partition_column_statistics", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_role_names", bytes); } - ThriftHiveMetastore_get_partition_column_statistics_result result; + ThriftHiveMetastore_get_role_names_result result; try { - iface_->get_partition_column_statistics(result.success, args.db_name, args.tbl_name, args.part_name, args.col_name); + iface_->get_role_names(result.success); result.__isset.success = true; - } catch (NoSuchObjectException &o1) { + } catch (MetaException &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"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_role_names"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("get_partition_column_statistics", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("get_role_names", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -32510,58 +35992,55 @@ void ThriftHiveMetastoreProcessor::process_get_partition_column_statistics(int32 } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_partition_column_statistics"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_role_names"); } - oprot->writeMessageBegin("get_partition_column_statistics", ::apache::thrift::protocol::T_REPLY, seqid); + 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_partition_column_statistics", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_role_names", bytes); } } -void ThriftHiveMetastoreProcessor::process_get_table_statistics_req(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +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.get_table_statistics_req", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.grant_role", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_table_statistics_req"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.grant_role"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_table_statistics_req"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.grant_role"); } - ThriftHiveMetastore_get_table_statistics_req_args args; + 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.get_table_statistics_req", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.grant_role", bytes); } - ThriftHiveMetastore_get_table_statistics_req_result result; + ThriftHiveMetastore_grant_role_result result; try { - iface_->get_table_statistics_req(result.success, args.request); + 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 (NoSuchObjectException &o1) { + } catch (MetaException &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"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.grant_role"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("get_table_statistics_req", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("grant_role", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -32570,58 +36049,55 @@ void ThriftHiveMetastoreProcessor::process_get_table_statistics_req(int32_t seqi } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_table_statistics_req"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.grant_role"); } - oprot->writeMessageBegin("get_table_statistics_req", ::apache::thrift::protocol::T_REPLY, seqid); + 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.get_table_statistics_req", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.grant_role", bytes); } } -void ThriftHiveMetastoreProcessor::process_get_partitions_statistics_req(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +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.get_partitions_statistics_req", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.revoke_role", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_partitions_statistics_req"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.revoke_role"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_partitions_statistics_req"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.revoke_role"); } - ThriftHiveMetastore_get_partitions_statistics_req_args args; + 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.get_partitions_statistics_req", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.revoke_role", bytes); } - ThriftHiveMetastore_get_partitions_statistics_req_result result; + ThriftHiveMetastore_revoke_role_result result; try { - iface_->get_partitions_statistics_req(result.success, args.request); + result.success = iface_->revoke_role(args.role_name, args.principal_name, args.principal_type); result.__isset.success = true; - } catch (NoSuchObjectException &o1) { + } catch (MetaException &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"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.revoke_role"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("get_partitions_statistics_req", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("revoke_role", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -32630,64 +36106,55 @@ void ThriftHiveMetastoreProcessor::process_get_partitions_statistics_req(int32_t } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_partitions_statistics_req"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.revoke_role"); } - oprot->writeMessageBegin("get_partitions_statistics_req", ::apache::thrift::protocol::T_REPLY, seqid); + 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.get_partitions_statistics_req", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.revoke_role", bytes); } } -void ThriftHiveMetastoreProcessor::process_delete_partition_column_statistics(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +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.delete_partition_column_statistics", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.list_roles", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.delete_partition_column_statistics"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.list_roles"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.delete_partition_column_statistics"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.list_roles"); } - ThriftHiveMetastore_delete_partition_column_statistics_args args; + 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.delete_partition_column_statistics", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.list_roles", bytes); } - ThriftHiveMetastore_delete_partition_column_statistics_result result; + ThriftHiveMetastore_list_roles_result result; try { - result.success = iface_->delete_partition_column_statistics(args.db_name, args.tbl_name, args.part_name, args.col_name); + iface_->list_roles(result.success, args.principal_name, args.principal_type); result.__isset.success = true; - } catch (NoSuchObjectException &o1) { + } catch (MetaException &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"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.list_roles"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("delete_partition_column_statistics", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("list_roles", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -32696,64 +36163,55 @@ void ThriftHiveMetastoreProcessor::process_delete_partition_column_statistics(in } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.delete_partition_column_statistics"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.list_roles"); } - oprot->writeMessageBegin("delete_partition_column_statistics", ::apache::thrift::protocol::T_REPLY, seqid); + 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.delete_partition_column_statistics", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.list_roles", bytes); } } -void ThriftHiveMetastoreProcessor::process_delete_table_column_statistics(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +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.delete_table_column_statistics", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_privilege_set", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.delete_table_column_statistics"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_privilege_set"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.delete_table_column_statistics"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_privilege_set"); } - ThriftHiveMetastore_delete_table_column_statistics_args args; + 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.delete_table_column_statistics", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_privilege_set", bytes); } - ThriftHiveMetastore_delete_table_column_statistics_result result; + ThriftHiveMetastore_get_privilege_set_result result; try { - result.success = iface_->delete_table_column_statistics(args.db_name, args.tbl_name, args.col_name); + iface_->get_privilege_set(result.success, args.hiveObject, args.user_name, args.group_names); result.__isset.success = true; - } catch (NoSuchObjectException &o1) { + } catch (MetaException &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"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_privilege_set"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("delete_table_column_statistics", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("get_privilege_set", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -32762,63 +36220,55 @@ void ThriftHiveMetastoreProcessor::process_delete_table_column_statistics(int32_ } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.delete_table_column_statistics"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_privilege_set"); } - oprot->writeMessageBegin("delete_table_column_statistics", ::apache::thrift::protocol::T_REPLY, seqid); + 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.delete_table_column_statistics", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_privilege_set", bytes); } } -void ThriftHiveMetastoreProcessor::process_create_function(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +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.create_function", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.list_privileges", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.create_function"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.list_privileges"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.create_function"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.list_privileges"); } - ThriftHiveMetastore_create_function_args args; + 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.create_function", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.list_privileges", bytes); } - ThriftHiveMetastore_create_function_result result; + ThriftHiveMetastore_list_privileges_result result; try { - iface_->create_function(args.func); - } catch (AlreadyExistsException &o1) { + 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 (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"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.list_privileges"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("create_function", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("list_privileges", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -32827,57 +36277,55 @@ void ThriftHiveMetastoreProcessor::process_create_function(int32_t seqid, ::apac } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.create_function"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.list_privileges"); } - oprot->writeMessageBegin("create_function", ::apache::thrift::protocol::T_REPLY, seqid); + 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.create_function", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.list_privileges", bytes); } } -void ThriftHiveMetastoreProcessor::process_drop_function(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +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.drop_function", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.grant_privileges", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.drop_function"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.grant_privileges"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.drop_function"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.grant_privileges"); } - ThriftHiveMetastore_drop_function_args args; + 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.drop_function", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.grant_privileges", bytes); } - ThriftHiveMetastore_drop_function_result result; + ThriftHiveMetastore_grant_privileges_result result; try { - iface_->drop_function(args.dbName, args.funcName); - } catch (NoSuchObjectException &o1) { + result.success = iface_->grant_privileges(args.privileges); + result.__isset.success = true; + } catch (MetaException &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"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.grant_privileges"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("drop_function", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("grant_privileges", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -32886,57 +36334,55 @@ void ThriftHiveMetastoreProcessor::process_drop_function(int32_t seqid, ::apache } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.drop_function"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.grant_privileges"); } - oprot->writeMessageBegin("drop_function", ::apache::thrift::protocol::T_REPLY, seqid); + 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.drop_function", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.grant_privileges", bytes); } } -void ThriftHiveMetastoreProcessor::process_alter_function(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +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.alter_function", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.revoke_privileges", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.alter_function"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.revoke_privileges"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.alter_function"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.revoke_privileges"); } - ThriftHiveMetastore_alter_function_args args; + 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.alter_function", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.revoke_privileges", bytes); } - ThriftHiveMetastore_alter_function_result result; + ThriftHiveMetastore_revoke_privileges_result result; try { - iface_->alter_function(args.dbName, args.funcName, args.newFunc); - } catch (InvalidOperationException &o1) { + result.success = iface_->revoke_privileges(args.privileges); + result.__isset.success = true; + } catch (MetaException &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"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.revoke_privileges"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("alter_function", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("revoke_privileges", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -32945,55 +36391,55 @@ void ThriftHiveMetastoreProcessor::process_alter_function(int32_t seqid, ::apach } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.alter_function"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.revoke_privileges"); } - oprot->writeMessageBegin("alter_function", ::apache::thrift::protocol::T_REPLY, seqid); + 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.alter_function", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.revoke_privileges", bytes); } } -void ThriftHiveMetastoreProcessor::process_get_functions(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +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.get_functions", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.set_ugi", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_functions"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.set_ugi"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_functions"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.set_ugi"); } - ThriftHiveMetastore_get_functions_args args; + 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.get_functions", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.set_ugi", bytes); } - ThriftHiveMetastore_get_functions_result result; + ThriftHiveMetastore_set_ugi_result result; try { - iface_->get_functions(result.success, args.dbName, args.pattern); + 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.get_functions"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.set_ugi"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("get_functions", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("set_ugi", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -33002,58 +36448,55 @@ void ThriftHiveMetastoreProcessor::process_get_functions(int32_t seqid, ::apache } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_functions"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.set_ugi"); } - oprot->writeMessageBegin("get_functions", ::apache::thrift::protocol::T_REPLY, seqid); + 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.get_functions", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.set_ugi", bytes); } } -void ThriftHiveMetastoreProcessor::process_get_function(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +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_function", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_delegation_token", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_function"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_delegation_token"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_function"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_delegation_token"); } - ThriftHiveMetastore_get_function_args args; + 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_function", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_delegation_token", bytes); } - ThriftHiveMetastore_get_function_result result; + ThriftHiveMetastore_get_delegation_token_result result; try { - iface_->get_function(result.success, args.dbName, args.funcName); + 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 (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"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_delegation_token"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("get_function", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("get_delegation_token", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -33062,55 +36505,55 @@ void ThriftHiveMetastoreProcessor::process_get_function(int32_t seqid, ::apache: } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_function"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_delegation_token"); } - oprot->writeMessageBegin("get_function", ::apache::thrift::protocol::T_REPLY, seqid); + 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_function", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_delegation_token", bytes); } } -void ThriftHiveMetastoreProcessor::process_create_role(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +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.create_role", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.renew_delegation_token", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.create_role"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.renew_delegation_token"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.create_role"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.renew_delegation_token"); } - ThriftHiveMetastore_create_role_args args; + 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.create_role", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.renew_delegation_token", bytes); } - ThriftHiveMetastore_create_role_result result; + ThriftHiveMetastore_renew_delegation_token_result result; try { - result.success = iface_->create_role(args.role); + 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.create_role"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.renew_delegation_token"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("create_role", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("renew_delegation_token", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -33119,55 +36562,54 @@ void ThriftHiveMetastoreProcessor::process_create_role(int32_t seqid, ::apache:: } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.create_role"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.renew_delegation_token"); } - oprot->writeMessageBegin("create_role", ::apache::thrift::protocol::T_REPLY, seqid); + 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.create_role", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.renew_delegation_token", bytes); } } -void ThriftHiveMetastoreProcessor::process_drop_role(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +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.drop_role", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.cancel_delegation_token", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.drop_role"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.cancel_delegation_token"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.drop_role"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.cancel_delegation_token"); } - ThriftHiveMetastore_drop_role_args args; + 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.drop_role", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.cancel_delegation_token", bytes); } - ThriftHiveMetastore_drop_role_result result; + ThriftHiveMetastore_cancel_delegation_token_result result; try { - result.success = iface_->drop_role(args.role_name); - result.__isset.success = true; + 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.drop_role"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.cancel_delegation_token"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("drop_role", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("cancel_delegation_token", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -33176,55 +36618,52 @@ void ThriftHiveMetastoreProcessor::process_drop_role(int32_t seqid, ::apache::th } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.drop_role"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.cancel_delegation_token"); } - oprot->writeMessageBegin("drop_role", ::apache::thrift::protocol::T_REPLY, seqid); + 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.drop_role", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.cancel_delegation_token", bytes); } } -void ThriftHiveMetastoreProcessor::process_get_role_names(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +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_role_names", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_open_txns", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_role_names"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_open_txns"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_role_names"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_open_txns"); } - ThriftHiveMetastore_get_role_names_args args; + 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_role_names", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_open_txns", bytes); } - ThriftHiveMetastore_get_role_names_result result; + ThriftHiveMetastore_get_open_txns_result result; try { - iface_->get_role_names(result.success); + iface_->get_open_txns(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"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_open_txns"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("get_role_names", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("get_open_txns", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -33233,55 +36672,52 @@ void ThriftHiveMetastoreProcessor::process_get_role_names(int32_t seqid, ::apach } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_role_names"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_open_txns"); } - oprot->writeMessageBegin("get_role_names", ::apache::thrift::protocol::T_REPLY, seqid); + 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_role_names", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_open_txns", bytes); } } -void ThriftHiveMetastoreProcessor::process_grant_role(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +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.grant_role", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.get_open_txns_info", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.grant_role"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_open_txns_info"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.grant_role"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_open_txns_info"); } - ThriftHiveMetastore_grant_role_args args; + 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.grant_role", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.get_open_txns_info", bytes); } - ThriftHiveMetastore_grant_role_result result; + ThriftHiveMetastore_get_open_txns_info_result result; try { - result.success = iface_->grant_role(args.role_name, args.principal_name, args.principal_type, args.grantor, args.grantorType, args.grant_option); + iface_->get_open_txns_info(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.grant_role"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.get_open_txns_info"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("grant_role", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("get_open_txns_info", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -33290,55 +36726,52 @@ void ThriftHiveMetastoreProcessor::process_grant_role(int32_t seqid, ::apache::t } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.grant_role"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_open_txns_info"); } - oprot->writeMessageBegin("grant_role", ::apache::thrift::protocol::T_REPLY, seqid); + 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.grant_role", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.get_open_txns_info", bytes); } } -void ThriftHiveMetastoreProcessor::process_revoke_role(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +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.revoke_role", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.open_txns", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.revoke_role"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.open_txns"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.revoke_role"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.open_txns"); } - ThriftHiveMetastore_revoke_role_args args; + 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.revoke_role", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.open_txns", bytes); } - ThriftHiveMetastore_revoke_role_result result; + ThriftHiveMetastore_open_txns_result result; try { - result.success = iface_->revoke_role(args.role_name, args.principal_name, args.principal_type); + iface_->open_txns(result.success, args.rqst); 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"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.open_txns"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("revoke_role", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("open_txns", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -33347,55 +36780,54 @@ void ThriftHiveMetastoreProcessor::process_revoke_role(int32_t seqid, ::apache:: } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.revoke_role"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.open_txns"); } - oprot->writeMessageBegin("revoke_role", ::apache::thrift::protocol::T_REPLY, seqid); + 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.revoke_role", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.open_txns", bytes); } } -void ThriftHiveMetastoreProcessor::process_list_roles(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +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.list_roles", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.abort_txn", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.list_roles"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.abort_txn"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.list_roles"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.abort_txn"); } - ThriftHiveMetastore_list_roles_args args; + 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.list_roles", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.abort_txn", bytes); } - ThriftHiveMetastore_list_roles_result result; + ThriftHiveMetastore_abort_txn_result result; try { - iface_->list_roles(result.success, args.principal_name, args.principal_type); - result.__isset.success = true; - } catch (MetaException &o1) { + 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.list_roles"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.abort_txn"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("list_roles", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("abort_txn", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -33404,55 +36836,57 @@ void ThriftHiveMetastoreProcessor::process_list_roles(int32_t seqid, ::apache::t } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.list_roles"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.abort_txn"); } - oprot->writeMessageBegin("list_roles", ::apache::thrift::protocol::T_REPLY, seqid); + 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.list_roles", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.abort_txn", bytes); } } -void ThriftHiveMetastoreProcessor::process_get_privilege_set(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +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.get_privilege_set", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.commit_txn", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_privilege_set"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.commit_txn"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_privilege_set"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.commit_txn"); } - ThriftHiveMetastore_get_privilege_set_args args; + 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.get_privilege_set", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.commit_txn", bytes); } - ThriftHiveMetastore_get_privilege_set_result result; + ThriftHiveMetastore_commit_txn_result result; try { - iface_->get_privilege_set(result.success, args.hiveObject, args.user_name, args.group_names); - result.__isset.success = true; - } catch (MetaException &o1) { + 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.get_privilege_set"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.commit_txn"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("get_privilege_set", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("commit_txn", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -33461,55 +36895,58 @@ void ThriftHiveMetastoreProcessor::process_get_privilege_set(int32_t seqid, ::ap } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_privilege_set"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.commit_txn"); } - oprot->writeMessageBegin("get_privilege_set", ::apache::thrift::protocol::T_REPLY, seqid); + 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.get_privilege_set", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.commit_txn", bytes); } } -void ThriftHiveMetastoreProcessor::process_list_privileges(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +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.list_privileges", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.lock", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.list_privileges"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.lock"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.list_privileges"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.lock"); } - ThriftHiveMetastore_list_privileges_args args; + 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.list_privileges", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.lock", bytes); } - ThriftHiveMetastore_list_privileges_result result; + ThriftHiveMetastore_lock_result result; try { - iface_->list_privileges(result.success, args.principal_name, args.principal_type, args.hiveObject); + iface_->lock(result.success, args.rqst); result.__isset.success = true; - } catch (MetaException &o1) { + } 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.list_privileges"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.lock"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("list_privileges", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("lock", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -33518,55 +36955,61 @@ void ThriftHiveMetastoreProcessor::process_list_privileges(int32_t seqid, ::apac } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.list_privileges"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.lock"); } - oprot->writeMessageBegin("list_privileges", ::apache::thrift::protocol::T_REPLY, seqid); + 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.list_privileges", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.lock", bytes); } } -void ThriftHiveMetastoreProcessor::process_grant_privileges(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +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.grant_privileges", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.check_lock", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.grant_privileges"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.check_lock"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.grant_privileges"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.check_lock"); } - ThriftHiveMetastore_grant_privileges_args args; + 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.grant_privileges", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.check_lock", bytes); } - ThriftHiveMetastore_grant_privileges_result result; + ThriftHiveMetastore_check_lock_result result; try { - result.success = iface_->grant_privileges(args.privileges); + iface_->check_lock(result.success, args.rqst); result.__isset.success = true; - } catch (MetaException &o1) { + } 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.grant_privileges"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.check_lock"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("grant_privileges", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("check_lock", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -33575,55 +37018,57 @@ void ThriftHiveMetastoreProcessor::process_grant_privileges(int32_t seqid, ::apa } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.grant_privileges"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.check_lock"); } - oprot->writeMessageBegin("grant_privileges", ::apache::thrift::protocol::T_REPLY, seqid); + 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.grant_privileges", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.check_lock", bytes); } } -void ThriftHiveMetastoreProcessor::process_revoke_privileges(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +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.revoke_privileges", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.unlock", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.revoke_privileges"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.unlock"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.revoke_privileges"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.unlock"); } - ThriftHiveMetastore_revoke_privileges_args args; + 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.revoke_privileges", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.unlock", bytes); } - ThriftHiveMetastore_revoke_privileges_result result; + ThriftHiveMetastore_unlock_result result; try { - result.success = iface_->revoke_privileges(args.privileges); - result.__isset.success = true; - } catch (MetaException &o1) { + 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.revoke_privileges"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.unlock"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("revoke_privileges", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("unlock", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -33632,55 +37077,52 @@ void ThriftHiveMetastoreProcessor::process_revoke_privileges(int32_t seqid, ::ap } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.revoke_privileges"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.unlock"); } - oprot->writeMessageBegin("revoke_privileges", ::apache::thrift::protocol::T_REPLY, seqid); + 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.revoke_privileges", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.unlock", bytes); } } -void ThriftHiveMetastoreProcessor::process_set_ugi(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +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.set_ugi", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.show_locks", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.set_ugi"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.show_locks"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.set_ugi"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.show_locks"); } - ThriftHiveMetastore_set_ugi_args args; + 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.set_ugi", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.show_locks", bytes); } - ThriftHiveMetastore_set_ugi_result result; + ThriftHiveMetastore_show_locks_result result; try { - iface_->set_ugi(result.success, args.user_name, args.group_names); + iface_->show_locks(result.success, args.rqst); 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"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.show_locks"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("set_ugi", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("show_locks", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -33689,55 +37131,60 @@ void ThriftHiveMetastoreProcessor::process_set_ugi(int32_t seqid, ::apache::thri } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.set_ugi"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.show_locks"); } - oprot->writeMessageBegin("set_ugi", ::apache::thrift::protocol::T_REPLY, seqid); + 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.set_ugi", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.show_locks", bytes); } } -void ThriftHiveMetastoreProcessor::process_get_delegation_token(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +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.get_delegation_token", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.heartbeat", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.get_delegation_token"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.heartbeat"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.get_delegation_token"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.heartbeat"); } - ThriftHiveMetastore_get_delegation_token_args args; + 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.get_delegation_token", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.heartbeat", bytes); } - ThriftHiveMetastore_get_delegation_token_result result; + ThriftHiveMetastore_heartbeat_result result; try { - iface_->get_delegation_token(result.success, args.token_owner, args.renewer_kerberos_principal_name); - result.__isset.success = true; - } catch (MetaException &o1) { + 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.get_delegation_token"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.heartbeat"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("get_delegation_token", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("heartbeat", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -33746,55 +37193,51 @@ void ThriftHiveMetastoreProcessor::process_get_delegation_token(int32_t seqid, : } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.get_delegation_token"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.heartbeat"); } - oprot->writeMessageBegin("get_delegation_token", ::apache::thrift::protocol::T_REPLY, seqid); + 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.get_delegation_token", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.heartbeat", bytes); } } -void ThriftHiveMetastoreProcessor::process_renew_delegation_token(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +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.renew_delegation_token", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.compact", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.renew_delegation_token"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.compact"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.renew_delegation_token"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.compact"); } - ThriftHiveMetastore_renew_delegation_token_args args; + 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.renew_delegation_token", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.compact", bytes); } - ThriftHiveMetastore_renew_delegation_token_result result; + ThriftHiveMetastore_compact_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; + iface_->compact(args.rqst); } catch (const std::exception& e) { if (this->eventHandler_.get() != NULL) { - this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.renew_delegation_token"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.compact"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("renew_delegation_token", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("compact", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -33803,54 +37246,52 @@ void ThriftHiveMetastoreProcessor::process_renew_delegation_token(int32_t seqid, } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.renew_delegation_token"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.compact"); } - oprot->writeMessageBegin("renew_delegation_token", ::apache::thrift::protocol::T_REPLY, seqid); + 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.renew_delegation_token", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.compact", bytes); } } -void ThriftHiveMetastoreProcessor::process_cancel_delegation_token(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext) +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.cancel_delegation_token", callContext); + ctx = this->eventHandler_->getContext("ThriftHiveMetastore.show_compact", callContext); } - ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.cancel_delegation_token"); + ::apache::thrift::TProcessorContextFreer freer(this->eventHandler_.get(), ctx, "ThriftHiveMetastore.show_compact"); if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.cancel_delegation_token"); + this->eventHandler_->preRead(ctx, "ThriftHiveMetastore.show_compact"); } - ThriftHiveMetastore_cancel_delegation_token_args args; + 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.cancel_delegation_token", bytes); + this->eventHandler_->postRead(ctx, "ThriftHiveMetastore.show_compact", bytes); } - ThriftHiveMetastore_cancel_delegation_token_result result; + ThriftHiveMetastore_show_compact_result result; try { - iface_->cancel_delegation_token(args.token_str_form); - } catch (MetaException &o1) { - result.o1 = o1; - result.__isset.o1 = true; + 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.cancel_delegation_token"); + this->eventHandler_->handlerError(ctx, "ThriftHiveMetastore.show_compact"); } ::apache::thrift::TApplicationException x(e.what()); - oprot->writeMessageBegin("cancel_delegation_token", ::apache::thrift::protocol::T_EXCEPTION, seqid); + oprot->writeMessageBegin("show_compact", ::apache::thrift::protocol::T_EXCEPTION, seqid); x.write(oprot); oprot->writeMessageEnd(); oprot->getTransport()->writeEnd(); @@ -33859,17 +37300,17 @@ void ThriftHiveMetastoreProcessor::process_cancel_delegation_token(int32_t seqid } if (this->eventHandler_.get() != NULL) { - this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.cancel_delegation_token"); + this->eventHandler_->preWrite(ctx, "ThriftHiveMetastore.show_compact"); } - oprot->writeMessageBegin("cancel_delegation_token", ::apache::thrift::protocol::T_REPLY, seqid); + 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.cancel_delegation_token", bytes); + this->eventHandler_->postWrite(ctx, "ThriftHiveMetastore.show_compact", bytes); } } diff --git metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.h metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.h index 828c50d..d24f781 100644 --- metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.h +++ metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.h @@ -108,6 +108,18 @@ class ThriftHiveMetastoreIf : virtual public ::facebook::fb303::FacebookService 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 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 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 compact(const CompactionRequest& rqst) = 0; + virtual void show_compact(ShowCompactResponse& _return, const ShowCompactRequest& rqst) = 0; }; class ThriftHiveMetastoreIfFactory : virtual public ::facebook::fb303::FacebookServiceIfFactory { @@ -434,6 +446,42 @@ class ThriftHiveMetastoreNull : virtual public ThriftHiveMetastoreIf , virtual p void cancel_delegation_token(const std::string& /* token_str_form */) { 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 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 compact(const CompactionRequest& /* rqst */) { + return; + } + void show_compact(ShowCompactResponse& /* _return */, const ShowCompactRequest& /* rqst */) { + return; + } }; typedef struct _ThriftHiveMetastore_create_database_args__isset { @@ -13452,234 +13500,1572 @@ class ThriftHiveMetastore_cancel_delegation_token_presult { }; -class ThriftHiveMetastoreClient : virtual public ThriftHiveMetastoreIf, public ::facebook::fb303::FacebookServiceClient { + +class ThriftHiveMetastore_get_open_txns_args { public: - ThriftHiveMetastoreClient(boost::shared_ptr< ::apache::thrift::protocol::TProtocol> prot) : - ::facebook::fb303::FacebookServiceClient(prot, prot) {} - ThriftHiveMetastoreClient(boost::shared_ptr< ::apache::thrift::protocol::TProtocol> iprot, boost::shared_ptr< ::apache::thrift::protocol::TProtocol> oprot) : - ::facebook::fb303::FacebookServiceClient(iprot, oprot) {} - boost::shared_ptr< ::apache::thrift::protocol::TProtocol> getInputProtocol() { - return piprot_; + + ThriftHiveMetastore_get_open_txns_args() { } - boost::shared_ptr< ::apache::thrift::protocol::TProtocol> getOutputProtocol() { - return poprot_; + + virtual ~ThriftHiveMetastore_get_open_txns_args() throw() {} + + + bool operator == (const ThriftHiveMetastore_get_open_txns_args & /* rhs */) const + { + return true; } - 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_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 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 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 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_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_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 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(); - 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 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_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_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_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); - 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_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); - 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); - 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 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() throw() {} + + + 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; +} _ThriftHiveMetastore_get_open_txns_result__isset; + +class ThriftHiveMetastore_get_open_txns_result { + public: + + ThriftHiveMetastore_get_open_txns_result() { + } + + virtual ~ThriftHiveMetastore_get_open_txns_result() throw() {} + + GetOpenTxnsResponse success; + + _ThriftHiveMetastore_get_open_txns_result__isset __isset; + + void __set_success(const GetOpenTxnsResponse& val) { + success = 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; +} _ThriftHiveMetastore_get_open_txns_presult__isset; + +class ThriftHiveMetastore_get_open_txns_presult { + public: + + + virtual ~ThriftHiveMetastore_get_open_txns_presult() throw() {} + + 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() { + } + + virtual ~ThriftHiveMetastore_get_open_txns_info_args() throw() {} + + + 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() throw() {} + + + 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; +} _ThriftHiveMetastore_get_open_txns_info_result__isset; + +class ThriftHiveMetastore_get_open_txns_info_result { + public: + + ThriftHiveMetastore_get_open_txns_info_result() { + } + + virtual ~ThriftHiveMetastore_get_open_txns_info_result() throw() {} + + GetOpenTxnsInfoResponse success; + + _ThriftHiveMetastore_get_open_txns_info_result__isset __isset; + + void __set_success(const GetOpenTxnsInfoResponse& val) { + success = 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; +} _ThriftHiveMetastore_get_open_txns_info_presult__isset; + +class ThriftHiveMetastore_get_open_txns_info_presult { + public: + + + virtual ~ThriftHiveMetastore_get_open_txns_info_presult() throw() {} + + 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; +} _ThriftHiveMetastore_open_txns_args__isset; + +class ThriftHiveMetastore_open_txns_args { + public: + + ThriftHiveMetastore_open_txns_args() { + } + + virtual ~ThriftHiveMetastore_open_txns_args() throw() {} + + OpenTxnRequest rqst; + + _ThriftHiveMetastore_open_txns_args__isset __isset; + + void __set_rqst(const OpenTxnRequest& val) { + rqst = 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() throw() {} + + 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; +} _ThriftHiveMetastore_open_txns_result__isset; + +class ThriftHiveMetastore_open_txns_result { + public: + + ThriftHiveMetastore_open_txns_result() { + } + + virtual ~ThriftHiveMetastore_open_txns_result() throw() {} + + OpenTxnsResponse success; + + _ThriftHiveMetastore_open_txns_result__isset __isset; + + void __set_success(const OpenTxnsResponse& val) { + success = 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; +} _ThriftHiveMetastore_open_txns_presult__isset; + +class ThriftHiveMetastore_open_txns_presult { + public: + + + virtual ~ThriftHiveMetastore_open_txns_presult() throw() {} + + 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; +} _ThriftHiveMetastore_abort_txn_args__isset; + +class ThriftHiveMetastore_abort_txn_args { + public: + + ThriftHiveMetastore_abort_txn_args() { + } + + virtual ~ThriftHiveMetastore_abort_txn_args() throw() {} + + AbortTxnRequest rqst; + + _ThriftHiveMetastore_abort_txn_args__isset __isset; + + void __set_rqst(const AbortTxnRequest& val) { + rqst = 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() throw() {} + + 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; +} _ThriftHiveMetastore_abort_txn_result__isset; + +class ThriftHiveMetastore_abort_txn_result { + public: + + ThriftHiveMetastore_abort_txn_result() { + } + + virtual ~ThriftHiveMetastore_abort_txn_result() throw() {} + + NoSuchTxnException o1; + + _ThriftHiveMetastore_abort_txn_result__isset __isset; + + void __set_o1(const NoSuchTxnException& val) { + o1 = 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; +} _ThriftHiveMetastore_abort_txn_presult__isset; + +class ThriftHiveMetastore_abort_txn_presult { + public: + + + virtual ~ThriftHiveMetastore_abort_txn_presult() throw() {} + + NoSuchTxnException o1; + + _ThriftHiveMetastore_abort_txn_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; +} _ThriftHiveMetastore_commit_txn_args__isset; + +class ThriftHiveMetastore_commit_txn_args { + public: + + ThriftHiveMetastore_commit_txn_args() { + } + + virtual ~ThriftHiveMetastore_commit_txn_args() throw() {} + + CommitTxnRequest rqst; + + _ThriftHiveMetastore_commit_txn_args__isset __isset; + + void __set_rqst(const CommitTxnRequest& val) { + rqst = 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() throw() {} + + 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; + bool o2; +} _ThriftHiveMetastore_commit_txn_result__isset; + +class ThriftHiveMetastore_commit_txn_result { + public: + + ThriftHiveMetastore_commit_txn_result() { + } + + virtual ~ThriftHiveMetastore_commit_txn_result() throw() {} + + NoSuchTxnException o1; + TxnAbortedException o2; + + _ThriftHiveMetastore_commit_txn_result__isset __isset; + + void __set_o1(const NoSuchTxnException& val) { + o1 = val; + } + + void __set_o2(const TxnAbortedException& val) { + o2 = 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; + bool o2; +} _ThriftHiveMetastore_commit_txn_presult__isset; + +class ThriftHiveMetastore_commit_txn_presult { + public: + + + virtual ~ThriftHiveMetastore_commit_txn_presult() throw() {} + + 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; +} _ThriftHiveMetastore_lock_args__isset; + +class ThriftHiveMetastore_lock_args { + public: + + ThriftHiveMetastore_lock_args() { + } + + virtual ~ThriftHiveMetastore_lock_args() throw() {} + + LockRequest rqst; + + _ThriftHiveMetastore_lock_args__isset __isset; + + void __set_rqst(const LockRequest& val) { + rqst = 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() throw() {} + + 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; + bool o1; + bool o2; +} _ThriftHiveMetastore_lock_result__isset; + +class ThriftHiveMetastore_lock_result { + public: + + ThriftHiveMetastore_lock_result() { + } + + virtual ~ThriftHiveMetastore_lock_result() throw() {} + + LockResponse success; + NoSuchTxnException o1; + TxnAbortedException o2; + + _ThriftHiveMetastore_lock_result__isset __isset; + + void __set_success(const LockResponse& val) { + success = val; + } + + void __set_o1(const NoSuchTxnException& val) { + o1 = val; + } + + void __set_o2(const TxnAbortedException& val) { + o2 = 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; + bool o1; + bool o2; +} _ThriftHiveMetastore_lock_presult__isset; + +class ThriftHiveMetastore_lock_presult { + public: + + + virtual ~ThriftHiveMetastore_lock_presult() throw() {} + + 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; +} _ThriftHiveMetastore_check_lock_args__isset; + +class ThriftHiveMetastore_check_lock_args { + public: + + ThriftHiveMetastore_check_lock_args() { + } + + virtual ~ThriftHiveMetastore_check_lock_args() throw() {} + + CheckLockRequest rqst; + + _ThriftHiveMetastore_check_lock_args__isset __isset; + + void __set_rqst(const CheckLockRequest& val) { + rqst = 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() throw() {} + + 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; + bool o1; + bool o2; + bool o3; +} _ThriftHiveMetastore_check_lock_result__isset; + +class ThriftHiveMetastore_check_lock_result { + public: + + ThriftHiveMetastore_check_lock_result() { + } + + virtual ~ThriftHiveMetastore_check_lock_result() throw() {} + + LockResponse success; + NoSuchTxnException o1; + TxnAbortedException o2; + NoSuchLockException o3; + + _ThriftHiveMetastore_check_lock_result__isset __isset; + + void __set_success(const LockResponse& val) { + success = val; + } + + void __set_o1(const NoSuchTxnException& val) { + o1 = val; + } + + void __set_o2(const TxnAbortedException& val) { + o2 = val; + } + + void __set_o3(const NoSuchLockException& val) { + o3 = 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; + bool o1; + bool o2; + bool o3; +} _ThriftHiveMetastore_check_lock_presult__isset; + +class ThriftHiveMetastore_check_lock_presult { + public: + + + virtual ~ThriftHiveMetastore_check_lock_presult() throw() {} + + 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; +} _ThriftHiveMetastore_unlock_args__isset; + +class ThriftHiveMetastore_unlock_args { + public: + + ThriftHiveMetastore_unlock_args() { + } + + virtual ~ThriftHiveMetastore_unlock_args() throw() {} + + UnlockRequest rqst; + + _ThriftHiveMetastore_unlock_args__isset __isset; + + void __set_rqst(const UnlockRequest& val) { + rqst = 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() throw() {} + + 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; + bool o2; +} _ThriftHiveMetastore_unlock_result__isset; + +class ThriftHiveMetastore_unlock_result { + public: + + ThriftHiveMetastore_unlock_result() { + } + + virtual ~ThriftHiveMetastore_unlock_result() throw() {} + + NoSuchLockException o1; + TxnOpenException o2; + + _ThriftHiveMetastore_unlock_result__isset __isset; + + void __set_o1(const NoSuchLockException& val) { + o1 = val; + } + + void __set_o2(const TxnOpenException& val) { + o2 = 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; + bool o2; +} _ThriftHiveMetastore_unlock_presult__isset; + +class ThriftHiveMetastore_unlock_presult { + public: + + + virtual ~ThriftHiveMetastore_unlock_presult() throw() {} + + 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; +} _ThriftHiveMetastore_show_locks_args__isset; + +class ThriftHiveMetastore_show_locks_args { + public: + + ThriftHiveMetastore_show_locks_args() { + } + + virtual ~ThriftHiveMetastore_show_locks_args() throw() {} + + ShowLocksRequest rqst; + + _ThriftHiveMetastore_show_locks_args__isset __isset; + + void __set_rqst(const ShowLocksRequest& val) { + rqst = 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() throw() {} + + 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; +} _ThriftHiveMetastore_show_locks_result__isset; + +class ThriftHiveMetastore_show_locks_result { + public: + + ThriftHiveMetastore_show_locks_result() { + } + + virtual ~ThriftHiveMetastore_show_locks_result() throw() {} + + ShowLocksResponse success; + + _ThriftHiveMetastore_show_locks_result__isset __isset; + + void __set_success(const ShowLocksResponse& val) { + success = 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; +} _ThriftHiveMetastore_show_locks_presult__isset; + +class ThriftHiveMetastore_show_locks_presult { + public: + + + virtual ~ThriftHiveMetastore_show_locks_presult() throw() {} + + 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; +} _ThriftHiveMetastore_heartbeat_args__isset; + +class ThriftHiveMetastore_heartbeat_args { + public: + + ThriftHiveMetastore_heartbeat_args() { + } + + virtual ~ThriftHiveMetastore_heartbeat_args() throw() {} + + HeartbeatRequest ids; + + _ThriftHiveMetastore_heartbeat_args__isset __isset; + + void __set_ids(const HeartbeatRequest& val) { + ids = 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() throw() {} + + 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; + bool o2; + bool o3; +} _ThriftHiveMetastore_heartbeat_result__isset; + +class ThriftHiveMetastore_heartbeat_result { + public: + + ThriftHiveMetastore_heartbeat_result() { + } + + virtual ~ThriftHiveMetastore_heartbeat_result() throw() {} + + NoSuchLockException o1; + NoSuchTxnException o2; + TxnAbortedException o3; + + _ThriftHiveMetastore_heartbeat_result__isset __isset; + + void __set_o1(const NoSuchLockException& val) { + o1 = val; + } + + void __set_o2(const NoSuchTxnException& val) { + o2 = val; + } + + void __set_o3(const TxnAbortedException& val) { + o3 = 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; + bool o2; + bool o3; +} _ThriftHiveMetastore_heartbeat_presult__isset; + +class ThriftHiveMetastore_heartbeat_presult { + public: + + + virtual ~ThriftHiveMetastore_heartbeat_presult() throw() {} + + NoSuchLockException o1; + NoSuchTxnException o2; + TxnAbortedException o3; + + _ThriftHiveMetastore_heartbeat_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; +} _ThriftHiveMetastore_compact_args__isset; + +class ThriftHiveMetastore_compact_args { + public: + + ThriftHiveMetastore_compact_args() { + } + + virtual ~ThriftHiveMetastore_compact_args() throw() {} + + CompactionRequest rqst; + + _ThriftHiveMetastore_compact_args__isset __isset; + + void __set_rqst(const CompactionRequest& val) { + rqst = 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() throw() {} + + const CompactionRequest* rqst; + + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + + +class ThriftHiveMetastore_compact_result { + public: + + ThriftHiveMetastore_compact_result() { + } + + virtual ~ThriftHiveMetastore_compact_result() throw() {} + + + 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() throw() {} + + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +typedef struct _ThriftHiveMetastore_show_compact_args__isset { + _ThriftHiveMetastore_show_compact_args__isset() : rqst(false) {} + bool rqst; +} _ThriftHiveMetastore_show_compact_args__isset; + +class ThriftHiveMetastore_show_compact_args { + public: + + ThriftHiveMetastore_show_compact_args() { + } + + virtual ~ThriftHiveMetastore_show_compact_args() throw() {} + + ShowCompactRequest rqst; + + _ThriftHiveMetastore_show_compact_args__isset __isset; + + void __set_rqst(const ShowCompactRequest& val) { + rqst = 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() throw() {} + + 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; +} _ThriftHiveMetastore_show_compact_result__isset; + +class ThriftHiveMetastore_show_compact_result { + public: + + ThriftHiveMetastore_show_compact_result() { + } + + virtual ~ThriftHiveMetastore_show_compact_result() throw() {} + + ShowCompactResponse success; + + _ThriftHiveMetastore_show_compact_result__isset __isset; + + void __set_success(const ShowCompactResponse& val) { + success = 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; +} _ThriftHiveMetastore_show_compact_presult__isset; + +class ThriftHiveMetastore_show_compact_presult { + public: + + + virtual ~ThriftHiveMetastore_show_compact_presult() throw() {} + + ShowCompactResponse* success; + + _ThriftHiveMetastore_show_compact_presult__isset __isset; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + +}; + +class ThriftHiveMetastoreClient : virtual public ThriftHiveMetastoreIf, public ::facebook::fb303::FacebookServiceClient { + public: + ThriftHiveMetastoreClient(boost::shared_ptr< ::apache::thrift::protocol::TProtocol> prot) : + ::facebook::fb303::FacebookServiceClient(prot, prot) {} + ThriftHiveMetastoreClient(boost::shared_ptr< ::apache::thrift::protocol::TProtocol> iprot, boost::shared_ptr< ::apache::thrift::protocol::TProtocol> oprot) : + ::facebook::fb303::FacebookServiceClient(iprot, oprot) {} + boost::shared_ptr< ::apache::thrift::protocol::TProtocol> getInputProtocol() { + return piprot_; + } + boost::shared_ptr< ::apache::thrift::protocol::TProtocol> getOutputProtocol() { + return poprot_; + } + 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_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 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 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 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_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_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 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(); + 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 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_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_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_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); + 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_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); + 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); + 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(); @@ -13740,6 +15126,42 @@ class ThriftHiveMetastoreClient : virtual public ThriftHiveMetastoreIf, public 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(); + 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 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 compact(const CompactionRequest& rqst); + void send_compact(const CompactionRequest& rqst); + void recv_compact(); + void show_compact(ShowCompactResponse& _return, const ShowCompactRequest& rqst); + void send_show_compact(const ShowCompactRequest& rqst); + void recv_show_compact(ShowCompactResponse& _return); }; class ThriftHiveMetastoreProcessor : public ::facebook::fb303::FacebookServiceProcessor { @@ -13842,6 +15264,18 @@ class ThriftHiveMetastoreProcessor : public ::facebook::fb303::FacebookServiceP 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_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_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_compact(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); public: ThriftHiveMetastoreProcessor(boost::shared_ptr iface) : ::facebook::fb303::FacebookServiceProcessor(iface), @@ -13938,6 +15372,18 @@ class ThriftHiveMetastoreProcessor : public ::facebook::fb303::FacebookServiceP 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_["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_["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_["compact"] = &ThriftHiveMetastoreProcessor::process_compact; + processMap_["show_compact"] = &ThriftHiveMetastoreProcessor::process_show_compact; } virtual ~ThriftHiveMetastoreProcessor() {} @@ -14851,6 +16297,121 @@ class ThriftHiveMetastoreMultiface : virtual public ThriftHiveMetastoreIf, publi ifaces_[i]->cancel_delegation_token(token_str_form); } + 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 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 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 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; + } + }; }}} // namespace diff --git metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore_server.skeleton.cpp metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore_server.skeleton.cpp index 1d4b18b..569e3df 100644 --- metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore_server.skeleton.cpp +++ metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore_server.skeleton.cpp @@ -482,6 +482,66 @@ class ThriftHiveMetastoreHandler : virtual public ThriftHiveMetastoreIf { printf("cancel_delegation_token\n"); } + void get_open_txns(GetOpenTxnsResponse& _return) { + // Your implementation goes here + printf("get_open_txns\n"); + } + + void get_open_txns_info(GetOpenTxnsInfoResponse& _return) { + // Your implementation goes here + printf("get_open_txns_info\n"); + } + + void open_txns(OpenTxnsResponse& _return, const OpenTxnRequest& rqst) { + // Your implementation goes here + printf("open_txns\n"); + } + + void abort_txn(const AbortTxnRequest& rqst) { + // Your implementation goes here + printf("abort_txn\n"); + } + + void commit_txn(const CommitTxnRequest& rqst) { + // Your implementation goes here + printf("commit_txn\n"); + } + + void lock(LockResponse& _return, const LockRequest& rqst) { + // Your implementation goes here + printf("lock\n"); + } + + void check_lock(LockResponse& _return, const CheckLockRequest& rqst) { + // Your implementation goes here + printf("check_lock\n"); + } + + void unlock(const UnlockRequest& rqst) { + // Your implementation goes here + printf("unlock\n"); + } + + void show_locks(ShowLocksResponse& _return, const ShowLocksRequest& rqst) { + // Your implementation goes here + printf("show_locks\n"); + } + + void heartbeat(const HeartbeatRequest& ids) { + // Your implementation goes here + printf("heartbeat\n"); + } + + void compact(const CompactionRequest& rqst) { + // Your implementation goes here + printf("compact\n"); + } + + void show_compact(ShowCompactResponse& _return, const ShowCompactRequest& rqst) { + // Your implementation goes here + printf("show_compact\n"); + } + }; int main(int argc, char **argv) { diff --git metastore/src/gen/thrift/gen-cpp/hive_metastore_types.cpp metastore/src/gen/thrift/gen-cpp/hive_metastore_types.cpp index b00bd55..2c48a58 100644 --- metastore/src/gen/thrift/gen-cpp/hive_metastore_types.cpp +++ metastore/src/gen/thrift/gen-cpp/hive_metastore_types.cpp @@ -46,6 +46,66 @@ const char* _kPartitionEventTypeNames[] = { }; const std::map _PartitionEventType_VALUES_TO_NAMES(::apache::thrift::TEnumIterator(1, _kPartitionEventTypeValues, _kPartitionEventTypeNames), ::apache::thrift::TEnumIterator(-1, NULL, NULL)); +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)); + +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)); + +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)); + +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)); + +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)); + int _kFunctionTypeValues[] = { FunctionType::JAVA }; @@ -5452,10 +5512,10 @@ void swap(Function &a, Function &b) { swap(a.__isset, b.__isset); } -const char* MetaException::ascii_fingerprint = "EFB929595D312AC8F305D5A794CFEDA1"; -const uint8_t MetaException::binary_fingerprint[16] = {0xEF,0xB9,0x29,0x59,0x5D,0x31,0x2A,0xC8,0xF3,0x05,0xD5,0xA7,0x94,0xCF,0xED,0xA1}; +const char* TxnInfo::ascii_fingerprint = "6C5C0773A901CCA3BE9D085B3B47A767"; +const uint8_t TxnInfo::binary_fingerprint[16] = {0x6C,0x5C,0x07,0x73,0xA9,0x01,0xCC,0xA3,0xBE,0x9D,0x08,0x5B,0x3B,0x47,0xA7,0x67}; -uint32_t MetaException::read(::apache::thrift::protocol::TProtocol* iprot) { +uint32_t TxnInfo::read(::apache::thrift::protocol::TProtocol* iprot) { uint32_t xfer = 0; std::string fname; @@ -5466,6 +5526,10 @@ uint32_t MetaException::read(::apache::thrift::protocol::TProtocol* iprot) { using ::apache::thrift::protocol::TProtocolException; + bool isset_id = false; + bool isset_state = false; + bool isset_user = false; + bool isset_hostname = false; while (true) { @@ -5476,9 +5540,35 @@ uint32_t MetaException::read(::apache::thrift::protocol::TProtocol* iprot) { 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 ecast285; + xfer += iprot->readI32(ecast285); + this->state = (TxnState::type)ecast285; + isset_state = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->message); - this->__isset.message = true; + 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); } @@ -5492,15 +5582,35 @@ uint32_t MetaException::read(::apache::thrift::protocol::TProtocol* iprot) { 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 MetaException::write(::apache::thrift::protocol::TProtocol* oprot) const { +uint32_t TxnInfo::write(::apache::thrift::protocol::TProtocol* oprot) const { uint32_t xfer = 0; - xfer += oprot->writeStructBegin("MetaException"); + xfer += oprot->writeStructBegin("TxnInfo"); - xfer += oprot->writeFieldBegin("message", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->message); + 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(); xfer += oprot->writeFieldStop(); @@ -5508,16 +5618,18 @@ uint32_t MetaException::write(::apache::thrift::protocol::TProtocol* oprot) cons return xfer; } -void swap(MetaException &a, MetaException &b) { +void swap(TxnInfo &a, TxnInfo &b) { using ::std::swap; - swap(a.message, b.message); - swap(a.__isset, b.__isset); + swap(a.id, b.id); + swap(a.state, b.state); + swap(a.user, b.user); + swap(a.hostname, b.hostname); } -const char* UnknownTableException::ascii_fingerprint = "EFB929595D312AC8F305D5A794CFEDA1"; -const uint8_t UnknownTableException::binary_fingerprint[16] = {0xEF,0xB9,0x29,0x59,0x5D,0x31,0x2A,0xC8,0xF3,0x05,0xD5,0xA7,0x94,0xCF,0xED,0xA1}; +const char* GetOpenTxnsInfoResponse::ascii_fingerprint = "CCF769BBD33005B61F2079A6665E3B9C"; +const uint8_t GetOpenTxnsInfoResponse::binary_fingerprint[16] = {0xCC,0xF7,0x69,0xBB,0xD3,0x30,0x05,0xB6,0x1F,0x20,0x79,0xA6,0x66,0x5E,0x3B,0x9C}; -uint32_t UnknownTableException::read(::apache::thrift::protocol::TProtocol* iprot) { +uint32_t GetOpenTxnsInfoResponse::read(::apache::thrift::protocol::TProtocol* iprot) { uint32_t xfer = 0; std::string fname; @@ -5528,6 +5640,8 @@ uint32_t UnknownTableException::read(::apache::thrift::protocol::TProtocol* ipro using ::apache::thrift::protocol::TProtocolException; + bool isset_txn_high_water_mark = false; + bool isset_open_txns = false; while (true) { @@ -5538,9 +5652,29 @@ uint32_t UnknownTableException::read(::apache::thrift::protocol::TProtocol* ipro switch (fid) { case 1: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->message); - this->__isset.message = true; + 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 _size286; + ::apache::thrift::protocol::TType _etype289; + xfer += iprot->readListBegin(_etype289, _size286); + this->open_txns.resize(_size286); + uint32_t _i290; + for (_i290 = 0; _i290 < _size286; ++_i290) + { + xfer += this->open_txns[_i290].read(iprot); + } + xfer += iprot->readListEnd(); + } + isset_open_txns = true; } else { xfer += iprot->skip(ftype); } @@ -5554,15 +5688,31 @@ uint32_t UnknownTableException::read(::apache::thrift::protocol::TProtocol* ipro 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 UnknownTableException::write(::apache::thrift::protocol::TProtocol* oprot) const { +uint32_t GetOpenTxnsInfoResponse::write(::apache::thrift::protocol::TProtocol* oprot) const { uint32_t xfer = 0; - xfer += oprot->writeStructBegin("UnknownTableException"); + xfer += oprot->writeStructBegin("GetOpenTxnsInfoResponse"); - xfer += oprot->writeFieldBegin("message", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->message); + 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 _iter291; + for (_iter291 = this->open_txns.begin(); _iter291 != this->open_txns.end(); ++_iter291) + { + xfer += (*_iter291).write(oprot); + } + xfer += oprot->writeListEnd(); + } xfer += oprot->writeFieldEnd(); xfer += oprot->writeFieldStop(); @@ -5570,16 +5720,16 @@ uint32_t UnknownTableException::write(::apache::thrift::protocol::TProtocol* opr return xfer; } -void swap(UnknownTableException &a, UnknownTableException &b) { +void swap(GetOpenTxnsInfoResponse &a, GetOpenTxnsInfoResponse &b) { using ::std::swap; - swap(a.message, b.message); - swap(a.__isset, b.__isset); + swap(a.txn_high_water_mark, b.txn_high_water_mark); + swap(a.open_txns, b.open_txns); } -const char* UnknownDBException::ascii_fingerprint = "EFB929595D312AC8F305D5A794CFEDA1"; -const uint8_t UnknownDBException::binary_fingerprint[16] = {0xEF,0xB9,0x29,0x59,0x5D,0x31,0x2A,0xC8,0xF3,0x05,0xD5,0xA7,0x94,0xCF,0xED,0xA1}; +const char* GetOpenTxnsResponse::ascii_fingerprint = "590531FF1BE8611678B255374F6109EE"; +const uint8_t GetOpenTxnsResponse::binary_fingerprint[16] = {0x59,0x05,0x31,0xFF,0x1B,0xE8,0x61,0x16,0x78,0xB2,0x55,0x37,0x4F,0x61,0x09,0xEE}; -uint32_t UnknownDBException::read(::apache::thrift::protocol::TProtocol* iprot) { +uint32_t GetOpenTxnsResponse::read(::apache::thrift::protocol::TProtocol* iprot) { uint32_t xfer = 0; std::string fname; @@ -5590,6 +5740,8 @@ uint32_t UnknownDBException::read(::apache::thrift::protocol::TProtocol* iprot) using ::apache::thrift::protocol::TProtocolException; + bool isset_txn_high_water_mark = false; + bool isset_open_txns = false; while (true) { @@ -5600,9 +5752,30 @@ uint32_t UnknownDBException::read(::apache::thrift::protocol::TProtocol* iprot) switch (fid) { case 1: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->message); - this->__isset.message = true; + 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_SET) { + { + this->open_txns.clear(); + uint32_t _size292; + ::apache::thrift::protocol::TType _etype295; + xfer += iprot->readSetBegin(_etype295, _size292); + uint32_t _i296; + for (_i296 = 0; _i296 < _size292; ++_i296) + { + int64_t _elem297; + xfer += iprot->readI64(_elem297); + this->open_txns.insert(_elem297); + } + xfer += iprot->readSetEnd(); + } + isset_open_txns = true; } else { xfer += iprot->skip(ftype); } @@ -5616,15 +5789,31 @@ uint32_t UnknownDBException::read(::apache::thrift::protocol::TProtocol* iprot) 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 UnknownDBException::write(::apache::thrift::protocol::TProtocol* oprot) const { +uint32_t GetOpenTxnsResponse::write(::apache::thrift::protocol::TProtocol* oprot) const { uint32_t xfer = 0; - xfer += oprot->writeStructBegin("UnknownDBException"); + xfer += oprot->writeStructBegin("GetOpenTxnsResponse"); - xfer += oprot->writeFieldBegin("message", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->message); + 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_SET, 2); + { + xfer += oprot->writeSetBegin(::apache::thrift::protocol::T_I64, static_cast(this->open_txns.size())); + std::set ::const_iterator _iter298; + for (_iter298 = this->open_txns.begin(); _iter298 != this->open_txns.end(); ++_iter298) + { + xfer += oprot->writeI64((*_iter298)); + } + xfer += oprot->writeSetEnd(); + } xfer += oprot->writeFieldEnd(); xfer += oprot->writeFieldStop(); @@ -5632,16 +5821,16 @@ uint32_t UnknownDBException::write(::apache::thrift::protocol::TProtocol* oprot) return xfer; } -void swap(UnknownDBException &a, UnknownDBException &b) { +void swap(GetOpenTxnsResponse &a, GetOpenTxnsResponse &b) { using ::std::swap; - swap(a.message, b.message); - swap(a.__isset, b.__isset); + swap(a.txn_high_water_mark, b.txn_high_water_mark); + swap(a.open_txns, b.open_txns); } -const char* AlreadyExistsException::ascii_fingerprint = "EFB929595D312AC8F305D5A794CFEDA1"; -const uint8_t AlreadyExistsException::binary_fingerprint[16] = {0xEF,0xB9,0x29,0x59,0x5D,0x31,0x2A,0xC8,0xF3,0x05,0xD5,0xA7,0x94,0xCF,0xED,0xA1}; +const char* OpenTxnRequest::ascii_fingerprint = "3368C2F81F2FEF71F11EDACDB2A3ECEF"; +const uint8_t OpenTxnRequest::binary_fingerprint[16] = {0x33,0x68,0xC2,0xF8,0x1F,0x2F,0xEF,0x71,0xF1,0x1E,0xDA,0xCD,0xB2,0xA3,0xEC,0xEF}; -uint32_t AlreadyExistsException::read(::apache::thrift::protocol::TProtocol* iprot) { +uint32_t OpenTxnRequest::read(::apache::thrift::protocol::TProtocol* iprot) { uint32_t xfer = 0; std::string fname; @@ -5652,6 +5841,9 @@ uint32_t AlreadyExistsException::read(::apache::thrift::protocol::TProtocol* ipr using ::apache::thrift::protocol::TProtocolException; + bool isset_num_txns = false; + bool isset_user = false; + bool isset_hostname = false; while (true) { @@ -5662,9 +5854,25 @@ uint32_t AlreadyExistsException::read(::apache::thrift::protocol::TProtocol* ipr 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->message); - this->__isset.message = true; + 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); } @@ -5678,15 +5886,29 @@ uint32_t AlreadyExistsException::read(::apache::thrift::protocol::TProtocol* ipr 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 AlreadyExistsException::write(::apache::thrift::protocol::TProtocol* oprot) const { +uint32_t OpenTxnRequest::write(::apache::thrift::protocol::TProtocol* oprot) const { uint32_t xfer = 0; - xfer += oprot->writeStructBegin("AlreadyExistsException"); + xfer += oprot->writeStructBegin("OpenTxnRequest"); - xfer += oprot->writeFieldBegin("message", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->message); + 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(); xfer += oprot->writeFieldStop(); @@ -5694,16 +5916,17 @@ uint32_t AlreadyExistsException::write(::apache::thrift::protocol::TProtocol* op return xfer; } -void swap(AlreadyExistsException &a, AlreadyExistsException &b) { +void swap(OpenTxnRequest &a, OpenTxnRequest &b) { using ::std::swap; - swap(a.message, b.message); - swap(a.__isset, b.__isset); + swap(a.num_txns, b.num_txns); + swap(a.user, b.user); + swap(a.hostname, b.hostname); } -const char* InvalidPartitionException::ascii_fingerprint = "EFB929595D312AC8F305D5A794CFEDA1"; -const uint8_t InvalidPartitionException::binary_fingerprint[16] = {0xEF,0xB9,0x29,0x59,0x5D,0x31,0x2A,0xC8,0xF3,0x05,0xD5,0xA7,0x94,0xCF,0xED,0xA1}; +const char* OpenTxnsResponse::ascii_fingerprint = "E49D7D1A9013CC81CD0F69D631EF82E4"; +const uint8_t OpenTxnsResponse::binary_fingerprint[16] = {0xE4,0x9D,0x7D,0x1A,0x90,0x13,0xCC,0x81,0xCD,0x0F,0x69,0xD6,0x31,0xEF,0x82,0xE4}; -uint32_t InvalidPartitionException::read(::apache::thrift::protocol::TProtocol* iprot) { +uint32_t OpenTxnsResponse::read(::apache::thrift::protocol::TProtocol* iprot) { uint32_t xfer = 0; std::string fname; @@ -5714,6 +5937,7 @@ uint32_t InvalidPartitionException::read(::apache::thrift::protocol::TProtocol* using ::apache::thrift::protocol::TProtocolException; + bool isset_txn_ids = false; while (true) { @@ -5724,9 +5948,21 @@ uint32_t InvalidPartitionException::read(::apache::thrift::protocol::TProtocol* switch (fid) { case 1: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->message); - this->__isset.message = true; + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + this->txn_ids.clear(); + uint32_t _size299; + ::apache::thrift::protocol::TType _etype302; + xfer += iprot->readListBegin(_etype302, _size299); + this->txn_ids.resize(_size299); + uint32_t _i303; + for (_i303 = 0; _i303 < _size299; ++_i303) + { + xfer += iprot->readI64(this->txn_ids[_i303]); + } + xfer += iprot->readListEnd(); + } + isset_txn_ids = true; } else { xfer += iprot->skip(ftype); } @@ -5740,15 +5976,25 @@ uint32_t InvalidPartitionException::read(::apache::thrift::protocol::TProtocol* xfer += iprot->readStructEnd(); + if (!isset_txn_ids) + throw TProtocolException(TProtocolException::INVALID_DATA); return xfer; } -uint32_t InvalidPartitionException::write(::apache::thrift::protocol::TProtocol* oprot) const { +uint32_t OpenTxnsResponse::write(::apache::thrift::protocol::TProtocol* oprot) const { uint32_t xfer = 0; - xfer += oprot->writeStructBegin("InvalidPartitionException"); + xfer += oprot->writeStructBegin("OpenTxnsResponse"); - xfer += oprot->writeFieldBegin("message", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->message); + 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 _iter304; + for (_iter304 = this->txn_ids.begin(); _iter304 != this->txn_ids.end(); ++_iter304) + { + xfer += oprot->writeI64((*_iter304)); + } + xfer += oprot->writeListEnd(); + } xfer += oprot->writeFieldEnd(); xfer += oprot->writeFieldStop(); @@ -5756,16 +6002,15 @@ uint32_t InvalidPartitionException::write(::apache::thrift::protocol::TProtocol* return xfer; } -void swap(InvalidPartitionException &a, InvalidPartitionException &b) { +void swap(OpenTxnsResponse &a, OpenTxnsResponse &b) { using ::std::swap; - swap(a.message, b.message); - swap(a.__isset, b.__isset); + swap(a.txn_ids, b.txn_ids); } -const char* UnknownPartitionException::ascii_fingerprint = "EFB929595D312AC8F305D5A794CFEDA1"; -const uint8_t UnknownPartitionException::binary_fingerprint[16] = {0xEF,0xB9,0x29,0x59,0x5D,0x31,0x2A,0xC8,0xF3,0x05,0xD5,0xA7,0x94,0xCF,0xED,0xA1}; +const char* AbortTxnRequest::ascii_fingerprint = "56A59CE7FFAF82BCA8A19FAACDE4FB75"; +const uint8_t AbortTxnRequest::binary_fingerprint[16] = {0x56,0xA5,0x9C,0xE7,0xFF,0xAF,0x82,0xBC,0xA8,0xA1,0x9F,0xAA,0xCD,0xE4,0xFB,0x75}; -uint32_t UnknownPartitionException::read(::apache::thrift::protocol::TProtocol* iprot) { +uint32_t AbortTxnRequest::read(::apache::thrift::protocol::TProtocol* iprot) { uint32_t xfer = 0; std::string fname; @@ -5776,6 +6021,7 @@ uint32_t UnknownPartitionException::read(::apache::thrift::protocol::TProtocol* using ::apache::thrift::protocol::TProtocolException; + bool isset_txnid = false; while (true) { @@ -5786,9 +6032,9 @@ uint32_t UnknownPartitionException::read(::apache::thrift::protocol::TProtocol* switch (fid) { case 1: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->message); - this->__isset.message = true; + if (ftype == ::apache::thrift::protocol::T_I64) { + xfer += iprot->readI64(this->txnid); + isset_txnid = true; } else { xfer += iprot->skip(ftype); } @@ -5802,15 +6048,17 @@ uint32_t UnknownPartitionException::read(::apache::thrift::protocol::TProtocol* xfer += iprot->readStructEnd(); + if (!isset_txnid) + throw TProtocolException(TProtocolException::INVALID_DATA); return xfer; } -uint32_t UnknownPartitionException::write(::apache::thrift::protocol::TProtocol* oprot) const { +uint32_t AbortTxnRequest::write(::apache::thrift::protocol::TProtocol* oprot) const { uint32_t xfer = 0; - xfer += oprot->writeStructBegin("UnknownPartitionException"); + xfer += oprot->writeStructBegin("AbortTxnRequest"); - xfer += oprot->writeFieldBegin("message", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->message); + xfer += oprot->writeFieldBegin("txnid", ::apache::thrift::protocol::T_I64, 1); + xfer += oprot->writeI64(this->txnid); xfer += oprot->writeFieldEnd(); xfer += oprot->writeFieldStop(); @@ -5818,16 +6066,15 @@ uint32_t UnknownPartitionException::write(::apache::thrift::protocol::TProtocol* return xfer; } -void swap(UnknownPartitionException &a, UnknownPartitionException &b) { +void swap(AbortTxnRequest &a, AbortTxnRequest &b) { using ::std::swap; - swap(a.message, b.message); - swap(a.__isset, b.__isset); + swap(a.txnid, b.txnid); } -const char* InvalidObjectException::ascii_fingerprint = "EFB929595D312AC8F305D5A794CFEDA1"; -const uint8_t InvalidObjectException::binary_fingerprint[16] = {0xEF,0xB9,0x29,0x59,0x5D,0x31,0x2A,0xC8,0xF3,0x05,0xD5,0xA7,0x94,0xCF,0xED,0xA1}; +const char* CommitTxnRequest::ascii_fingerprint = "56A59CE7FFAF82BCA8A19FAACDE4FB75"; +const uint8_t CommitTxnRequest::binary_fingerprint[16] = {0x56,0xA5,0x9C,0xE7,0xFF,0xAF,0x82,0xBC,0xA8,0xA1,0x9F,0xAA,0xCD,0xE4,0xFB,0x75}; -uint32_t InvalidObjectException::read(::apache::thrift::protocol::TProtocol* iprot) { +uint32_t CommitTxnRequest::read(::apache::thrift::protocol::TProtocol* iprot) { uint32_t xfer = 0; std::string fname; @@ -5838,6 +6085,7 @@ uint32_t InvalidObjectException::read(::apache::thrift::protocol::TProtocol* ipr using ::apache::thrift::protocol::TProtocolException; + bool isset_txnid = false; while (true) { @@ -5848,9 +6096,9 @@ uint32_t InvalidObjectException::read(::apache::thrift::protocol::TProtocol* ipr switch (fid) { case 1: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->message); - this->__isset.message = true; + if (ftype == ::apache::thrift::protocol::T_I64) { + xfer += iprot->readI64(this->txnid); + isset_txnid = true; } else { xfer += iprot->skip(ftype); } @@ -5864,15 +6112,17 @@ uint32_t InvalidObjectException::read(::apache::thrift::protocol::TProtocol* ipr xfer += iprot->readStructEnd(); + if (!isset_txnid) + throw TProtocolException(TProtocolException::INVALID_DATA); return xfer; } -uint32_t InvalidObjectException::write(::apache::thrift::protocol::TProtocol* oprot) const { +uint32_t CommitTxnRequest::write(::apache::thrift::protocol::TProtocol* oprot) const { uint32_t xfer = 0; - xfer += oprot->writeStructBegin("InvalidObjectException"); + xfer += oprot->writeStructBegin("CommitTxnRequest"); - xfer += oprot->writeFieldBegin("message", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->message); + xfer += oprot->writeFieldBegin("txnid", ::apache::thrift::protocol::T_I64, 1); + xfer += oprot->writeI64(this->txnid); xfer += oprot->writeFieldEnd(); xfer += oprot->writeFieldStop(); @@ -5880,16 +6130,15 @@ uint32_t InvalidObjectException::write(::apache::thrift::protocol::TProtocol* op return xfer; } -void swap(InvalidObjectException &a, InvalidObjectException &b) { +void swap(CommitTxnRequest &a, CommitTxnRequest &b) { using ::std::swap; - swap(a.message, b.message); - swap(a.__isset, b.__isset); + swap(a.txnid, b.txnid); } -const char* NoSuchObjectException::ascii_fingerprint = "EFB929595D312AC8F305D5A794CFEDA1"; -const uint8_t NoSuchObjectException::binary_fingerprint[16] = {0xEF,0xB9,0x29,0x59,0x5D,0x31,0x2A,0xC8,0xF3,0x05,0xD5,0xA7,0x94,0xCF,0xED,0xA1}; +const char* LockComponent::ascii_fingerprint = "38B02531B0840AC9C72904A4649FD15F"; +const uint8_t LockComponent::binary_fingerprint[16] = {0x38,0xB0,0x25,0x31,0xB0,0x84,0x0A,0xC9,0xC7,0x29,0x04,0xA4,0x64,0x9F,0xD1,0x5F}; -uint32_t NoSuchObjectException::read(::apache::thrift::protocol::TProtocol* iprot) { +uint32_t LockComponent::read(::apache::thrift::protocol::TProtocol* iprot) { uint32_t xfer = 0; std::string fname; @@ -5900,6 +6149,9 @@ uint32_t NoSuchObjectException::read(::apache::thrift::protocol::TProtocol* ipro using ::apache::thrift::protocol::TProtocolException; + bool isset_type = false; + bool isset_level = false; + bool isset_dbname = false; while (true) { @@ -5910,9 +6162,45 @@ uint32_t NoSuchObjectException::read(::apache::thrift::protocol::TProtocol* ipro switch (fid) { case 1: + if (ftype == ::apache::thrift::protocol::T_I32) { + int32_t ecast305; + xfer += iprot->readI32(ecast305); + this->type = (LockType::type)ecast305; + isset_type = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 2: + if (ftype == ::apache::thrift::protocol::T_I32) { + int32_t ecast306; + xfer += iprot->readI32(ecast306); + this->level = (LockLevel::type)ecast306; + isset_level = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 3: if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->message); - this->__isset.message = true; + 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); } @@ -5926,30 +6214,1754 @@ uint32_t NoSuchObjectException::read(::apache::thrift::protocol::TProtocol* ipro 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 NoSuchObjectException::write(::apache::thrift::protocol::TProtocol* oprot) const { +uint32_t LockComponent::write(::apache::thrift::protocol::TProtocol* oprot) const { uint32_t xfer = 0; - xfer += oprot->writeStructBegin("NoSuchObjectException"); + xfer += oprot->writeStructBegin("LockComponent"); - xfer += oprot->writeFieldBegin("message", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->message); + 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(); + } xfer += oprot->writeFieldStop(); xfer += oprot->writeStructEnd(); return xfer; } -void swap(NoSuchObjectException &a, NoSuchObjectException &b) { +void swap(LockComponent &a, LockComponent &b) { using ::std::swap; - swap(a.message, b.message); + 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.__isset, b.__isset); } -const char* IndexAlreadyExistsException::ascii_fingerprint = "EFB929595D312AC8F305D5A794CFEDA1"; -const uint8_t IndexAlreadyExistsException::binary_fingerprint[16] = {0xEF,0xB9,0x29,0x59,0x5D,0x31,0x2A,0xC8,0xF3,0x05,0xD5,0xA7,0x94,0xCF,0xED,0xA1}; +const char* LockRequest::ascii_fingerprint = "46BC5ED7196BC16CB216AD5CC67C6930"; +const uint8_t LockRequest::binary_fingerprint[16] = {0x46,0xBC,0x5E,0xD7,0x19,0x6B,0xC1,0x6C,0xB2,0x16,0xAD,0x5C,0xC6,0x7C,0x69,0x30}; + +uint32_t LockRequest::read(::apache::thrift::protocol::TProtocol* iprot) { + + 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 _size307; + ::apache::thrift::protocol::TType _etype310; + xfer += iprot->readListBegin(_etype310, _size307); + this->component.resize(_size307); + uint32_t _i311; + for (_i311 = 0; _i311 < _size307; ++_i311) + { + xfer += this->component[_i311].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; + 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; + 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 _iter312; + for (_iter312 = this->component.begin(); _iter312 != this->component.end(); ++_iter312) + { + xfer += (*_iter312).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(); + + 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.__isset, b.__isset); +} + +const char* LockResponse::ascii_fingerprint = "DFA40D9D2884599F3D1E7A57578F1384"; +const uint8_t LockResponse::binary_fingerprint[16] = {0xDF,0xA4,0x0D,0x9D,0x28,0x84,0x59,0x9F,0x3D,0x1E,0x7A,0x57,0x57,0x8F,0x13,0x84}; + +uint32_t LockResponse::read(::apache::thrift::protocol::TProtocol* iprot) { + + 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 ecast313; + xfer += iprot->readI32(ecast313); + this->state = (LockState::type)ecast313; + 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; + 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); +} + +const char* CheckLockRequest::ascii_fingerprint = "56A59CE7FFAF82BCA8A19FAACDE4FB75"; +const uint8_t CheckLockRequest::binary_fingerprint[16] = {0x56,0xA5,0x9C,0xE7,0xFF,0xAF,0x82,0xBC,0xA8,0xA1,0x9F,0xAA,0xCD,0xE4,0xFB,0x75}; + +uint32_t CheckLockRequest::read(::apache::thrift::protocol::TProtocol* iprot) { + + 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 CheckLockRequest::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + xfer += oprot->writeStructBegin("CheckLockRequest"); + + 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(CheckLockRequest &a, CheckLockRequest &b) { + using ::std::swap; + swap(a.lockid, b.lockid); +} + +const char* UnlockRequest::ascii_fingerprint = "56A59CE7FFAF82BCA8A19FAACDE4FB75"; +const uint8_t UnlockRequest::binary_fingerprint[16] = {0x56,0xA5,0x9C,0xE7,0xFF,0xAF,0x82,0xBC,0xA8,0xA1,0x9F,0xAA,0xCD,0xE4,0xFB,0x75}; + +uint32_t UnlockRequest::read(::apache::thrift::protocol::TProtocol* iprot) { + + 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; + 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); +} + +const char* ShowLocksRequest::ascii_fingerprint = "99914B932BD37A50B983C5E7C90AE93B"; +const uint8_t ShowLocksRequest::binary_fingerprint[16] = {0x99,0x91,0x4B,0x93,0x2B,0xD3,0x7A,0x50,0xB9,0x83,0xC5,0xE7,0xC9,0x0A,0xE9,0x3B}; + +uint32_t ShowLocksRequest::read(::apache::thrift::protocol::TProtocol* iprot) { + + uint32_t xfer = 0; + std::string fname; + ::apache::thrift::protocol::TType ftype; + int16_t fid; + + xfer += 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 ShowLocksRequest::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + xfer += oprot->writeStructBegin("ShowLocksRequest"); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(ShowLocksRequest &a, ShowLocksRequest &b) { + using ::std::swap; + (void) a; + (void) b; +} + +const char* ShowLocksResponseElement::ascii_fingerprint = "5AD11F0E0EF1EE0A7C08B00FEFCFF24F"; +const uint8_t ShowLocksResponseElement::binary_fingerprint[16] = {0x5A,0xD1,0x1F,0x0E,0x0E,0xF1,0xEE,0x0A,0x7C,0x08,0xB0,0x0F,0xEF,0xCF,0xF2,0x4F}; + +uint32_t ShowLocksResponseElement::read(::apache::thrift::protocol::TProtocol* iprot) { + + 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 ecast314; + xfer += iprot->readI32(ecast314); + this->state = (LockState::type)ecast314; + isset_state = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 6: + if (ftype == ::apache::thrift::protocol::T_I32) { + int32_t ecast315; + xfer += iprot->readI32(ecast315); + this->type = (LockType::type)ecast315; + 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; + 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; + 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(); + + 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.__isset, b.__isset); +} + +const char* ShowLocksResponse::ascii_fingerprint = "BD598AA60FE941361FB54C43973C011F"; +const uint8_t ShowLocksResponse::binary_fingerprint[16] = {0xBD,0x59,0x8A,0xA6,0x0F,0xE9,0x41,0x36,0x1F,0xB5,0x4C,0x43,0x97,0x3C,0x01,0x1F}; + +uint32_t ShowLocksResponse::read(::apache::thrift::protocol::TProtocol* iprot) { + + uint32_t xfer = 0; + std::string fname; + ::apache::thrift::protocol::TType ftype; + int16_t fid; + + xfer += iprot->readStructBegin(fname); + + using ::apache::thrift::protocol::TProtocolException; + + + while (true) + { + 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 _size316; + ::apache::thrift::protocol::TType _etype319; + xfer += iprot->readListBegin(_etype319, _size316); + this->locks.resize(_size316); + uint32_t _i320; + for (_i320 = 0; _i320 < _size316; ++_i320) + { + xfer += this->locks[_i320].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; + 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 _iter321; + for (_iter321 = this->locks.begin(); _iter321 != this->locks.end(); ++_iter321) + { + xfer += (*_iter321).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); +} + +const char* HeartbeatRequest::ascii_fingerprint = "0354D07C94CB8542872CA1277008860A"; +const uint8_t HeartbeatRequest::binary_fingerprint[16] = {0x03,0x54,0xD0,0x7C,0x94,0xCB,0x85,0x42,0x87,0x2C,0xA1,0x27,0x70,0x08,0x86,0x0A}; + +uint32_t HeartbeatRequest::read(::apache::thrift::protocol::TProtocol* iprot) { + + uint32_t xfer = 0; + std::string fname; + ::apache::thrift::protocol::TType ftype; + int16_t fid; + + xfer += iprot->readStructBegin(fname); + + using ::apache::thrift::protocol::TProtocolException; + + + while (true) + { + 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; + 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); +} + +const char* CompactionRequest::ascii_fingerprint = "899FD1F339D8318D628687CC2CE2864B"; +const uint8_t CompactionRequest::binary_fingerprint[16] = {0x89,0x9F,0xD1,0xF3,0x39,0xD8,0x31,0x8D,0x62,0x86,0x87,0xCC,0x2C,0xE2,0x86,0x4B}; + +uint32_t CompactionRequest::read(::apache::thrift::protocol::TProtocol* iprot) { + + 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 ecast322; + xfer += iprot->readI32(ecast322); + this->type = (CompactionType::type)ecast322; + 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; + 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; + 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(); + } + 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.__isset, b.__isset); +} + +const char* ShowCompactRequest::ascii_fingerprint = "99914B932BD37A50B983C5E7C90AE93B"; +const uint8_t ShowCompactRequest::binary_fingerprint[16] = {0x99,0x91,0x4B,0x93,0x2B,0xD3,0x7A,0x50,0xB9,0x83,0xC5,0xE7,0xC9,0x0A,0xE9,0x3B}; + +uint32_t ShowCompactRequest::read(::apache::thrift::protocol::TProtocol* iprot) { + + uint32_t xfer = 0; + std::string fname; + ::apache::thrift::protocol::TType ftype; + int16_t fid; + + xfer += 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; + 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; +} + +const char* ShowCompactResponseElement::ascii_fingerprint = "42464F3A97707F984FDE462104223A69"; +const uint8_t ShowCompactResponseElement::binary_fingerprint[16] = {0x42,0x46,0x4F,0x3A,0x97,0x70,0x7F,0x98,0x4F,0xDE,0x46,0x21,0x04,0x22,0x3A,0x69}; + +uint32_t ShowCompactResponseElement::read(::apache::thrift::protocol::TProtocol* iprot) { + + 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_partitionname = false; + bool isset_type = false; + bool isset_state = false; + bool isset_workerid = false; + bool isset_start = false; + bool isset_runAs = 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); + isset_partitionname = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 4: + if (ftype == ::apache::thrift::protocol::T_I32) { + int32_t ecast323; + xfer += iprot->readI32(ecast323); + this->type = (CompactionType::type)ecast323; + 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); + isset_workerid = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 7: + if (ftype == ::apache::thrift::protocol::T_I64) { + xfer += iprot->readI64(this->start); + isset_start = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 8: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->runAs); + isset_runAs = 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_partitionname) + throw TProtocolException(TProtocolException::INVALID_DATA); + if (!isset_type) + throw TProtocolException(TProtocolException::INVALID_DATA); + if (!isset_state) + throw TProtocolException(TProtocolException::INVALID_DATA); + if (!isset_workerid) + throw TProtocolException(TProtocolException::INVALID_DATA); + if (!isset_start) + throw TProtocolException(TProtocolException::INVALID_DATA); + if (!isset_runAs) + throw TProtocolException(TProtocolException::INVALID_DATA); + return xfer; +} + +uint32_t ShowCompactResponseElement::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + 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(); + + 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(); + + xfer += oprot->writeFieldBegin("workerid", ::apache::thrift::protocol::T_STRING, 6); + xfer += oprot->writeString(this->workerid); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("start", ::apache::thrift::protocol::T_I64, 7); + xfer += oprot->writeI64(this->start); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("runAs", ::apache::thrift::protocol::T_STRING, 8); + xfer += oprot->writeString(this->runAs); + 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); +} + +const char* ShowCompactResponse::ascii_fingerprint = "3CCBC4D398CA25527272FE78625DE88A"; +const uint8_t ShowCompactResponse::binary_fingerprint[16] = {0x3C,0xCB,0xC4,0xD3,0x98,0xCA,0x25,0x52,0x72,0x72,0xFE,0x78,0x62,0x5D,0xE8,0x8A}; + +uint32_t ShowCompactResponse::read(::apache::thrift::protocol::TProtocol* iprot) { + + 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 _size324; + ::apache::thrift::protocol::TType _etype327; + xfer += iprot->readListBegin(_etype327, _size324); + this->compacts.resize(_size324); + uint32_t _i328; + for (_i328 = 0; _i328 < _size324; ++_i328) + { + xfer += this->compacts[_i328].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; + 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 _iter329; + for (_iter329 = this->compacts.begin(); _iter329 != this->compacts.end(); ++_iter329) + { + xfer += (*_iter329).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); +} + +const char* MetaException::ascii_fingerprint = "EFB929595D312AC8F305D5A794CFEDA1"; +const uint8_t MetaException::binary_fingerprint[16] = {0xEF,0xB9,0x29,0x59,0x5D,0x31,0x2A,0xC8,0xF3,0x05,0xD5,0xA7,0x94,0xCF,0xED,0xA1}; + +uint32_t MetaException::read(::apache::thrift::protocol::TProtocol* iprot) { + + uint32_t xfer = 0; + std::string fname; + ::apache::thrift::protocol::TType ftype; + int16_t fid; + + xfer += iprot->readStructBegin(fname); + + using ::apache::thrift::protocol::TProtocolException; + + + while (true) + { + xfer += iprot->readFieldBegin(fname, ftype, 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; + 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); +} + +const char* UnknownTableException::ascii_fingerprint = "EFB929595D312AC8F305D5A794CFEDA1"; +const uint8_t UnknownTableException::binary_fingerprint[16] = {0xEF,0xB9,0x29,0x59,0x5D,0x31,0x2A,0xC8,0xF3,0x05,0xD5,0xA7,0x94,0xCF,0xED,0xA1}; + +uint32_t UnknownTableException::read(::apache::thrift::protocol::TProtocol* iprot) { + + uint32_t xfer = 0; + std::string fname; + ::apache::thrift::protocol::TType ftype; + int16_t fid; + + xfer += iprot->readStructBegin(fname); + + using ::apache::thrift::protocol::TProtocolException; + + + while (true) + { + xfer += iprot->readFieldBegin(fname, ftype, 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; + 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); +} + +const char* UnknownDBException::ascii_fingerprint = "EFB929595D312AC8F305D5A794CFEDA1"; +const uint8_t UnknownDBException::binary_fingerprint[16] = {0xEF,0xB9,0x29,0x59,0x5D,0x31,0x2A,0xC8,0xF3,0x05,0xD5,0xA7,0x94,0xCF,0xED,0xA1}; + +uint32_t UnknownDBException::read(::apache::thrift::protocol::TProtocol* iprot) { + + uint32_t xfer = 0; + std::string fname; + ::apache::thrift::protocol::TType ftype; + int16_t fid; + + xfer += iprot->readStructBegin(fname); + + using ::apache::thrift::protocol::TProtocolException; + + + while (true) + { + xfer += iprot->readFieldBegin(fname, ftype, 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; + 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); +} + +const char* AlreadyExistsException::ascii_fingerprint = "EFB929595D312AC8F305D5A794CFEDA1"; +const uint8_t AlreadyExistsException::binary_fingerprint[16] = {0xEF,0xB9,0x29,0x59,0x5D,0x31,0x2A,0xC8,0xF3,0x05,0xD5,0xA7,0x94,0xCF,0xED,0xA1}; + +uint32_t AlreadyExistsException::read(::apache::thrift::protocol::TProtocol* iprot) { + + uint32_t xfer = 0; + std::string fname; + ::apache::thrift::protocol::TType ftype; + int16_t fid; + + xfer += iprot->readStructBegin(fname); + + using ::apache::thrift::protocol::TProtocolException; + + + while (true) + { + xfer += iprot->readFieldBegin(fname, ftype, 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; + 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); +} + +const char* InvalidPartitionException::ascii_fingerprint = "EFB929595D312AC8F305D5A794CFEDA1"; +const uint8_t InvalidPartitionException::binary_fingerprint[16] = {0xEF,0xB9,0x29,0x59,0x5D,0x31,0x2A,0xC8,0xF3,0x05,0xD5,0xA7,0x94,0xCF,0xED,0xA1}; + +uint32_t InvalidPartitionException::read(::apache::thrift::protocol::TProtocol* iprot) { + + uint32_t xfer = 0; + std::string fname; + ::apache::thrift::protocol::TType ftype; + int16_t fid; + + xfer += iprot->readStructBegin(fname); + + using ::apache::thrift::protocol::TProtocolException; + + + while (true) + { + xfer += iprot->readFieldBegin(fname, ftype, 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; + 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); +} + +const char* UnknownPartitionException::ascii_fingerprint = "EFB929595D312AC8F305D5A794CFEDA1"; +const uint8_t UnknownPartitionException::binary_fingerprint[16] = {0xEF,0xB9,0x29,0x59,0x5D,0x31,0x2A,0xC8,0xF3,0x05,0xD5,0xA7,0x94,0xCF,0xED,0xA1}; + +uint32_t UnknownPartitionException::read(::apache::thrift::protocol::TProtocol* iprot) { + + uint32_t xfer = 0; + std::string fname; + ::apache::thrift::protocol::TType ftype; + int16_t fid; + + xfer += iprot->readStructBegin(fname); + + using ::apache::thrift::protocol::TProtocolException; + + + while (true) + { + xfer += iprot->readFieldBegin(fname, ftype, 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; + 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); +} + +const char* InvalidObjectException::ascii_fingerprint = "EFB929595D312AC8F305D5A794CFEDA1"; +const uint8_t InvalidObjectException::binary_fingerprint[16] = {0xEF,0xB9,0x29,0x59,0x5D,0x31,0x2A,0xC8,0xF3,0x05,0xD5,0xA7,0x94,0xCF,0xED,0xA1}; + +uint32_t InvalidObjectException::read(::apache::thrift::protocol::TProtocol* iprot) { + + uint32_t xfer = 0; + std::string fname; + ::apache::thrift::protocol::TType ftype; + int16_t fid; + + xfer += iprot->readStructBegin(fname); + + using ::apache::thrift::protocol::TProtocolException; + + + while (true) + { + xfer += iprot->readFieldBegin(fname, ftype, 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; + 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); +} + +const char* NoSuchObjectException::ascii_fingerprint = "EFB929595D312AC8F305D5A794CFEDA1"; +const uint8_t NoSuchObjectException::binary_fingerprint[16] = {0xEF,0xB9,0x29,0x59,0x5D,0x31,0x2A,0xC8,0xF3,0x05,0xD5,0xA7,0x94,0xCF,0xED,0xA1}; + +uint32_t NoSuchObjectException::read(::apache::thrift::protocol::TProtocol* iprot) { + + uint32_t xfer = 0; + std::string fname; + ::apache::thrift::protocol::TType ftype; + int16_t fid; + + xfer += iprot->readStructBegin(fname); + + using ::apache::thrift::protocol::TProtocolException; + + + while (true) + { + xfer += iprot->readFieldBegin(fname, ftype, 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; + 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); +} + +const char* IndexAlreadyExistsException::ascii_fingerprint = "EFB929595D312AC8F305D5A794CFEDA1"; +const uint8_t IndexAlreadyExistsException::binary_fingerprint[16] = {0xEF,0xB9,0x29,0x59,0x5D,0x31,0x2A,0xC8,0xF3,0x05,0xD5,0xA7,0x94,0xCF,0xED,0xA1}; uint32_t IndexAlreadyExistsException::read(::apache::thrift::protocol::TProtocol* iprot) { @@ -6196,4 +8208,252 @@ void swap(InvalidInputException &a, InvalidInputException &b) { swap(a.__isset, b.__isset); } +const char* NoSuchTxnException::ascii_fingerprint = "EFB929595D312AC8F305D5A794CFEDA1"; +const uint8_t NoSuchTxnException::binary_fingerprint[16] = {0xEF,0xB9,0x29,0x59,0x5D,0x31,0x2A,0xC8,0xF3,0x05,0xD5,0xA7,0x94,0xCF,0xED,0xA1}; + +uint32_t NoSuchTxnException::read(::apache::thrift::protocol::TProtocol* iprot) { + + uint32_t xfer = 0; + std::string fname; + ::apache::thrift::protocol::TType ftype; + int16_t fid; + + xfer += iprot->readStructBegin(fname); + + using ::apache::thrift::protocol::TProtocolException; + + + while (true) + { + xfer += iprot->readFieldBegin(fname, ftype, 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; + 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); +} + +const char* TxnAbortedException::ascii_fingerprint = "EFB929595D312AC8F305D5A794CFEDA1"; +const uint8_t TxnAbortedException::binary_fingerprint[16] = {0xEF,0xB9,0x29,0x59,0x5D,0x31,0x2A,0xC8,0xF3,0x05,0xD5,0xA7,0x94,0xCF,0xED,0xA1}; + +uint32_t TxnAbortedException::read(::apache::thrift::protocol::TProtocol* iprot) { + + uint32_t xfer = 0; + std::string fname; + ::apache::thrift::protocol::TType ftype; + int16_t fid; + + xfer += iprot->readStructBegin(fname); + + using ::apache::thrift::protocol::TProtocolException; + + + while (true) + { + xfer += iprot->readFieldBegin(fname, ftype, 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; + 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); +} + +const char* TxnOpenException::ascii_fingerprint = "EFB929595D312AC8F305D5A794CFEDA1"; +const uint8_t TxnOpenException::binary_fingerprint[16] = {0xEF,0xB9,0x29,0x59,0x5D,0x31,0x2A,0xC8,0xF3,0x05,0xD5,0xA7,0x94,0xCF,0xED,0xA1}; + +uint32_t TxnOpenException::read(::apache::thrift::protocol::TProtocol* iprot) { + + uint32_t xfer = 0; + std::string fname; + ::apache::thrift::protocol::TType ftype; + int16_t fid; + + xfer += iprot->readStructBegin(fname); + + using ::apache::thrift::protocol::TProtocolException; + + + while (true) + { + xfer += iprot->readFieldBegin(fname, ftype, 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; + 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); +} + +const char* NoSuchLockException::ascii_fingerprint = "EFB929595D312AC8F305D5A794CFEDA1"; +const uint8_t NoSuchLockException::binary_fingerprint[16] = {0xEF,0xB9,0x29,0x59,0x5D,0x31,0x2A,0xC8,0xF3,0x05,0xD5,0xA7,0x94,0xCF,0xED,0xA1}; + +uint32_t NoSuchLockException::read(::apache::thrift::protocol::TProtocol* iprot) { + + uint32_t xfer = 0; + std::string fname; + ::apache::thrift::protocol::TType ftype; + int16_t fid; + + xfer += iprot->readStructBegin(fname); + + using ::apache::thrift::protocol::TProtocolException; + + + while (true) + { + xfer += iprot->readFieldBegin(fname, ftype, 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; + 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); +} + }}} // namespace diff --git metastore/src/gen/thrift/gen-cpp/hive_metastore_types.h metastore/src/gen/thrift/gen-cpp/hive_metastore_types.h index 33f5766..0c9dac7 100644 --- metastore/src/gen/thrift/gen-cpp/hive_metastore_types.h +++ metastore/src/gen/thrift/gen-cpp/hive_metastore_types.h @@ -47,6 +47,56 @@ struct PartitionEventType { extern const std::map _PartitionEventType_VALUES_TO_NAMES; +struct TxnState { + enum type { + COMMITTED = 1, + ABORTED = 2, + OPEN = 3 + }; +}; + +extern const std::map _TxnState_VALUES_TO_NAMES; + +struct LockLevel { + enum type { + DB = 1, + TABLE = 2, + PARTITION = 3 + }; +}; + +extern const std::map _LockLevel_VALUES_TO_NAMES; + +struct LockState { + enum type { + ACQUIRED = 1, + WAITING = 2, + ABORT = 3, + NOT_ACQUIRED = 4 + }; +}; + +extern const std::map _LockState_VALUES_TO_NAMES; + +struct LockType { + enum type { + SHARED_READ = 1, + SHARED_WRITE = 2, + EXCLUSIVE = 3 + }; +}; + +extern const std::map _LockType_VALUES_TO_NAMES; + +struct CompactionType { + enum type { + MINOR = 1, + MAJOR = 2 + }; +}; + +extern const std::map _CompactionType_VALUES_TO_NAMES; + struct FunctionType { enum type { JAVA = 1 @@ -2903,439 +2953,1499 @@ class Function { void swap(Function &a, Function &b); -typedef struct _MetaException__isset { - _MetaException__isset() : message(false) {} - bool message; -} _MetaException__isset; -class MetaException : public ::apache::thrift::TException { +class TxnInfo { public: - static const char* ascii_fingerprint; // = "EFB929595D312AC8F305D5A794CFEDA1"; - static const uint8_t binary_fingerprint[16]; // = {0xEF,0xB9,0x29,0x59,0x5D,0x31,0x2A,0xC8,0xF3,0x05,0xD5,0xA7,0x94,0xCF,0xED,0xA1}; + static const char* ascii_fingerprint; // = "6C5C0773A901CCA3BE9D085B3B47A767"; + static const uint8_t binary_fingerprint[16]; // = {0x6C,0x5C,0x07,0x73,0xA9,0x01,0xCC,0xA3,0xBE,0x9D,0x08,0x5B,0x3B,0x47,0xA7,0x67}; - MetaException() : message() { + TxnInfo() : id(0), state((TxnState::type)0), user(), hostname() { } - virtual ~MetaException() throw() {} + virtual ~TxnInfo() throw() {} - std::string message; + int64_t id; + TxnState::type state; + std::string user; + std::string hostname; - _MetaException__isset __isset; + void __set_id(const int64_t val) { + id = val; + } - void __set_message(const std::string& val) { - message = val; + void __set_state(const TxnState::type val) { + state = val; } - bool operator == (const MetaException & rhs) const + void __set_user(const std::string& val) { + user = val; + } + + void __set_hostname(const std::string& val) { + hostname = val; + } + + bool operator == (const TxnInfo & rhs) const { - if (!(message == rhs.message)) + if (!(id == rhs.id)) + return false; + if (!(state == rhs.state)) + return false; + if (!(user == rhs.user)) + return false; + if (!(hostname == rhs.hostname)) return false; return true; } - bool operator != (const MetaException &rhs) const { + bool operator != (const TxnInfo &rhs) const { return !(*this == rhs); } - bool operator < (const MetaException & ) const; + bool operator < (const TxnInfo & ) const; uint32_t read(::apache::thrift::protocol::TProtocol* iprot); uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; }; -void swap(MetaException &a, MetaException &b); +void swap(TxnInfo &a, TxnInfo &b); -typedef struct _UnknownTableException__isset { - _UnknownTableException__isset() : message(false) {} - bool message; -} _UnknownTableException__isset; -class UnknownTableException : public ::apache::thrift::TException { +class GetOpenTxnsInfoResponse { public: - static const char* ascii_fingerprint; // = "EFB929595D312AC8F305D5A794CFEDA1"; - static const uint8_t binary_fingerprint[16]; // = {0xEF,0xB9,0x29,0x59,0x5D,0x31,0x2A,0xC8,0xF3,0x05,0xD5,0xA7,0x94,0xCF,0xED,0xA1}; + static const char* ascii_fingerprint; // = "CCF769BBD33005B61F2079A6665E3B9C"; + static const uint8_t binary_fingerprint[16]; // = {0xCC,0xF7,0x69,0xBB,0xD3,0x30,0x05,0xB6,0x1F,0x20,0x79,0xA6,0x66,0x5E,0x3B,0x9C}; - UnknownTableException() : message() { + GetOpenTxnsInfoResponse() : txn_high_water_mark(0) { } - virtual ~UnknownTableException() throw() {} + virtual ~GetOpenTxnsInfoResponse() throw() {} - std::string message; + int64_t txn_high_water_mark; + std::vector open_txns; - _UnknownTableException__isset __isset; + void __set_txn_high_water_mark(const int64_t val) { + txn_high_water_mark = val; + } - void __set_message(const std::string& val) { - message = val; + void __set_open_txns(const std::vector & val) { + open_txns = val; } - bool operator == (const UnknownTableException & rhs) const + bool operator == (const GetOpenTxnsInfoResponse & rhs) const { - if (!(message == rhs.message)) + 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 UnknownTableException &rhs) const { + bool operator != (const GetOpenTxnsInfoResponse &rhs) const { return !(*this == rhs); } - bool operator < (const UnknownTableException & ) const; + bool operator < (const GetOpenTxnsInfoResponse & ) const; uint32_t read(::apache::thrift::protocol::TProtocol* iprot); uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; }; -void swap(UnknownTableException &a, UnknownTableException &b); +void swap(GetOpenTxnsInfoResponse &a, GetOpenTxnsInfoResponse &b); -typedef struct _UnknownDBException__isset { - _UnknownDBException__isset() : message(false) {} - bool message; -} _UnknownDBException__isset; -class UnknownDBException : public ::apache::thrift::TException { +class GetOpenTxnsResponse { public: - static const char* ascii_fingerprint; // = "EFB929595D312AC8F305D5A794CFEDA1"; - static const uint8_t binary_fingerprint[16]; // = {0xEF,0xB9,0x29,0x59,0x5D,0x31,0x2A,0xC8,0xF3,0x05,0xD5,0xA7,0x94,0xCF,0xED,0xA1}; + static const char* ascii_fingerprint; // = "590531FF1BE8611678B255374F6109EE"; + static const uint8_t binary_fingerprint[16]; // = {0x59,0x05,0x31,0xFF,0x1B,0xE8,0x61,0x16,0x78,0xB2,0x55,0x37,0x4F,0x61,0x09,0xEE}; - UnknownDBException() : message() { + GetOpenTxnsResponse() : txn_high_water_mark(0) { } - virtual ~UnknownDBException() throw() {} + virtual ~GetOpenTxnsResponse() throw() {} - std::string message; + int64_t txn_high_water_mark; + std::set open_txns; - _UnknownDBException__isset __isset; + void __set_txn_high_water_mark(const int64_t val) { + txn_high_water_mark = val; + } - void __set_message(const std::string& val) { - message = val; + void __set_open_txns(const std::set & val) { + open_txns = val; } - bool operator == (const UnknownDBException & rhs) const + bool operator == (const GetOpenTxnsResponse & rhs) const { - if (!(message == rhs.message)) + 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 UnknownDBException &rhs) const { + bool operator != (const GetOpenTxnsResponse &rhs) const { return !(*this == rhs); } - bool operator < (const UnknownDBException & ) const; + bool operator < (const GetOpenTxnsResponse & ) const; uint32_t read(::apache::thrift::protocol::TProtocol* iprot); uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; }; -void swap(UnknownDBException &a, UnknownDBException &b); +void swap(GetOpenTxnsResponse &a, GetOpenTxnsResponse &b); -typedef struct _AlreadyExistsException__isset { - _AlreadyExistsException__isset() : message(false) {} - bool message; -} _AlreadyExistsException__isset; -class AlreadyExistsException : public ::apache::thrift::TException { +class OpenTxnRequest { public: - static const char* ascii_fingerprint; // = "EFB929595D312AC8F305D5A794CFEDA1"; - static const uint8_t binary_fingerprint[16]; // = {0xEF,0xB9,0x29,0x59,0x5D,0x31,0x2A,0xC8,0xF3,0x05,0xD5,0xA7,0x94,0xCF,0xED,0xA1}; + static const char* ascii_fingerprint; // = "3368C2F81F2FEF71F11EDACDB2A3ECEF"; + static const uint8_t binary_fingerprint[16]; // = {0x33,0x68,0xC2,0xF8,0x1F,0x2F,0xEF,0x71,0xF1,0x1E,0xDA,0xCD,0xB2,0xA3,0xEC,0xEF}; - AlreadyExistsException() : message() { + OpenTxnRequest() : num_txns(0), user(), hostname() { } - virtual ~AlreadyExistsException() throw() {} + virtual ~OpenTxnRequest() throw() {} - std::string message; + int32_t num_txns; + std::string user; + std::string hostname; - _AlreadyExistsException__isset __isset; + void __set_num_txns(const int32_t val) { + num_txns = val; + } - void __set_message(const std::string& val) { - message = val; + void __set_user(const std::string& val) { + user = val; } - bool operator == (const AlreadyExistsException & rhs) const + void __set_hostname(const std::string& val) { + hostname = val; + } + + bool operator == (const OpenTxnRequest & rhs) const { - if (!(message == rhs.message)) + if (!(num_txns == rhs.num_txns)) + return false; + if (!(user == rhs.user)) + return false; + if (!(hostname == rhs.hostname)) return false; return true; } - bool operator != (const AlreadyExistsException &rhs) const { + bool operator != (const OpenTxnRequest &rhs) const { return !(*this == rhs); } - bool operator < (const AlreadyExistsException & ) const; + bool operator < (const OpenTxnRequest & ) const; uint32_t read(::apache::thrift::protocol::TProtocol* iprot); uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; }; -void swap(AlreadyExistsException &a, AlreadyExistsException &b); +void swap(OpenTxnRequest &a, OpenTxnRequest &b); -typedef struct _InvalidPartitionException__isset { - _InvalidPartitionException__isset() : message(false) {} - bool message; -} _InvalidPartitionException__isset; -class InvalidPartitionException : public ::apache::thrift::TException { +class OpenTxnsResponse { public: - static const char* ascii_fingerprint; // = "EFB929595D312AC8F305D5A794CFEDA1"; - static const uint8_t binary_fingerprint[16]; // = {0xEF,0xB9,0x29,0x59,0x5D,0x31,0x2A,0xC8,0xF3,0x05,0xD5,0xA7,0x94,0xCF,0xED,0xA1}; + static const char* ascii_fingerprint; // = "E49D7D1A9013CC81CD0F69D631EF82E4"; + static const uint8_t binary_fingerprint[16]; // = {0xE4,0x9D,0x7D,0x1A,0x90,0x13,0xCC,0x81,0xCD,0x0F,0x69,0xD6,0x31,0xEF,0x82,0xE4}; - InvalidPartitionException() : message() { + OpenTxnsResponse() { } - virtual ~InvalidPartitionException() throw() {} - - std::string message; + virtual ~OpenTxnsResponse() throw() {} - _InvalidPartitionException__isset __isset; + std::vector txn_ids; - void __set_message(const std::string& val) { - message = val; + void __set_txn_ids(const std::vector & val) { + txn_ids = val; } - bool operator == (const InvalidPartitionException & rhs) const + bool operator == (const OpenTxnsResponse & rhs) const { - if (!(message == rhs.message)) + if (!(txn_ids == rhs.txn_ids)) return false; return true; } - bool operator != (const InvalidPartitionException &rhs) const { + bool operator != (const OpenTxnsResponse &rhs) const { return !(*this == rhs); } - bool operator < (const InvalidPartitionException & ) const; + bool operator < (const OpenTxnsResponse & ) const; uint32_t read(::apache::thrift::protocol::TProtocol* iprot); uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; }; -void swap(InvalidPartitionException &a, InvalidPartitionException &b); +void swap(OpenTxnsResponse &a, OpenTxnsResponse &b); -typedef struct _UnknownPartitionException__isset { - _UnknownPartitionException__isset() : message(false) {} - bool message; -} _UnknownPartitionException__isset; -class UnknownPartitionException : public ::apache::thrift::TException { +class AbortTxnRequest { public: - static const char* ascii_fingerprint; // = "EFB929595D312AC8F305D5A794CFEDA1"; - static const uint8_t binary_fingerprint[16]; // = {0xEF,0xB9,0x29,0x59,0x5D,0x31,0x2A,0xC8,0xF3,0x05,0xD5,0xA7,0x94,0xCF,0xED,0xA1}; + static const char* ascii_fingerprint; // = "56A59CE7FFAF82BCA8A19FAACDE4FB75"; + static const uint8_t binary_fingerprint[16]; // = {0x56,0xA5,0x9C,0xE7,0xFF,0xAF,0x82,0xBC,0xA8,0xA1,0x9F,0xAA,0xCD,0xE4,0xFB,0x75}; - UnknownPartitionException() : message() { + AbortTxnRequest() : txnid(0) { } - virtual ~UnknownPartitionException() throw() {} - - std::string message; + virtual ~AbortTxnRequest() throw() {} - _UnknownPartitionException__isset __isset; + int64_t txnid; - void __set_message(const std::string& val) { - message = val; + void __set_txnid(const int64_t val) { + txnid = val; } - bool operator == (const UnknownPartitionException & rhs) const + bool operator == (const AbortTxnRequest & rhs) const { - if (!(message == rhs.message)) + if (!(txnid == rhs.txnid)) return false; return true; } - bool operator != (const UnknownPartitionException &rhs) const { + bool operator != (const AbortTxnRequest &rhs) const { return !(*this == rhs); } - bool operator < (const UnknownPartitionException & ) const; + bool operator < (const AbortTxnRequest & ) const; uint32_t read(::apache::thrift::protocol::TProtocol* iprot); uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; }; -void swap(UnknownPartitionException &a, UnknownPartitionException &b); +void swap(AbortTxnRequest &a, AbortTxnRequest &b); -typedef struct _InvalidObjectException__isset { - _InvalidObjectException__isset() : message(false) {} - bool message; -} _InvalidObjectException__isset; -class InvalidObjectException : public ::apache::thrift::TException { +class CommitTxnRequest { public: - static const char* ascii_fingerprint; // = "EFB929595D312AC8F305D5A794CFEDA1"; - static const uint8_t binary_fingerprint[16]; // = {0xEF,0xB9,0x29,0x59,0x5D,0x31,0x2A,0xC8,0xF3,0x05,0xD5,0xA7,0x94,0xCF,0xED,0xA1}; + static const char* ascii_fingerprint; // = "56A59CE7FFAF82BCA8A19FAACDE4FB75"; + static const uint8_t binary_fingerprint[16]; // = {0x56,0xA5,0x9C,0xE7,0xFF,0xAF,0x82,0xBC,0xA8,0xA1,0x9F,0xAA,0xCD,0xE4,0xFB,0x75}; - InvalidObjectException() : message() { + CommitTxnRequest() : txnid(0) { } - virtual ~InvalidObjectException() throw() {} - - std::string message; + virtual ~CommitTxnRequest() throw() {} - _InvalidObjectException__isset __isset; + int64_t txnid; - void __set_message(const std::string& val) { - message = val; + void __set_txnid(const int64_t val) { + txnid = val; } - bool operator == (const InvalidObjectException & rhs) const + bool operator == (const CommitTxnRequest & rhs) const { - if (!(message == rhs.message)) + if (!(txnid == rhs.txnid)) return false; return true; } - bool operator != (const InvalidObjectException &rhs) const { + bool operator != (const CommitTxnRequest &rhs) const { return !(*this == rhs); } - bool operator < (const InvalidObjectException & ) const; + bool operator < (const CommitTxnRequest & ) const; uint32_t read(::apache::thrift::protocol::TProtocol* iprot); uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; }; -void swap(InvalidObjectException &a, InvalidObjectException &b); +void swap(CommitTxnRequest &a, CommitTxnRequest &b); -typedef struct _NoSuchObjectException__isset { - _NoSuchObjectException__isset() : message(false) {} - bool message; -} _NoSuchObjectException__isset; +typedef struct _LockComponent__isset { + _LockComponent__isset() : tablename(false), partitionname(false) {} + bool tablename; + bool partitionname; +} _LockComponent__isset; -class NoSuchObjectException : public ::apache::thrift::TException { +class LockComponent { public: - static const char* ascii_fingerprint; // = "EFB929595D312AC8F305D5A794CFEDA1"; - static const uint8_t binary_fingerprint[16]; // = {0xEF,0xB9,0x29,0x59,0x5D,0x31,0x2A,0xC8,0xF3,0x05,0xD5,0xA7,0x94,0xCF,0xED,0xA1}; + static const char* ascii_fingerprint; // = "38B02531B0840AC9C72904A4649FD15F"; + static const uint8_t binary_fingerprint[16]; // = {0x38,0xB0,0x25,0x31,0xB0,0x84,0x0A,0xC9,0xC7,0x29,0x04,0xA4,0x64,0x9F,0xD1,0x5F}; - NoSuchObjectException() : message() { + LockComponent() : type((LockType::type)0), level((LockLevel::type)0), dbname(), tablename(), partitionname() { } - virtual ~NoSuchObjectException() throw() {} + virtual ~LockComponent() throw() {} - std::string message; + LockType::type type; + LockLevel::type level; + std::string dbname; + std::string tablename; + std::string partitionname; - _NoSuchObjectException__isset __isset; + _LockComponent__isset __isset; - void __set_message(const std::string& val) { - message = val; + void __set_type(const LockType::type val) { + type = val; } - bool operator == (const NoSuchObjectException & rhs) const + void __set_level(const LockLevel::type val) { + level = val; + } + + void __set_dbname(const std::string& val) { + dbname = val; + } + + void __set_tablename(const std::string& val) { + tablename = val; + __isset.tablename = true; + } + + void __set_partitionname(const std::string& val) { + partitionname = val; + __isset.partitionname = true; + } + + bool operator == (const LockComponent & rhs) const { - if (!(message == rhs.message)) + 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; return true; } - bool operator != (const NoSuchObjectException &rhs) const { + bool operator != (const LockComponent &rhs) const { return !(*this == rhs); } - bool operator < (const NoSuchObjectException & ) const; + bool operator < (const LockComponent & ) const; uint32_t read(::apache::thrift::protocol::TProtocol* iprot); uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; }; -void swap(NoSuchObjectException &a, NoSuchObjectException &b); +void swap(LockComponent &a, LockComponent &b); -typedef struct _IndexAlreadyExistsException__isset { - _IndexAlreadyExistsException__isset() : message(false) {} - bool message; -} _IndexAlreadyExistsException__isset; +typedef struct _LockRequest__isset { + _LockRequest__isset() : txnid(false) {} + bool txnid; +} _LockRequest__isset; -class IndexAlreadyExistsException : public ::apache::thrift::TException { +class LockRequest { public: - static const char* ascii_fingerprint; // = "EFB929595D312AC8F305D5A794CFEDA1"; - static const uint8_t binary_fingerprint[16]; // = {0xEF,0xB9,0x29,0x59,0x5D,0x31,0x2A,0xC8,0xF3,0x05,0xD5,0xA7,0x94,0xCF,0xED,0xA1}; + static const char* ascii_fingerprint; // = "46BC5ED7196BC16CB216AD5CC67C6930"; + static const uint8_t binary_fingerprint[16]; // = {0x46,0xBC,0x5E,0xD7,0x19,0x6B,0xC1,0x6C,0xB2,0x16,0xAD,0x5C,0xC6,0x7C,0x69,0x30}; - IndexAlreadyExistsException() : message() { + LockRequest() : txnid(0), user(), hostname() { } - virtual ~IndexAlreadyExistsException() throw() {} + virtual ~LockRequest() throw() {} - std::string message; + std::vector component; + int64_t txnid; + std::string user; + std::string hostname; - _IndexAlreadyExistsException__isset __isset; + _LockRequest__isset __isset; - void __set_message(const std::string& val) { - message = val; + void __set_component(const std::vector & val) { + component = val; } - bool operator == (const IndexAlreadyExistsException & rhs) const + void __set_txnid(const int64_t val) { + txnid = val; + __isset.txnid = true; + } + + void __set_user(const std::string& val) { + user = val; + } + + void __set_hostname(const std::string& val) { + hostname = val; + } + + bool operator == (const LockRequest & rhs) const { - if (!(message == rhs.message)) + 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; return true; } - bool operator != (const IndexAlreadyExistsException &rhs) const { + bool operator != (const LockRequest &rhs) const { return !(*this == rhs); } - bool operator < (const IndexAlreadyExistsException & ) const; + bool operator < (const LockRequest & ) const; uint32_t read(::apache::thrift::protocol::TProtocol* iprot); uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; }; -void swap(IndexAlreadyExistsException &a, IndexAlreadyExistsException &b); +void swap(LockRequest &a, LockRequest &b); -typedef struct _InvalidOperationException__isset { - _InvalidOperationException__isset() : message(false) {} - bool message; -} _InvalidOperationException__isset; -class InvalidOperationException : public ::apache::thrift::TException { +class LockResponse { public: - static const char* ascii_fingerprint; // = "EFB929595D312AC8F305D5A794CFEDA1"; - static const uint8_t binary_fingerprint[16]; // = {0xEF,0xB9,0x29,0x59,0x5D,0x31,0x2A,0xC8,0xF3,0x05,0xD5,0xA7,0x94,0xCF,0xED,0xA1}; + static const char* ascii_fingerprint; // = "DFA40D9D2884599F3D1E7A57578F1384"; + static const uint8_t binary_fingerprint[16]; // = {0xDF,0xA4,0x0D,0x9D,0x28,0x84,0x59,0x9F,0x3D,0x1E,0x7A,0x57,0x57,0x8F,0x13,0x84}; - InvalidOperationException() : message() { + LockResponse() : lockid(0), state((LockState::type)0) { } - virtual ~InvalidOperationException() throw() {} + virtual ~LockResponse() throw() {} - std::string message; + int64_t lockid; + LockState::type state; - _InvalidOperationException__isset __isset; + void __set_lockid(const int64_t val) { + lockid = val; + } - void __set_message(const std::string& val) { - message = val; + void __set_state(const LockState::type val) { + state = val; } - bool operator == (const InvalidOperationException & rhs) const + bool operator == (const LockResponse & rhs) const { - if (!(message == rhs.message)) + if (!(lockid == rhs.lockid)) + return false; + if (!(state == rhs.state)) return false; return true; } - bool operator != (const InvalidOperationException &rhs) const { + bool operator != (const LockResponse &rhs) const { return !(*this == rhs); } - bool operator < (const InvalidOperationException & ) const; + bool operator < (const LockResponse & ) const; uint32_t read(::apache::thrift::protocol::TProtocol* iprot); uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; }; -void swap(InvalidOperationException &a, InvalidOperationException &b); +void swap(LockResponse &a, LockResponse &b); -typedef struct _ConfigValSecurityException__isset { - _ConfigValSecurityException__isset() : message(false) {} - bool message; + +class CheckLockRequest { + public: + + static const char* ascii_fingerprint; // = "56A59CE7FFAF82BCA8A19FAACDE4FB75"; + static const uint8_t binary_fingerprint[16]; // = {0x56,0xA5,0x9C,0xE7,0xFF,0xAF,0x82,0xBC,0xA8,0xA1,0x9F,0xAA,0xCD,0xE4,0xFB,0x75}; + + CheckLockRequest() : lockid(0) { + } + + virtual ~CheckLockRequest() throw() {} + + int64_t lockid; + + void __set_lockid(const int64_t val) { + lockid = val; + } + + bool operator == (const CheckLockRequest & rhs) const + { + if (!(lockid == rhs.lockid)) + 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; + +}; + +void swap(CheckLockRequest &a, CheckLockRequest &b); + + +class UnlockRequest { + public: + + static const char* ascii_fingerprint; // = "56A59CE7FFAF82BCA8A19FAACDE4FB75"; + static const uint8_t binary_fingerprint[16]; // = {0x56,0xA5,0x9C,0xE7,0xFF,0xAF,0x82,0xBC,0xA8,0xA1,0x9F,0xAA,0xCD,0xE4,0xFB,0x75}; + + UnlockRequest() : lockid(0) { + } + + virtual ~UnlockRequest() throw() {} + + int64_t lockid; + + void __set_lockid(const int64_t val) { + lockid = 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; + +}; + +void swap(UnlockRequest &a, UnlockRequest &b); + + +class ShowLocksRequest { + public: + + static const char* ascii_fingerprint; // = "99914B932BD37A50B983C5E7C90AE93B"; + static const uint8_t binary_fingerprint[16]; // = {0x99,0x91,0x4B,0x93,0x2B,0xD3,0x7A,0x50,0xB9,0x83,0xC5,0xE7,0xC9,0x0A,0xE9,0x3B}; + + ShowLocksRequest() { + } + + virtual ~ShowLocksRequest() throw() {} + + + bool operator == (const ShowLocksRequest & /* rhs */) const + { + 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; + +}; + +void swap(ShowLocksRequest &a, ShowLocksRequest &b); + +typedef struct _ShowLocksResponseElement__isset { + _ShowLocksResponseElement__isset() : tablename(false), partname(false), txnid(false), acquiredat(false) {} + bool tablename; + bool partname; + bool txnid; + bool acquiredat; +} _ShowLocksResponseElement__isset; + +class ShowLocksResponseElement { + public: + + static const char* ascii_fingerprint; // = "5AD11F0E0EF1EE0A7C08B00FEFCFF24F"; + static const uint8_t binary_fingerprint[16]; // = {0x5A,0xD1,0x1F,0x0E,0x0E,0xF1,0xEE,0x0A,0x7C,0x08,0xB0,0x0F,0xEF,0xCF,0xF2,0x4F}; + + ShowLocksResponseElement() : lockid(0), dbname(), tablename(), partname(), state((LockState::type)0), type((LockType::type)0), txnid(0), lastheartbeat(0), acquiredat(0), user(), hostname() { + } + + virtual ~ShowLocksResponseElement() throw() {} + + 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; + + _ShowLocksResponseElement__isset __isset; + + void __set_lockid(const int64_t val) { + lockid = val; + } + + void __set_dbname(const std::string& val) { + dbname = val; + } + + void __set_tablename(const std::string& val) { + tablename = val; + __isset.tablename = true; + } + + void __set_partname(const std::string& val) { + partname = val; + __isset.partname = true; + } + + void __set_state(const LockState::type val) { + state = val; + } + + void __set_type(const LockType::type val) { + type = val; + } + + void __set_txnid(const int64_t val) { + txnid = val; + __isset.txnid = true; + } + + void __set_lastheartbeat(const int64_t val) { + lastheartbeat = val; + } + + void __set_acquiredat(const int64_t val) { + acquiredat = val; + __isset.acquiredat = true; + } + + void __set_user(const std::string& val) { + user = val; + } + + void __set_hostname(const std::string& val) { + hostname = 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; + 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; + +}; + +void swap(ShowLocksResponseElement &a, ShowLocksResponseElement &b); + +typedef struct _ShowLocksResponse__isset { + _ShowLocksResponse__isset() : locks(false) {} + bool locks; +} _ShowLocksResponse__isset; + +class ShowLocksResponse { + public: + + static const char* ascii_fingerprint; // = "BD598AA60FE941361FB54C43973C011F"; + static const uint8_t binary_fingerprint[16]; // = {0xBD,0x59,0x8A,0xA6,0x0F,0xE9,0x41,0x36,0x1F,0xB5,0x4C,0x43,0x97,0x3C,0x01,0x1F}; + + ShowLocksResponse() { + } + + virtual ~ShowLocksResponse() throw() {} + + std::vector locks; + + _ShowLocksResponse__isset __isset; + + void __set_locks(const std::vector & val) { + locks = 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; + +}; + +void swap(ShowLocksResponse &a, ShowLocksResponse &b); + +typedef struct _HeartbeatRequest__isset { + _HeartbeatRequest__isset() : lockid(false), txnid(false) {} + bool lockid; + bool txnid; +} _HeartbeatRequest__isset; + +class HeartbeatRequest { + public: + + static const char* ascii_fingerprint; // = "0354D07C94CB8542872CA1277008860A"; + static const uint8_t binary_fingerprint[16]; // = {0x03,0x54,0xD0,0x7C,0x94,0xCB,0x85,0x42,0x87,0x2C,0xA1,0x27,0x70,0x08,0x86,0x0A}; + + HeartbeatRequest() : lockid(0), txnid(0) { + } + + virtual ~HeartbeatRequest() throw() {} + + int64_t lockid; + int64_t txnid; + + _HeartbeatRequest__isset __isset; + + void __set_lockid(const int64_t val) { + lockid = val; + __isset.lockid = true; + } + + void __set_txnid(const int64_t val) { + txnid = val; + __isset.txnid = true; + } + + 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; + +}; + +void swap(HeartbeatRequest &a, HeartbeatRequest &b); + +typedef struct _CompactionRequest__isset { + _CompactionRequest__isset() : partitionname(false), runas(false) {} + bool partitionname; + bool runas; +} _CompactionRequest__isset; + +class CompactionRequest { + public: + + static const char* ascii_fingerprint; // = "899FD1F339D8318D628687CC2CE2864B"; + static const uint8_t binary_fingerprint[16]; // = {0x89,0x9F,0xD1,0xF3,0x39,0xD8,0x31,0x8D,0x62,0x86,0x87,0xCC,0x2C,0xE2,0x86,0x4B}; + + CompactionRequest() : dbname(), tablename(), partitionname(), type((CompactionType::type)0), runas() { + } + + virtual ~CompactionRequest() throw() {} + + std::string dbname; + std::string tablename; + std::string partitionname; + CompactionType::type type; + std::string runas; + + _CompactionRequest__isset __isset; + + void __set_dbname(const std::string& val) { + dbname = val; + } + + void __set_tablename(const std::string& val) { + tablename = val; + } + + void __set_partitionname(const std::string& val) { + partitionname = val; + __isset.partitionname = true; + } + + void __set_type(const CompactionType::type val) { + type = val; + } + + void __set_runas(const std::string& val) { + runas = val; + __isset.runas = true; + } + + 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; + 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; + +}; + +void swap(CompactionRequest &a, CompactionRequest &b); + + +class ShowCompactRequest { + public: + + static const char* ascii_fingerprint; // = "99914B932BD37A50B983C5E7C90AE93B"; + static const uint8_t binary_fingerprint[16]; // = {0x99,0x91,0x4B,0x93,0x2B,0xD3,0x7A,0x50,0xB9,0x83,0xC5,0xE7,0xC9,0x0A,0xE9,0x3B}; + + ShowCompactRequest() { + } + + virtual ~ShowCompactRequest() throw() {} + + + 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; + +}; + +void swap(ShowCompactRequest &a, ShowCompactRequest &b); + + +class ShowCompactResponseElement { + public: + + static const char* ascii_fingerprint; // = "42464F3A97707F984FDE462104223A69"; + static const uint8_t binary_fingerprint[16]; // = {0x42,0x46,0x4F,0x3A,0x97,0x70,0x7F,0x98,0x4F,0xDE,0x46,0x21,0x04,0x22,0x3A,0x69}; + + ShowCompactResponseElement() : dbname(), tablename(), partitionname(), type((CompactionType::type)0), state(), workerid(), start(0), runAs() { + } + + virtual ~ShowCompactResponseElement() throw() {} + + std::string dbname; + std::string tablename; + std::string partitionname; + CompactionType::type type; + std::string state; + std::string workerid; + int64_t start; + std::string runAs; + + void __set_dbname(const std::string& val) { + dbname = val; + } + + void __set_tablename(const std::string& val) { + tablename = val; + } + + void __set_partitionname(const std::string& val) { + partitionname = val; + } + + void __set_type(const CompactionType::type val) { + type = val; + } + + void __set_state(const std::string& val) { + state = val; + } + + void __set_workerid(const std::string& val) { + workerid = val; + } + + void __set_start(const int64_t val) { + start = val; + } + + void __set_runAs(const std::string& val) { + runAs = val; + } + + bool operator == (const ShowCompactResponseElement & rhs) const + { + if (!(dbname == rhs.dbname)) + return false; + if (!(tablename == rhs.tablename)) + return false; + if (!(partitionname == rhs.partitionname)) + return false; + if (!(type == rhs.type)) + return false; + if (!(state == rhs.state)) + return false; + if (!(workerid == rhs.workerid)) + return false; + if (!(start == rhs.start)) + return false; + if (!(runAs == rhs.runAs)) + 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; + +}; + +void swap(ShowCompactResponseElement &a, ShowCompactResponseElement &b); + + +class ShowCompactResponse { + public: + + static const char* ascii_fingerprint; // = "3CCBC4D398CA25527272FE78625DE88A"; + static const uint8_t binary_fingerprint[16]; // = {0x3C,0xCB,0xC4,0xD3,0x98,0xCA,0x25,0x52,0x72,0x72,0xFE,0x78,0x62,0x5D,0xE8,0x8A}; + + ShowCompactResponse() { + } + + virtual ~ShowCompactResponse() throw() {} + + std::vector compacts; + + void __set_compacts(const std::vector & val) { + compacts = 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; + +}; + +void swap(ShowCompactResponse &a, ShowCompactResponse &b); + +typedef struct _MetaException__isset { + _MetaException__isset() : message(false) {} + bool message; +} _MetaException__isset; + +class MetaException : public ::apache::thrift::TException { + public: + + static const char* ascii_fingerprint; // = "EFB929595D312AC8F305D5A794CFEDA1"; + static const uint8_t binary_fingerprint[16]; // = {0xEF,0xB9,0x29,0x59,0x5D,0x31,0x2A,0xC8,0xF3,0x05,0xD5,0xA7,0x94,0xCF,0xED,0xA1}; + + MetaException() : message() { + } + + virtual ~MetaException() throw() {} + + std::string message; + + _MetaException__isset __isset; + + void __set_message(const std::string& val) { + message = 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; + +}; + +void swap(MetaException &a, MetaException &b); + +typedef struct _UnknownTableException__isset { + _UnknownTableException__isset() : message(false) {} + bool message; +} _UnknownTableException__isset; + +class UnknownTableException : public ::apache::thrift::TException { + public: + + static const char* ascii_fingerprint; // = "EFB929595D312AC8F305D5A794CFEDA1"; + static const uint8_t binary_fingerprint[16]; // = {0xEF,0xB9,0x29,0x59,0x5D,0x31,0x2A,0xC8,0xF3,0x05,0xD5,0xA7,0x94,0xCF,0xED,0xA1}; + + UnknownTableException() : message() { + } + + virtual ~UnknownTableException() throw() {} + + std::string message; + + _UnknownTableException__isset __isset; + + void __set_message(const std::string& val) { + message = 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; + +}; + +void swap(UnknownTableException &a, UnknownTableException &b); + +typedef struct _UnknownDBException__isset { + _UnknownDBException__isset() : message(false) {} + bool message; +} _UnknownDBException__isset; + +class UnknownDBException : public ::apache::thrift::TException { + public: + + static const char* ascii_fingerprint; // = "EFB929595D312AC8F305D5A794CFEDA1"; + static const uint8_t binary_fingerprint[16]; // = {0xEF,0xB9,0x29,0x59,0x5D,0x31,0x2A,0xC8,0xF3,0x05,0xD5,0xA7,0x94,0xCF,0xED,0xA1}; + + UnknownDBException() : message() { + } + + virtual ~UnknownDBException() throw() {} + + std::string message; + + _UnknownDBException__isset __isset; + + void __set_message(const std::string& val) { + message = 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; + +}; + +void swap(UnknownDBException &a, UnknownDBException &b); + +typedef struct _AlreadyExistsException__isset { + _AlreadyExistsException__isset() : message(false) {} + bool message; +} _AlreadyExistsException__isset; + +class AlreadyExistsException : public ::apache::thrift::TException { + public: + + static const char* ascii_fingerprint; // = "EFB929595D312AC8F305D5A794CFEDA1"; + static const uint8_t binary_fingerprint[16]; // = {0xEF,0xB9,0x29,0x59,0x5D,0x31,0x2A,0xC8,0xF3,0x05,0xD5,0xA7,0x94,0xCF,0xED,0xA1}; + + AlreadyExistsException() : message() { + } + + virtual ~AlreadyExistsException() throw() {} + + std::string message; + + _AlreadyExistsException__isset __isset; + + void __set_message(const std::string& val) { + message = 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; + +}; + +void swap(AlreadyExistsException &a, AlreadyExistsException &b); + +typedef struct _InvalidPartitionException__isset { + _InvalidPartitionException__isset() : message(false) {} + bool message; +} _InvalidPartitionException__isset; + +class InvalidPartitionException : public ::apache::thrift::TException { + public: + + static const char* ascii_fingerprint; // = "EFB929595D312AC8F305D5A794CFEDA1"; + static const uint8_t binary_fingerprint[16]; // = {0xEF,0xB9,0x29,0x59,0x5D,0x31,0x2A,0xC8,0xF3,0x05,0xD5,0xA7,0x94,0xCF,0xED,0xA1}; + + InvalidPartitionException() : message() { + } + + virtual ~InvalidPartitionException() throw() {} + + std::string message; + + _InvalidPartitionException__isset __isset; + + void __set_message(const std::string& val) { + message = 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; + +}; + +void swap(InvalidPartitionException &a, InvalidPartitionException &b); + +typedef struct _UnknownPartitionException__isset { + _UnknownPartitionException__isset() : message(false) {} + bool message; +} _UnknownPartitionException__isset; + +class UnknownPartitionException : public ::apache::thrift::TException { + public: + + static const char* ascii_fingerprint; // = "EFB929595D312AC8F305D5A794CFEDA1"; + static const uint8_t binary_fingerprint[16]; // = {0xEF,0xB9,0x29,0x59,0x5D,0x31,0x2A,0xC8,0xF3,0x05,0xD5,0xA7,0x94,0xCF,0xED,0xA1}; + + UnknownPartitionException() : message() { + } + + virtual ~UnknownPartitionException() throw() {} + + std::string message; + + _UnknownPartitionException__isset __isset; + + void __set_message(const std::string& val) { + message = 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; + +}; + +void swap(UnknownPartitionException &a, UnknownPartitionException &b); + +typedef struct _InvalidObjectException__isset { + _InvalidObjectException__isset() : message(false) {} + bool message; +} _InvalidObjectException__isset; + +class InvalidObjectException : public ::apache::thrift::TException { + public: + + static const char* ascii_fingerprint; // = "EFB929595D312AC8F305D5A794CFEDA1"; + static const uint8_t binary_fingerprint[16]; // = {0xEF,0xB9,0x29,0x59,0x5D,0x31,0x2A,0xC8,0xF3,0x05,0xD5,0xA7,0x94,0xCF,0xED,0xA1}; + + InvalidObjectException() : message() { + } + + virtual ~InvalidObjectException() throw() {} + + std::string message; + + _InvalidObjectException__isset __isset; + + void __set_message(const std::string& val) { + message = 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; + +}; + +void swap(InvalidObjectException &a, InvalidObjectException &b); + +typedef struct _NoSuchObjectException__isset { + _NoSuchObjectException__isset() : message(false) {} + bool message; +} _NoSuchObjectException__isset; + +class NoSuchObjectException : public ::apache::thrift::TException { + public: + + static const char* ascii_fingerprint; // = "EFB929595D312AC8F305D5A794CFEDA1"; + static const uint8_t binary_fingerprint[16]; // = {0xEF,0xB9,0x29,0x59,0x5D,0x31,0x2A,0xC8,0xF3,0x05,0xD5,0xA7,0x94,0xCF,0xED,0xA1}; + + NoSuchObjectException() : message() { + } + + virtual ~NoSuchObjectException() throw() {} + + std::string message; + + _NoSuchObjectException__isset __isset; + + void __set_message(const std::string& val) { + message = 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; + +}; + +void swap(NoSuchObjectException &a, NoSuchObjectException &b); + +typedef struct _IndexAlreadyExistsException__isset { + _IndexAlreadyExistsException__isset() : message(false) {} + bool message; +} _IndexAlreadyExistsException__isset; + +class IndexAlreadyExistsException : public ::apache::thrift::TException { + public: + + static const char* ascii_fingerprint; // = "EFB929595D312AC8F305D5A794CFEDA1"; + static const uint8_t binary_fingerprint[16]; // = {0xEF,0xB9,0x29,0x59,0x5D,0x31,0x2A,0xC8,0xF3,0x05,0xD5,0xA7,0x94,0xCF,0xED,0xA1}; + + IndexAlreadyExistsException() : message() { + } + + virtual ~IndexAlreadyExistsException() throw() {} + + std::string message; + + _IndexAlreadyExistsException__isset __isset; + + void __set_message(const std::string& val) { + message = 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; + +}; + +void swap(IndexAlreadyExistsException &a, IndexAlreadyExistsException &b); + +typedef struct _InvalidOperationException__isset { + _InvalidOperationException__isset() : message(false) {} + bool message; +} _InvalidOperationException__isset; + +class InvalidOperationException : public ::apache::thrift::TException { + public: + + static const char* ascii_fingerprint; // = "EFB929595D312AC8F305D5A794CFEDA1"; + static const uint8_t binary_fingerprint[16]; // = {0xEF,0xB9,0x29,0x59,0x5D,0x31,0x2A,0xC8,0xF3,0x05,0xD5,0xA7,0x94,0xCF,0xED,0xA1}; + + InvalidOperationException() : message() { + } + + virtual ~InvalidOperationException() throw() {} + + std::string message; + + _InvalidOperationException__isset __isset; + + void __set_message(const std::string& val) { + message = 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; + +}; + +void swap(InvalidOperationException &a, InvalidOperationException &b); + +typedef struct _ConfigValSecurityException__isset { + _ConfigValSecurityException__isset() : message(false) {} + bool message; } _ConfigValSecurityException__isset; class ConfigValSecurityException : public ::apache::thrift::TException { @@ -3419,6 +4529,178 @@ class InvalidInputException : public ::apache::thrift::TException { void swap(InvalidInputException &a, InvalidInputException &b); +typedef struct _NoSuchTxnException__isset { + _NoSuchTxnException__isset() : message(false) {} + bool message; +} _NoSuchTxnException__isset; + +class NoSuchTxnException : public ::apache::thrift::TException { + public: + + static const char* ascii_fingerprint; // = "EFB929595D312AC8F305D5A794CFEDA1"; + static const uint8_t binary_fingerprint[16]; // = {0xEF,0xB9,0x29,0x59,0x5D,0x31,0x2A,0xC8,0xF3,0x05,0xD5,0xA7,0x94,0xCF,0xED,0xA1}; + + NoSuchTxnException() : message() { + } + + virtual ~NoSuchTxnException() throw() {} + + std::string message; + + _NoSuchTxnException__isset __isset; + + void __set_message(const std::string& val) { + message = 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; + +}; + +void swap(NoSuchTxnException &a, NoSuchTxnException &b); + +typedef struct _TxnAbortedException__isset { + _TxnAbortedException__isset() : message(false) {} + bool message; +} _TxnAbortedException__isset; + +class TxnAbortedException : public ::apache::thrift::TException { + public: + + static const char* ascii_fingerprint; // = "EFB929595D312AC8F305D5A794CFEDA1"; + static const uint8_t binary_fingerprint[16]; // = {0xEF,0xB9,0x29,0x59,0x5D,0x31,0x2A,0xC8,0xF3,0x05,0xD5,0xA7,0x94,0xCF,0xED,0xA1}; + + TxnAbortedException() : message() { + } + + virtual ~TxnAbortedException() throw() {} + + std::string message; + + _TxnAbortedException__isset __isset; + + void __set_message(const std::string& val) { + message = 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; + +}; + +void swap(TxnAbortedException &a, TxnAbortedException &b); + +typedef struct _TxnOpenException__isset { + _TxnOpenException__isset() : message(false) {} + bool message; +} _TxnOpenException__isset; + +class TxnOpenException : public ::apache::thrift::TException { + public: + + static const char* ascii_fingerprint; // = "EFB929595D312AC8F305D5A794CFEDA1"; + static const uint8_t binary_fingerprint[16]; // = {0xEF,0xB9,0x29,0x59,0x5D,0x31,0x2A,0xC8,0xF3,0x05,0xD5,0xA7,0x94,0xCF,0xED,0xA1}; + + TxnOpenException() : message() { + } + + virtual ~TxnOpenException() throw() {} + + std::string message; + + _TxnOpenException__isset __isset; + + void __set_message(const std::string& val) { + message = 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; + +}; + +void swap(TxnOpenException &a, TxnOpenException &b); + +typedef struct _NoSuchLockException__isset { + _NoSuchLockException__isset() : message(false) {} + bool message; +} _NoSuchLockException__isset; + +class NoSuchLockException : public ::apache::thrift::TException { + public: + + static const char* ascii_fingerprint; // = "EFB929595D312AC8F305D5A794CFEDA1"; + static const uint8_t binary_fingerprint[16]; // = {0xEF,0xB9,0x29,0x59,0x5D,0x31,0x2A,0xC8,0xF3,0x05,0xD5,0xA7,0x94,0xCF,0xED,0xA1}; + + NoSuchLockException() : message() { + } + + virtual ~NoSuchLockException() throw() {} + + std::string message; + + _NoSuchLockException__isset __isset; + + void __set_message(const std::string& val) { + message = 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; + +}; + +void swap(NoSuchLockException &a, NoSuchLockException &b); + }}} // namespace #endif diff --git metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AbortTxnRequest.java metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AbortTxnRequest.java new file mode 100644 index 0000000..5d44585 --- /dev/null +++ metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AbortTxnRequest.java @@ -0,0 +1,383 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hadoop.hive.metastore.api; + +import org.apache.commons.lang.builder.HashCodeBuilder; +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class AbortTxnRequest implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("AbortTxnRequest"); + + private static final org.apache.thrift.protocol.TField TXNID_FIELD_DESC = new org.apache.thrift.protocol.TField("txnid", org.apache.thrift.protocol.TType.I64, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new AbortTxnRequestStandardSchemeFactory()); + schemes.put(TupleScheme.class, new AbortTxnRequestTupleSchemeFactory()); + } + + private long txnid; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + TXNID((short)1, "txnid"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // TXNID + return TXNID; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + private static final int __TXNID_ISSET_ID = 0; + private byte __isset_bitfield = 0; + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.TXNID, new org.apache.thrift.meta_data.FieldMetaData("txnid", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(AbortTxnRequest.class, metaDataMap); + } + + public AbortTxnRequest() { + } + + public AbortTxnRequest( + long txnid) + { + this(); + this.txnid = txnid; + setTxnidIsSet(true); + } + + /** + * Performs a deep copy on other. + */ + public AbortTxnRequest(AbortTxnRequest other) { + __isset_bitfield = other.__isset_bitfield; + this.txnid = other.txnid; + } + + public AbortTxnRequest deepCopy() { + return new AbortTxnRequest(this); + } + + @Override + public void clear() { + setTxnidIsSet(false); + this.txnid = 0; + } + + public long getTxnid() { + return this.txnid; + } + + public void setTxnid(long txnid) { + this.txnid = txnid; + setTxnidIsSet(true); + } + + public void unsetTxnid() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __TXNID_ISSET_ID); + } + + /** Returns true if field txnid is set (has been assigned a value) and false otherwise */ + public boolean isSetTxnid() { + return EncodingUtils.testBit(__isset_bitfield, __TXNID_ISSET_ID); + } + + public void setTxnidIsSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __TXNID_ISSET_ID, value); + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case TXNID: + if (value == null) { + unsetTxnid(); + } else { + setTxnid((Long)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case TXNID: + return Long.valueOf(getTxnid()); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case TXNID: + return isSetTxnid(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof AbortTxnRequest) + return this.equals((AbortTxnRequest)that); + return false; + } + + public boolean equals(AbortTxnRequest that) { + if (that == null) + return false; + + boolean this_present_txnid = true; + boolean that_present_txnid = true; + if (this_present_txnid || that_present_txnid) { + if (!(this_present_txnid && that_present_txnid)) + return false; + if (this.txnid != that.txnid) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_txnid = true; + builder.append(present_txnid); + if (present_txnid) + builder.append(txnid); + + return builder.toHashCode(); + } + + public int compareTo(AbortTxnRequest other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + AbortTxnRequest typedOther = (AbortTxnRequest)other; + + lastComparison = Boolean.valueOf(isSetTxnid()).compareTo(typedOther.isSetTxnid()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetTxnid()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.txnid, typedOther.txnid); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("AbortTxnRequest("); + boolean first = true; + + sb.append("txnid:"); + sb.append(this.txnid); + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetTxnid()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'txnid' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. + __isset_bitfield = 0; + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class AbortTxnRequestStandardSchemeFactory implements SchemeFactory { + public AbortTxnRequestStandardScheme getScheme() { + return new AbortTxnRequestStandardScheme(); + } + } + + private static class AbortTxnRequestStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, AbortTxnRequest struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // TXNID + if (schemeField.type == org.apache.thrift.protocol.TType.I64) { + struct.txnid = iprot.readI64(); + struct.setTxnidIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, AbortTxnRequest struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + oprot.writeFieldBegin(TXNID_FIELD_DESC); + oprot.writeI64(struct.txnid); + oprot.writeFieldEnd(); + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class AbortTxnRequestTupleSchemeFactory implements SchemeFactory { + public AbortTxnRequestTupleScheme getScheme() { + return new AbortTxnRequestTupleScheme(); + } + } + + private static class AbortTxnRequestTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, AbortTxnRequest struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + oprot.writeI64(struct.txnid); + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, AbortTxnRequest struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.txnid = iprot.readI64(); + struct.setTxnidIsSet(true); + } + } + +} + diff --git metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddPartitionsRequest.java metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddPartitionsRequest.java index 9674f66..c27a36e 100644 --- metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddPartitionsRequest.java +++ metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddPartitionsRequest.java @@ -704,7 +704,7 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, AddPartitionsReques struct.parts = new ArrayList(_list322.size); for (int _i323 = 0; _i323 < _list322.size; ++_i323) { - Partition _elem324; // optional + Partition _elem324; // required _elem324 = new Partition(); _elem324.read(iprot); struct.parts.add(_elem324); @@ -824,7 +824,7 @@ public void read(org.apache.thrift.protocol.TProtocol prot, AddPartitionsRequest struct.parts = new ArrayList(_list327.size); for (int _i328 = 0; _i328 < _list327.size; ++_i328) { - Partition _elem329; // optional + Partition _elem329; // required _elem329 = new Partition(); _elem329.read(iprot); struct.parts.add(_elem329); diff --git metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddPartitionsResult.java metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddPartitionsResult.java index 022b1a3..160b4bb 100644 --- metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddPartitionsResult.java +++ metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddPartitionsResult.java @@ -346,7 +346,7 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, AddPartitionsResult struct.partitions = new ArrayList(_list314.size); for (int _i315 = 0; _i315 < _list314.size; ++_i315) { - Partition _elem316; // optional + Partition _elem316; // required _elem316 = new Partition(); _elem316.read(iprot); struct.partitions.add(_elem316); @@ -428,7 +428,7 @@ public void read(org.apache.thrift.protocol.TProtocol prot, AddPartitionsResult struct.partitions = new ArrayList(_list319.size); for (int _i320 = 0; _i320 < _list319.size; ++_i320) { - Partition _elem321; // optional + Partition _elem321; // required _elem321 = new Partition(); _elem321.read(iprot); struct.partitions.add(_elem321); diff --git metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CheckLockRequest.java metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CheckLockRequest.java new file mode 100644 index 0000000..ae34203 --- /dev/null +++ metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CheckLockRequest.java @@ -0,0 +1,383 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hadoop.hive.metastore.api; + +import org.apache.commons.lang.builder.HashCodeBuilder; +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class CheckLockRequest implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("CheckLockRequest"); + + private static final org.apache.thrift.protocol.TField LOCKID_FIELD_DESC = new org.apache.thrift.protocol.TField("lockid", org.apache.thrift.protocol.TType.I64, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new CheckLockRequestStandardSchemeFactory()); + schemes.put(TupleScheme.class, new CheckLockRequestTupleSchemeFactory()); + } + + private long lockid; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + LOCKID((short)1, "lockid"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // LOCKID + return LOCKID; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + private static final int __LOCKID_ISSET_ID = 0; + private byte __isset_bitfield = 0; + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.LOCKID, new org.apache.thrift.meta_data.FieldMetaData("lockid", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(CheckLockRequest.class, metaDataMap); + } + + public CheckLockRequest() { + } + + public CheckLockRequest( + long lockid) + { + this(); + this.lockid = lockid; + setLockidIsSet(true); + } + + /** + * Performs a deep copy on other. + */ + public CheckLockRequest(CheckLockRequest other) { + __isset_bitfield = other.__isset_bitfield; + this.lockid = other.lockid; + } + + public CheckLockRequest deepCopy() { + return new CheckLockRequest(this); + } + + @Override + public void clear() { + setLockidIsSet(false); + this.lockid = 0; + } + + public long getLockid() { + return this.lockid; + } + + public void setLockid(long lockid) { + this.lockid = lockid; + setLockidIsSet(true); + } + + public void unsetLockid() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __LOCKID_ISSET_ID); + } + + /** Returns true if field lockid is set (has been assigned a value) and false otherwise */ + public boolean isSetLockid() { + return EncodingUtils.testBit(__isset_bitfield, __LOCKID_ISSET_ID); + } + + public void setLockidIsSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __LOCKID_ISSET_ID, value); + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case LOCKID: + if (value == null) { + unsetLockid(); + } else { + setLockid((Long)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case LOCKID: + return Long.valueOf(getLockid()); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case LOCKID: + return isSetLockid(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof CheckLockRequest) + return this.equals((CheckLockRequest)that); + return false; + } + + public boolean equals(CheckLockRequest that) { + if (that == null) + return false; + + boolean this_present_lockid = true; + boolean that_present_lockid = true; + if (this_present_lockid || that_present_lockid) { + if (!(this_present_lockid && that_present_lockid)) + return false; + if (this.lockid != that.lockid) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_lockid = true; + builder.append(present_lockid); + if (present_lockid) + builder.append(lockid); + + return builder.toHashCode(); + } + + public int compareTo(CheckLockRequest other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + CheckLockRequest typedOther = (CheckLockRequest)other; + + lastComparison = Boolean.valueOf(isSetLockid()).compareTo(typedOther.isSetLockid()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetLockid()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.lockid, typedOther.lockid); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("CheckLockRequest("); + boolean first = true; + + sb.append("lockid:"); + sb.append(this.lockid); + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetLockid()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'lockid' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. + __isset_bitfield = 0; + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class CheckLockRequestStandardSchemeFactory implements SchemeFactory { + public CheckLockRequestStandardScheme getScheme() { + return new CheckLockRequestStandardScheme(); + } + } + + private static class CheckLockRequestStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, CheckLockRequest struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // LOCKID + if (schemeField.type == org.apache.thrift.protocol.TType.I64) { + struct.lockid = iprot.readI64(); + struct.setLockidIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, CheckLockRequest struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + oprot.writeFieldBegin(LOCKID_FIELD_DESC); + oprot.writeI64(struct.lockid); + oprot.writeFieldEnd(); + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class CheckLockRequestTupleSchemeFactory implements SchemeFactory { + public CheckLockRequestTupleScheme getScheme() { + return new CheckLockRequestTupleScheme(); + } + } + + private static class CheckLockRequestTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, CheckLockRequest struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + oprot.writeI64(struct.lockid); + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, CheckLockRequest struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.lockid = iprot.readI64(); + struct.setLockidIsSet(true); + } + } + +} + diff --git metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ColumnStatistics.java metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ColumnStatistics.java index 6cdada6..5eefdd9 100644 --- metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ColumnStatistics.java +++ metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ColumnStatistics.java @@ -451,7 +451,7 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, ColumnStatistics st struct.statsObj = new ArrayList(_list220.size); for (int _i221 = 0; _i221 < _list220.size; ++_i221) { - ColumnStatisticsObj _elem222; // optional + ColumnStatisticsObj _elem222; // required _elem222 = new ColumnStatisticsObj(); _elem222.read(iprot); struct.statsObj.add(_elem222); @@ -531,7 +531,7 @@ public void read(org.apache.thrift.protocol.TProtocol prot, ColumnStatistics str struct.statsObj = new ArrayList(_list225.size); for (int _i226 = 0; _i226 < _list225.size; ++_i226) { - ColumnStatisticsObj _elem227; // optional + ColumnStatisticsObj _elem227; // required _elem227 = new ColumnStatisticsObj(); _elem227.read(iprot); struct.statsObj.add(_elem227); diff --git metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CommitTxnRequest.java metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CommitTxnRequest.java new file mode 100644 index 0000000..aeb2421 --- /dev/null +++ metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CommitTxnRequest.java @@ -0,0 +1,383 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hadoop.hive.metastore.api; + +import org.apache.commons.lang.builder.HashCodeBuilder; +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class CommitTxnRequest implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("CommitTxnRequest"); + + private static final org.apache.thrift.protocol.TField TXNID_FIELD_DESC = new org.apache.thrift.protocol.TField("txnid", org.apache.thrift.protocol.TType.I64, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new CommitTxnRequestStandardSchemeFactory()); + schemes.put(TupleScheme.class, new CommitTxnRequestTupleSchemeFactory()); + } + + private long txnid; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + TXNID((short)1, "txnid"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // TXNID + return TXNID; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + private static final int __TXNID_ISSET_ID = 0; + private byte __isset_bitfield = 0; + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.TXNID, new org.apache.thrift.meta_data.FieldMetaData("txnid", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(CommitTxnRequest.class, metaDataMap); + } + + public CommitTxnRequest() { + } + + public CommitTxnRequest( + long txnid) + { + this(); + this.txnid = txnid; + setTxnidIsSet(true); + } + + /** + * Performs a deep copy on other. + */ + public CommitTxnRequest(CommitTxnRequest other) { + __isset_bitfield = other.__isset_bitfield; + this.txnid = other.txnid; + } + + public CommitTxnRequest deepCopy() { + return new CommitTxnRequest(this); + } + + @Override + public void clear() { + setTxnidIsSet(false); + this.txnid = 0; + } + + public long getTxnid() { + return this.txnid; + } + + public void setTxnid(long txnid) { + this.txnid = txnid; + setTxnidIsSet(true); + } + + public void unsetTxnid() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __TXNID_ISSET_ID); + } + + /** Returns true if field txnid is set (has been assigned a value) and false otherwise */ + public boolean isSetTxnid() { + return EncodingUtils.testBit(__isset_bitfield, __TXNID_ISSET_ID); + } + + public void setTxnidIsSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __TXNID_ISSET_ID, value); + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case TXNID: + if (value == null) { + unsetTxnid(); + } else { + setTxnid((Long)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case TXNID: + return Long.valueOf(getTxnid()); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case TXNID: + return isSetTxnid(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof CommitTxnRequest) + return this.equals((CommitTxnRequest)that); + return false; + } + + public boolean equals(CommitTxnRequest that) { + if (that == null) + return false; + + boolean this_present_txnid = true; + boolean that_present_txnid = true; + if (this_present_txnid || that_present_txnid) { + if (!(this_present_txnid && that_present_txnid)) + return false; + if (this.txnid != that.txnid) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_txnid = true; + builder.append(present_txnid); + if (present_txnid) + builder.append(txnid); + + return builder.toHashCode(); + } + + public int compareTo(CommitTxnRequest other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + CommitTxnRequest typedOther = (CommitTxnRequest)other; + + lastComparison = Boolean.valueOf(isSetTxnid()).compareTo(typedOther.isSetTxnid()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetTxnid()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.txnid, typedOther.txnid); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("CommitTxnRequest("); + boolean first = true; + + sb.append("txnid:"); + sb.append(this.txnid); + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetTxnid()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'txnid' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. + __isset_bitfield = 0; + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class CommitTxnRequestStandardSchemeFactory implements SchemeFactory { + public CommitTxnRequestStandardScheme getScheme() { + return new CommitTxnRequestStandardScheme(); + } + } + + private static class CommitTxnRequestStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, CommitTxnRequest struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // TXNID + if (schemeField.type == org.apache.thrift.protocol.TType.I64) { + struct.txnid = iprot.readI64(); + struct.setTxnidIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, CommitTxnRequest struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + oprot.writeFieldBegin(TXNID_FIELD_DESC); + oprot.writeI64(struct.txnid); + oprot.writeFieldEnd(); + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class CommitTxnRequestTupleSchemeFactory implements SchemeFactory { + public CommitTxnRequestTupleScheme getScheme() { + return new CommitTxnRequestTupleScheme(); + } + } + + private static class CommitTxnRequestTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, CommitTxnRequest struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + oprot.writeI64(struct.txnid); + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, CommitTxnRequest struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.txnid = iprot.readI64(); + struct.setTxnidIsSet(true); + } + } + +} + diff --git metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CompactionRequest.java metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CompactionRequest.java new file mode 100644 index 0000000..939c15a --- /dev/null +++ metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CompactionRequest.java @@ -0,0 +1,815 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hadoop.hive.metastore.api; + +import org.apache.commons.lang.builder.HashCodeBuilder; +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class CompactionRequest implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("CompactionRequest"); + + private static final org.apache.thrift.protocol.TField DBNAME_FIELD_DESC = new org.apache.thrift.protocol.TField("dbname", org.apache.thrift.protocol.TType.STRING, (short)1); + private static final org.apache.thrift.protocol.TField TABLENAME_FIELD_DESC = new org.apache.thrift.protocol.TField("tablename", org.apache.thrift.protocol.TType.STRING, (short)2); + private static final org.apache.thrift.protocol.TField PARTITIONNAME_FIELD_DESC = new org.apache.thrift.protocol.TField("partitionname", org.apache.thrift.protocol.TType.STRING, (short)3); + private static final org.apache.thrift.protocol.TField TYPE_FIELD_DESC = new org.apache.thrift.protocol.TField("type", org.apache.thrift.protocol.TType.I32, (short)4); + private static final org.apache.thrift.protocol.TField RUNAS_FIELD_DESC = new org.apache.thrift.protocol.TField("runas", org.apache.thrift.protocol.TType.STRING, (short)5); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new CompactionRequestStandardSchemeFactory()); + schemes.put(TupleScheme.class, new CompactionRequestTupleSchemeFactory()); + } + + private String dbname; // required + private String tablename; // required + private String partitionname; // optional + private CompactionType type; // required + private String runas; // optional + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + DBNAME((short)1, "dbname"), + TABLENAME((short)2, "tablename"), + PARTITIONNAME((short)3, "partitionname"), + /** + * + * @see CompactionType + */ + TYPE((short)4, "type"), + RUNAS((short)5, "runas"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // DBNAME + return DBNAME; + case 2: // TABLENAME + return TABLENAME; + case 3: // PARTITIONNAME + return PARTITIONNAME; + case 4: // TYPE + return TYPE; + case 5: // RUNAS + return RUNAS; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + private _Fields optionals[] = {_Fields.PARTITIONNAME,_Fields.RUNAS}; + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.DBNAME, new org.apache.thrift.meta_data.FieldMetaData("dbname", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + tmpMap.put(_Fields.TABLENAME, new org.apache.thrift.meta_data.FieldMetaData("tablename", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + tmpMap.put(_Fields.PARTITIONNAME, new org.apache.thrift.meta_data.FieldMetaData("partitionname", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + tmpMap.put(_Fields.TYPE, new org.apache.thrift.meta_data.FieldMetaData("type", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, CompactionType.class))); + tmpMap.put(_Fields.RUNAS, new org.apache.thrift.meta_data.FieldMetaData("runas", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(CompactionRequest.class, metaDataMap); + } + + public CompactionRequest() { + } + + public CompactionRequest( + String dbname, + String tablename, + CompactionType type) + { + this(); + this.dbname = dbname; + this.tablename = tablename; + this.type = type; + } + + /** + * Performs a deep copy on other. + */ + public CompactionRequest(CompactionRequest other) { + if (other.isSetDbname()) { + this.dbname = other.dbname; + } + if (other.isSetTablename()) { + this.tablename = other.tablename; + } + if (other.isSetPartitionname()) { + this.partitionname = other.partitionname; + } + if (other.isSetType()) { + this.type = other.type; + } + if (other.isSetRunas()) { + this.runas = other.runas; + } + } + + public CompactionRequest deepCopy() { + return new CompactionRequest(this); + } + + @Override + public void clear() { + this.dbname = null; + this.tablename = null; + this.partitionname = null; + this.type = null; + this.runas = null; + } + + public String getDbname() { + return this.dbname; + } + + public void setDbname(String dbname) { + this.dbname = dbname; + } + + public void unsetDbname() { + this.dbname = null; + } + + /** Returns true if field dbname is set (has been assigned a value) and false otherwise */ + public boolean isSetDbname() { + return this.dbname != null; + } + + public void setDbnameIsSet(boolean value) { + if (!value) { + this.dbname = null; + } + } + + public String getTablename() { + return this.tablename; + } + + public void setTablename(String tablename) { + this.tablename = tablename; + } + + public void unsetTablename() { + this.tablename = null; + } + + /** Returns true if field tablename is set (has been assigned a value) and false otherwise */ + public boolean isSetTablename() { + return this.tablename != null; + } + + public void setTablenameIsSet(boolean value) { + if (!value) { + this.tablename = null; + } + } + + public String getPartitionname() { + return this.partitionname; + } + + public void setPartitionname(String partitionname) { + this.partitionname = partitionname; + } + + public void unsetPartitionname() { + this.partitionname = null; + } + + /** Returns true if field partitionname is set (has been assigned a value) and false otherwise */ + public boolean isSetPartitionname() { + return this.partitionname != null; + } + + public void setPartitionnameIsSet(boolean value) { + if (!value) { + this.partitionname = null; + } + } + + /** + * + * @see CompactionType + */ + public CompactionType getType() { + return this.type; + } + + /** + * + * @see CompactionType + */ + public void setType(CompactionType type) { + this.type = type; + } + + public void unsetType() { + this.type = null; + } + + /** Returns true if field type is set (has been assigned a value) and false otherwise */ + public boolean isSetType() { + return this.type != null; + } + + public void setTypeIsSet(boolean value) { + if (!value) { + this.type = null; + } + } + + public String getRunas() { + return this.runas; + } + + public void setRunas(String runas) { + this.runas = runas; + } + + public void unsetRunas() { + this.runas = null; + } + + /** Returns true if field runas is set (has been assigned a value) and false otherwise */ + public boolean isSetRunas() { + return this.runas != null; + } + + public void setRunasIsSet(boolean value) { + if (!value) { + this.runas = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case DBNAME: + if (value == null) { + unsetDbname(); + } else { + setDbname((String)value); + } + break; + + case TABLENAME: + if (value == null) { + unsetTablename(); + } else { + setTablename((String)value); + } + break; + + case PARTITIONNAME: + if (value == null) { + unsetPartitionname(); + } else { + setPartitionname((String)value); + } + break; + + case TYPE: + if (value == null) { + unsetType(); + } else { + setType((CompactionType)value); + } + break; + + case RUNAS: + if (value == null) { + unsetRunas(); + } else { + setRunas((String)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case DBNAME: + return getDbname(); + + case TABLENAME: + return getTablename(); + + case PARTITIONNAME: + return getPartitionname(); + + case TYPE: + return getType(); + + case RUNAS: + return getRunas(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case DBNAME: + return isSetDbname(); + case TABLENAME: + return isSetTablename(); + case PARTITIONNAME: + return isSetPartitionname(); + case TYPE: + return isSetType(); + case RUNAS: + return isSetRunas(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof CompactionRequest) + return this.equals((CompactionRequest)that); + return false; + } + + public boolean equals(CompactionRequest that) { + if (that == null) + return false; + + boolean this_present_dbname = true && this.isSetDbname(); + boolean that_present_dbname = true && that.isSetDbname(); + if (this_present_dbname || that_present_dbname) { + if (!(this_present_dbname && that_present_dbname)) + return false; + if (!this.dbname.equals(that.dbname)) + return false; + } + + boolean this_present_tablename = true && this.isSetTablename(); + boolean that_present_tablename = true && that.isSetTablename(); + if (this_present_tablename || that_present_tablename) { + if (!(this_present_tablename && that_present_tablename)) + return false; + if (!this.tablename.equals(that.tablename)) + return false; + } + + boolean this_present_partitionname = true && this.isSetPartitionname(); + boolean that_present_partitionname = true && that.isSetPartitionname(); + if (this_present_partitionname || that_present_partitionname) { + if (!(this_present_partitionname && that_present_partitionname)) + return false; + if (!this.partitionname.equals(that.partitionname)) + return false; + } + + boolean this_present_type = true && this.isSetType(); + boolean that_present_type = true && that.isSetType(); + if (this_present_type || that_present_type) { + if (!(this_present_type && that_present_type)) + return false; + if (!this.type.equals(that.type)) + return false; + } + + boolean this_present_runas = true && this.isSetRunas(); + boolean that_present_runas = true && that.isSetRunas(); + if (this_present_runas || that_present_runas) { + if (!(this_present_runas && that_present_runas)) + return false; + if (!this.runas.equals(that.runas)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_dbname = true && (isSetDbname()); + builder.append(present_dbname); + if (present_dbname) + builder.append(dbname); + + boolean present_tablename = true && (isSetTablename()); + builder.append(present_tablename); + if (present_tablename) + builder.append(tablename); + + boolean present_partitionname = true && (isSetPartitionname()); + builder.append(present_partitionname); + if (present_partitionname) + builder.append(partitionname); + + boolean present_type = true && (isSetType()); + builder.append(present_type); + if (present_type) + builder.append(type.getValue()); + + boolean present_runas = true && (isSetRunas()); + builder.append(present_runas); + if (present_runas) + builder.append(runas); + + return builder.toHashCode(); + } + + public int compareTo(CompactionRequest other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + CompactionRequest typedOther = (CompactionRequest)other; + + lastComparison = Boolean.valueOf(isSetDbname()).compareTo(typedOther.isSetDbname()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetDbname()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.dbname, typedOther.dbname); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetTablename()).compareTo(typedOther.isSetTablename()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetTablename()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tablename, typedOther.tablename); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetPartitionname()).compareTo(typedOther.isSetPartitionname()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetPartitionname()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.partitionname, typedOther.partitionname); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetType()).compareTo(typedOther.isSetType()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetType()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.type, typedOther.type); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetRunas()).compareTo(typedOther.isSetRunas()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetRunas()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.runas, typedOther.runas); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("CompactionRequest("); + boolean first = true; + + sb.append("dbname:"); + if (this.dbname == null) { + sb.append("null"); + } else { + sb.append(this.dbname); + } + first = false; + if (!first) sb.append(", "); + sb.append("tablename:"); + if (this.tablename == null) { + sb.append("null"); + } else { + sb.append(this.tablename); + } + first = false; + if (isSetPartitionname()) { + if (!first) sb.append(", "); + sb.append("partitionname:"); + if (this.partitionname == null) { + sb.append("null"); + } else { + sb.append(this.partitionname); + } + first = false; + } + if (!first) sb.append(", "); + sb.append("type:"); + if (this.type == null) { + sb.append("null"); + } else { + sb.append(this.type); + } + first = false; + if (isSetRunas()) { + if (!first) sb.append(", "); + sb.append("runas:"); + if (this.runas == null) { + sb.append("null"); + } else { + sb.append(this.runas); + } + first = false; + } + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetDbname()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'dbname' is unset! Struct:" + toString()); + } + + if (!isSetTablename()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'tablename' is unset! Struct:" + toString()); + } + + if (!isSetType()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'type' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class CompactionRequestStandardSchemeFactory implements SchemeFactory { + public CompactionRequestStandardScheme getScheme() { + return new CompactionRequestStandardScheme(); + } + } + + private static class CompactionRequestStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, CompactionRequest struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // DBNAME + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.dbname = iprot.readString(); + struct.setDbnameIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // TABLENAME + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.tablename = iprot.readString(); + struct.setTablenameIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 3: // PARTITIONNAME + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.partitionname = iprot.readString(); + struct.setPartitionnameIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 4: // TYPE + if (schemeField.type == org.apache.thrift.protocol.TType.I32) { + struct.type = CompactionType.findByValue(iprot.readI32()); + struct.setTypeIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 5: // RUNAS + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.runas = iprot.readString(); + struct.setRunasIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, CompactionRequest struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.dbname != null) { + oprot.writeFieldBegin(DBNAME_FIELD_DESC); + oprot.writeString(struct.dbname); + oprot.writeFieldEnd(); + } + if (struct.tablename != null) { + oprot.writeFieldBegin(TABLENAME_FIELD_DESC); + oprot.writeString(struct.tablename); + oprot.writeFieldEnd(); + } + if (struct.partitionname != null) { + if (struct.isSetPartitionname()) { + oprot.writeFieldBegin(PARTITIONNAME_FIELD_DESC); + oprot.writeString(struct.partitionname); + oprot.writeFieldEnd(); + } + } + if (struct.type != null) { + oprot.writeFieldBegin(TYPE_FIELD_DESC); + oprot.writeI32(struct.type.getValue()); + oprot.writeFieldEnd(); + } + if (struct.runas != null) { + if (struct.isSetRunas()) { + oprot.writeFieldBegin(RUNAS_FIELD_DESC); + oprot.writeString(struct.runas); + oprot.writeFieldEnd(); + } + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class CompactionRequestTupleSchemeFactory implements SchemeFactory { + public CompactionRequestTupleScheme getScheme() { + return new CompactionRequestTupleScheme(); + } + } + + private static class CompactionRequestTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, CompactionRequest struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + oprot.writeString(struct.dbname); + oprot.writeString(struct.tablename); + oprot.writeI32(struct.type.getValue()); + BitSet optionals = new BitSet(); + if (struct.isSetPartitionname()) { + optionals.set(0); + } + if (struct.isSetRunas()) { + optionals.set(1); + } + oprot.writeBitSet(optionals, 2); + if (struct.isSetPartitionname()) { + oprot.writeString(struct.partitionname); + } + if (struct.isSetRunas()) { + oprot.writeString(struct.runas); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, CompactionRequest struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.dbname = iprot.readString(); + struct.setDbnameIsSet(true); + struct.tablename = iprot.readString(); + struct.setTablenameIsSet(true); + struct.type = CompactionType.findByValue(iprot.readI32()); + struct.setTypeIsSet(true); + BitSet incoming = iprot.readBitSet(2); + if (incoming.get(0)) { + struct.partitionname = iprot.readString(); + struct.setPartitionnameIsSet(true); + } + if (incoming.get(1)) { + struct.runas = iprot.readString(); + struct.setRunasIsSet(true); + } + } + } + +} + diff --git metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CompactionType.java metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CompactionType.java new file mode 100644 index 0000000..e4ecbcc --- /dev/null +++ metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CompactionType.java @@ -0,0 +1,45 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hadoop.hive.metastore.api; + + +import java.util.Map; +import java.util.HashMap; +import org.apache.thrift.TEnum; + +public enum CompactionType implements org.apache.thrift.TEnum { + MINOR(1), + MAJOR(2); + + private final int value; + + private CompactionType(int value) { + this.value = value; + } + + /** + * Get the integer value of this enum value, as defined in the Thrift IDL. + */ + public int getValue() { + return value; + } + + /** + * Find a the enum type by its integer value, as defined in the Thrift IDL. + * @return null if the value is not found. + */ + public static CompactionType findByValue(int value) { + switch (value) { + case 1: + return MINOR; + case 2: + return MAJOR; + default: + return null; + } + } +} diff --git metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/DropPartitionsResult.java metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/DropPartitionsResult.java index 7560e66..a394c38 100644 --- metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/DropPartitionsResult.java +++ metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/DropPartitionsResult.java @@ -346,7 +346,7 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, DropPartitionsResul struct.partitions = new ArrayList(_list330.size); for (int _i331 = 0; _i331 < _list330.size; ++_i331) { - Partition _elem332; // optional + Partition _elem332; // required _elem332 = new Partition(); _elem332.read(iprot); struct.partitions.add(_elem332); @@ -428,7 +428,7 @@ public void read(org.apache.thrift.protocol.TProtocol prot, DropPartitionsResult struct.partitions = new ArrayList(_list335.size); for (int _i336 = 0; _i336 < _list335.size; ++_i336) { - Partition _elem337; // optional + Partition _elem337; // required _elem337 = new Partition(); _elem337.read(iprot); struct.partitions.add(_elem337); diff --git metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Function.java metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Function.java index 9f895ad..8088f23 100644 --- metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Function.java +++ metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Function.java @@ -997,7 +997,7 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, Function struct) th struct.resourceUris = new ArrayList(_list354.size); for (int _i355 = 0; _i355 < _list354.size; ++_i355) { - ResourceUri _elem356; // optional + ResourceUri _elem356; // required _elem356 = new ResourceUri(); _elem356.read(iprot); struct.resourceUris.add(_elem356); @@ -1180,7 +1180,7 @@ public void read(org.apache.thrift.protocol.TProtocol prot, Function struct) thr struct.resourceUris = new ArrayList(_list359.size); for (int _i360 = 0; _i360 < _list359.size; ++_i360) { - ResourceUri _elem361; // optional + ResourceUri _elem361; // required _elem361 = new ResourceUri(); _elem361.read(iprot); struct.resourceUris.add(_elem361); diff --git metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetOpenTxnsInfoResponse.java metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetOpenTxnsInfoResponse.java new file mode 100644 index 0000000..66669d6 --- /dev/null +++ metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetOpenTxnsInfoResponse.java @@ -0,0 +1,538 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hadoop.hive.metastore.api; + +import org.apache.commons.lang.builder.HashCodeBuilder; +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class GetOpenTxnsInfoResponse implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("GetOpenTxnsInfoResponse"); + + private static final org.apache.thrift.protocol.TField TXN_HIGH_WATER_MARK_FIELD_DESC = new org.apache.thrift.protocol.TField("txn_high_water_mark", org.apache.thrift.protocol.TType.I64, (short)1); + private static final org.apache.thrift.protocol.TField OPEN_TXNS_FIELD_DESC = new org.apache.thrift.protocol.TField("open_txns", org.apache.thrift.protocol.TType.LIST, (short)2); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new GetOpenTxnsInfoResponseStandardSchemeFactory()); + schemes.put(TupleScheme.class, new GetOpenTxnsInfoResponseTupleSchemeFactory()); + } + + private long txn_high_water_mark; // required + private List open_txns; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + TXN_HIGH_WATER_MARK((short)1, "txn_high_water_mark"), + OPEN_TXNS((short)2, "open_txns"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // TXN_HIGH_WATER_MARK + return TXN_HIGH_WATER_MARK; + case 2: // OPEN_TXNS + return OPEN_TXNS; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + private static final int __TXN_HIGH_WATER_MARK_ISSET_ID = 0; + private byte __isset_bitfield = 0; + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.TXN_HIGH_WATER_MARK, new org.apache.thrift.meta_data.FieldMetaData("txn_high_water_mark", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64))); + tmpMap.put(_Fields.OPEN_TXNS, new org.apache.thrift.meta_data.FieldMetaData("open_txns", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TxnInfo.class)))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(GetOpenTxnsInfoResponse.class, metaDataMap); + } + + public GetOpenTxnsInfoResponse() { + } + + public GetOpenTxnsInfoResponse( + long txn_high_water_mark, + List open_txns) + { + this(); + this.txn_high_water_mark = txn_high_water_mark; + setTxn_high_water_markIsSet(true); + this.open_txns = open_txns; + } + + /** + * Performs a deep copy on other. + */ + public GetOpenTxnsInfoResponse(GetOpenTxnsInfoResponse other) { + __isset_bitfield = other.__isset_bitfield; + this.txn_high_water_mark = other.txn_high_water_mark; + if (other.isSetOpen_txns()) { + List __this__open_txns = new ArrayList(); + for (TxnInfo other_element : other.open_txns) { + __this__open_txns.add(new TxnInfo(other_element)); + } + this.open_txns = __this__open_txns; + } + } + + public GetOpenTxnsInfoResponse deepCopy() { + return new GetOpenTxnsInfoResponse(this); + } + + @Override + public void clear() { + setTxn_high_water_markIsSet(false); + this.txn_high_water_mark = 0; + this.open_txns = null; + } + + public long getTxn_high_water_mark() { + return this.txn_high_water_mark; + } + + public void setTxn_high_water_mark(long txn_high_water_mark) { + this.txn_high_water_mark = txn_high_water_mark; + setTxn_high_water_markIsSet(true); + } + + public void unsetTxn_high_water_mark() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __TXN_HIGH_WATER_MARK_ISSET_ID); + } + + /** Returns true if field txn_high_water_mark is set (has been assigned a value) and false otherwise */ + public boolean isSetTxn_high_water_mark() { + return EncodingUtils.testBit(__isset_bitfield, __TXN_HIGH_WATER_MARK_ISSET_ID); + } + + public void setTxn_high_water_markIsSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __TXN_HIGH_WATER_MARK_ISSET_ID, value); + } + + public int getOpen_txnsSize() { + return (this.open_txns == null) ? 0 : this.open_txns.size(); + } + + public java.util.Iterator getOpen_txnsIterator() { + return (this.open_txns == null) ? null : this.open_txns.iterator(); + } + + public void addToOpen_txns(TxnInfo elem) { + if (this.open_txns == null) { + this.open_txns = new ArrayList(); + } + this.open_txns.add(elem); + } + + public List getOpen_txns() { + return this.open_txns; + } + + public void setOpen_txns(List open_txns) { + this.open_txns = open_txns; + } + + public void unsetOpen_txns() { + this.open_txns = null; + } + + /** Returns true if field open_txns is set (has been assigned a value) and false otherwise */ + public boolean isSetOpen_txns() { + return this.open_txns != null; + } + + public void setOpen_txnsIsSet(boolean value) { + if (!value) { + this.open_txns = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case TXN_HIGH_WATER_MARK: + if (value == null) { + unsetTxn_high_water_mark(); + } else { + setTxn_high_water_mark((Long)value); + } + break; + + case OPEN_TXNS: + if (value == null) { + unsetOpen_txns(); + } else { + setOpen_txns((List)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case TXN_HIGH_WATER_MARK: + return Long.valueOf(getTxn_high_water_mark()); + + case OPEN_TXNS: + return getOpen_txns(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case TXN_HIGH_WATER_MARK: + return isSetTxn_high_water_mark(); + case OPEN_TXNS: + return isSetOpen_txns(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof GetOpenTxnsInfoResponse) + return this.equals((GetOpenTxnsInfoResponse)that); + return false; + } + + public boolean equals(GetOpenTxnsInfoResponse that) { + if (that == null) + return false; + + boolean this_present_txn_high_water_mark = true; + boolean that_present_txn_high_water_mark = true; + if (this_present_txn_high_water_mark || that_present_txn_high_water_mark) { + if (!(this_present_txn_high_water_mark && that_present_txn_high_water_mark)) + return false; + if (this.txn_high_water_mark != that.txn_high_water_mark) + return false; + } + + boolean this_present_open_txns = true && this.isSetOpen_txns(); + boolean that_present_open_txns = true && that.isSetOpen_txns(); + if (this_present_open_txns || that_present_open_txns) { + if (!(this_present_open_txns && that_present_open_txns)) + return false; + if (!this.open_txns.equals(that.open_txns)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_txn_high_water_mark = true; + builder.append(present_txn_high_water_mark); + if (present_txn_high_water_mark) + builder.append(txn_high_water_mark); + + boolean present_open_txns = true && (isSetOpen_txns()); + builder.append(present_open_txns); + if (present_open_txns) + builder.append(open_txns); + + return builder.toHashCode(); + } + + public int compareTo(GetOpenTxnsInfoResponse other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + GetOpenTxnsInfoResponse typedOther = (GetOpenTxnsInfoResponse)other; + + lastComparison = Boolean.valueOf(isSetTxn_high_water_mark()).compareTo(typedOther.isSetTxn_high_water_mark()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetTxn_high_water_mark()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.txn_high_water_mark, typedOther.txn_high_water_mark); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetOpen_txns()).compareTo(typedOther.isSetOpen_txns()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetOpen_txns()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.open_txns, typedOther.open_txns); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("GetOpenTxnsInfoResponse("); + boolean first = true; + + sb.append("txn_high_water_mark:"); + sb.append(this.txn_high_water_mark); + first = false; + if (!first) sb.append(", "); + sb.append("open_txns:"); + if (this.open_txns == null) { + sb.append("null"); + } else { + sb.append(this.open_txns); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetTxn_high_water_mark()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'txn_high_water_mark' is unset! Struct:" + toString()); + } + + if (!isSetOpen_txns()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'open_txns' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. + __isset_bitfield = 0; + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class GetOpenTxnsInfoResponseStandardSchemeFactory implements SchemeFactory { + public GetOpenTxnsInfoResponseStandardScheme getScheme() { + return new GetOpenTxnsInfoResponseStandardScheme(); + } + } + + private static class GetOpenTxnsInfoResponseStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, GetOpenTxnsInfoResponse struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // TXN_HIGH_WATER_MARK + if (schemeField.type == org.apache.thrift.protocol.TType.I64) { + struct.txn_high_water_mark = iprot.readI64(); + struct.setTxn_high_water_markIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // OPEN_TXNS + if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { + { + org.apache.thrift.protocol.TList _list362 = iprot.readListBegin(); + struct.open_txns = new ArrayList(_list362.size); + for (int _i363 = 0; _i363 < _list362.size; ++_i363) + { + TxnInfo _elem364; // required + _elem364 = new TxnInfo(); + _elem364.read(iprot); + struct.open_txns.add(_elem364); + } + iprot.readListEnd(); + } + struct.setOpen_txnsIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, GetOpenTxnsInfoResponse struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + oprot.writeFieldBegin(TXN_HIGH_WATER_MARK_FIELD_DESC); + oprot.writeI64(struct.txn_high_water_mark); + oprot.writeFieldEnd(); + if (struct.open_txns != null) { + oprot.writeFieldBegin(OPEN_TXNS_FIELD_DESC); + { + oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.open_txns.size())); + for (TxnInfo _iter365 : struct.open_txns) + { + _iter365.write(oprot); + } + oprot.writeListEnd(); + } + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class GetOpenTxnsInfoResponseTupleSchemeFactory implements SchemeFactory { + public GetOpenTxnsInfoResponseTupleScheme getScheme() { + return new GetOpenTxnsInfoResponseTupleScheme(); + } + } + + private static class GetOpenTxnsInfoResponseTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, GetOpenTxnsInfoResponse struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + oprot.writeI64(struct.txn_high_water_mark); + { + oprot.writeI32(struct.open_txns.size()); + for (TxnInfo _iter366 : struct.open_txns) + { + _iter366.write(oprot); + } + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, GetOpenTxnsInfoResponse struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.txn_high_water_mark = iprot.readI64(); + struct.setTxn_high_water_markIsSet(true); + { + org.apache.thrift.protocol.TList _list367 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); + struct.open_txns = new ArrayList(_list367.size); + for (int _i368 = 0; _i368 < _list367.size; ++_i368) + { + TxnInfo _elem369; // required + _elem369 = new TxnInfo(); + _elem369.read(iprot); + struct.open_txns.add(_elem369); + } + } + struct.setOpen_txnsIsSet(true); + } + } + +} + diff --git metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetOpenTxnsResponse.java metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetOpenTxnsResponse.java new file mode 100644 index 0000000..1bfdf60 --- /dev/null +++ metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetOpenTxnsResponse.java @@ -0,0 +1,536 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hadoop.hive.metastore.api; + +import org.apache.commons.lang.builder.HashCodeBuilder; +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class GetOpenTxnsResponse implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("GetOpenTxnsResponse"); + + private static final org.apache.thrift.protocol.TField TXN_HIGH_WATER_MARK_FIELD_DESC = new org.apache.thrift.protocol.TField("txn_high_water_mark", org.apache.thrift.protocol.TType.I64, (short)1); + private static final org.apache.thrift.protocol.TField OPEN_TXNS_FIELD_DESC = new org.apache.thrift.protocol.TField("open_txns", org.apache.thrift.protocol.TType.SET, (short)2); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new GetOpenTxnsResponseStandardSchemeFactory()); + schemes.put(TupleScheme.class, new GetOpenTxnsResponseTupleSchemeFactory()); + } + + private long txn_high_water_mark; // required + private Set open_txns; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + TXN_HIGH_WATER_MARK((short)1, "txn_high_water_mark"), + OPEN_TXNS((short)2, "open_txns"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // TXN_HIGH_WATER_MARK + return TXN_HIGH_WATER_MARK; + case 2: // OPEN_TXNS + return OPEN_TXNS; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + private static final int __TXN_HIGH_WATER_MARK_ISSET_ID = 0; + private byte __isset_bitfield = 0; + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.TXN_HIGH_WATER_MARK, new org.apache.thrift.meta_data.FieldMetaData("txn_high_water_mark", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64))); + tmpMap.put(_Fields.OPEN_TXNS, new org.apache.thrift.meta_data.FieldMetaData("open_txns", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.SetMetaData(org.apache.thrift.protocol.TType.SET, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(GetOpenTxnsResponse.class, metaDataMap); + } + + public GetOpenTxnsResponse() { + } + + public GetOpenTxnsResponse( + long txn_high_water_mark, + Set open_txns) + { + this(); + this.txn_high_water_mark = txn_high_water_mark; + setTxn_high_water_markIsSet(true); + this.open_txns = open_txns; + } + + /** + * Performs a deep copy on other. + */ + public GetOpenTxnsResponse(GetOpenTxnsResponse other) { + __isset_bitfield = other.__isset_bitfield; + this.txn_high_water_mark = other.txn_high_water_mark; + if (other.isSetOpen_txns()) { + Set __this__open_txns = new HashSet(); + for (Long other_element : other.open_txns) { + __this__open_txns.add(other_element); + } + this.open_txns = __this__open_txns; + } + } + + public GetOpenTxnsResponse deepCopy() { + return new GetOpenTxnsResponse(this); + } + + @Override + public void clear() { + setTxn_high_water_markIsSet(false); + this.txn_high_water_mark = 0; + this.open_txns = null; + } + + public long getTxn_high_water_mark() { + return this.txn_high_water_mark; + } + + public void setTxn_high_water_mark(long txn_high_water_mark) { + this.txn_high_water_mark = txn_high_water_mark; + setTxn_high_water_markIsSet(true); + } + + public void unsetTxn_high_water_mark() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __TXN_HIGH_WATER_MARK_ISSET_ID); + } + + /** Returns true if field txn_high_water_mark is set (has been assigned a value) and false otherwise */ + public boolean isSetTxn_high_water_mark() { + return EncodingUtils.testBit(__isset_bitfield, __TXN_HIGH_WATER_MARK_ISSET_ID); + } + + public void setTxn_high_water_markIsSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __TXN_HIGH_WATER_MARK_ISSET_ID, value); + } + + public int getOpen_txnsSize() { + return (this.open_txns == null) ? 0 : this.open_txns.size(); + } + + public java.util.Iterator getOpen_txnsIterator() { + return (this.open_txns == null) ? null : this.open_txns.iterator(); + } + + public void addToOpen_txns(long elem) { + if (this.open_txns == null) { + this.open_txns = new HashSet(); + } + this.open_txns.add(elem); + } + + public Set getOpen_txns() { + return this.open_txns; + } + + public void setOpen_txns(Set open_txns) { + this.open_txns = open_txns; + } + + public void unsetOpen_txns() { + this.open_txns = null; + } + + /** Returns true if field open_txns is set (has been assigned a value) and false otherwise */ + public boolean isSetOpen_txns() { + return this.open_txns != null; + } + + public void setOpen_txnsIsSet(boolean value) { + if (!value) { + this.open_txns = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case TXN_HIGH_WATER_MARK: + if (value == null) { + unsetTxn_high_water_mark(); + } else { + setTxn_high_water_mark((Long)value); + } + break; + + case OPEN_TXNS: + if (value == null) { + unsetOpen_txns(); + } else { + setOpen_txns((Set)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case TXN_HIGH_WATER_MARK: + return Long.valueOf(getTxn_high_water_mark()); + + case OPEN_TXNS: + return getOpen_txns(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case TXN_HIGH_WATER_MARK: + return isSetTxn_high_water_mark(); + case OPEN_TXNS: + return isSetOpen_txns(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof GetOpenTxnsResponse) + return this.equals((GetOpenTxnsResponse)that); + return false; + } + + public boolean equals(GetOpenTxnsResponse that) { + if (that == null) + return false; + + boolean this_present_txn_high_water_mark = true; + boolean that_present_txn_high_water_mark = true; + if (this_present_txn_high_water_mark || that_present_txn_high_water_mark) { + if (!(this_present_txn_high_water_mark && that_present_txn_high_water_mark)) + return false; + if (this.txn_high_water_mark != that.txn_high_water_mark) + return false; + } + + boolean this_present_open_txns = true && this.isSetOpen_txns(); + boolean that_present_open_txns = true && that.isSetOpen_txns(); + if (this_present_open_txns || that_present_open_txns) { + if (!(this_present_open_txns && that_present_open_txns)) + return false; + if (!this.open_txns.equals(that.open_txns)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_txn_high_water_mark = true; + builder.append(present_txn_high_water_mark); + if (present_txn_high_water_mark) + builder.append(txn_high_water_mark); + + boolean present_open_txns = true && (isSetOpen_txns()); + builder.append(present_open_txns); + if (present_open_txns) + builder.append(open_txns); + + return builder.toHashCode(); + } + + public int compareTo(GetOpenTxnsResponse other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + GetOpenTxnsResponse typedOther = (GetOpenTxnsResponse)other; + + lastComparison = Boolean.valueOf(isSetTxn_high_water_mark()).compareTo(typedOther.isSetTxn_high_water_mark()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetTxn_high_water_mark()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.txn_high_water_mark, typedOther.txn_high_water_mark); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetOpen_txns()).compareTo(typedOther.isSetOpen_txns()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetOpen_txns()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.open_txns, typedOther.open_txns); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("GetOpenTxnsResponse("); + boolean first = true; + + sb.append("txn_high_water_mark:"); + sb.append(this.txn_high_water_mark); + first = false; + if (!first) sb.append(", "); + sb.append("open_txns:"); + if (this.open_txns == null) { + sb.append("null"); + } else { + sb.append(this.open_txns); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetTxn_high_water_mark()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'txn_high_water_mark' is unset! Struct:" + toString()); + } + + if (!isSetOpen_txns()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'open_txns' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. + __isset_bitfield = 0; + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class GetOpenTxnsResponseStandardSchemeFactory implements SchemeFactory { + public GetOpenTxnsResponseStandardScheme getScheme() { + return new GetOpenTxnsResponseStandardScheme(); + } + } + + private static class GetOpenTxnsResponseStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, GetOpenTxnsResponse struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // TXN_HIGH_WATER_MARK + if (schemeField.type == org.apache.thrift.protocol.TType.I64) { + struct.txn_high_water_mark = iprot.readI64(); + struct.setTxn_high_water_markIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // OPEN_TXNS + if (schemeField.type == org.apache.thrift.protocol.TType.SET) { + { + org.apache.thrift.protocol.TSet _set370 = iprot.readSetBegin(); + struct.open_txns = new HashSet(2*_set370.size); + for (int _i371 = 0; _i371 < _set370.size; ++_i371) + { + long _elem372; // required + _elem372 = iprot.readI64(); + struct.open_txns.add(_elem372); + } + iprot.readSetEnd(); + } + struct.setOpen_txnsIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, GetOpenTxnsResponse struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + oprot.writeFieldBegin(TXN_HIGH_WATER_MARK_FIELD_DESC); + oprot.writeI64(struct.txn_high_water_mark); + oprot.writeFieldEnd(); + if (struct.open_txns != null) { + oprot.writeFieldBegin(OPEN_TXNS_FIELD_DESC); + { + oprot.writeSetBegin(new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.I64, struct.open_txns.size())); + for (long _iter373 : struct.open_txns) + { + oprot.writeI64(_iter373); + } + oprot.writeSetEnd(); + } + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class GetOpenTxnsResponseTupleSchemeFactory implements SchemeFactory { + public GetOpenTxnsResponseTupleScheme getScheme() { + return new GetOpenTxnsResponseTupleScheme(); + } + } + + private static class GetOpenTxnsResponseTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, GetOpenTxnsResponse struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + oprot.writeI64(struct.txn_high_water_mark); + { + oprot.writeI32(struct.open_txns.size()); + for (long _iter374 : struct.open_txns) + { + oprot.writeI64(_iter374); + } + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, GetOpenTxnsResponse struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.txn_high_water_mark = iprot.readI64(); + struct.setTxn_high_water_markIsSet(true); + { + org.apache.thrift.protocol.TSet _set375 = new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.I64, iprot.readI32()); + struct.open_txns = new HashSet(2*_set375.size); + for (int _i376 = 0; _i376 < _set375.size; ++_i376) + { + long _elem377; // required + _elem377 = iprot.readI64(); + struct.open_txns.add(_elem377); + } + } + struct.setOpen_txnsIsSet(true); + } + } + +} + diff --git metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/HeartbeatRequest.java metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/HeartbeatRequest.java new file mode 100644 index 0000000..04439b9 --- /dev/null +++ metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/HeartbeatRequest.java @@ -0,0 +1,485 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hadoop.hive.metastore.api; + +import org.apache.commons.lang.builder.HashCodeBuilder; +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class HeartbeatRequest implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("HeartbeatRequest"); + + private static final org.apache.thrift.protocol.TField LOCKID_FIELD_DESC = new org.apache.thrift.protocol.TField("lockid", org.apache.thrift.protocol.TType.I64, (short)1); + private static final org.apache.thrift.protocol.TField TXNID_FIELD_DESC = new org.apache.thrift.protocol.TField("txnid", org.apache.thrift.protocol.TType.I64, (short)2); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new HeartbeatRequestStandardSchemeFactory()); + schemes.put(TupleScheme.class, new HeartbeatRequestTupleSchemeFactory()); + } + + private long lockid; // optional + private long txnid; // optional + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + LOCKID((short)1, "lockid"), + TXNID((short)2, "txnid"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // LOCKID + return LOCKID; + case 2: // TXNID + return TXNID; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + private static final int __LOCKID_ISSET_ID = 0; + private static final int __TXNID_ISSET_ID = 1; + private byte __isset_bitfield = 0; + private _Fields optionals[] = {_Fields.LOCKID,_Fields.TXNID}; + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.LOCKID, new org.apache.thrift.meta_data.FieldMetaData("lockid", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64))); + tmpMap.put(_Fields.TXNID, new org.apache.thrift.meta_data.FieldMetaData("txnid", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(HeartbeatRequest.class, metaDataMap); + } + + public HeartbeatRequest() { + } + + /** + * Performs a deep copy on other. + */ + public HeartbeatRequest(HeartbeatRequest other) { + __isset_bitfield = other.__isset_bitfield; + this.lockid = other.lockid; + this.txnid = other.txnid; + } + + public HeartbeatRequest deepCopy() { + return new HeartbeatRequest(this); + } + + @Override + public void clear() { + setLockidIsSet(false); + this.lockid = 0; + setTxnidIsSet(false); + this.txnid = 0; + } + + public long getLockid() { + return this.lockid; + } + + public void setLockid(long lockid) { + this.lockid = lockid; + setLockidIsSet(true); + } + + public void unsetLockid() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __LOCKID_ISSET_ID); + } + + /** Returns true if field lockid is set (has been assigned a value) and false otherwise */ + public boolean isSetLockid() { + return EncodingUtils.testBit(__isset_bitfield, __LOCKID_ISSET_ID); + } + + public void setLockidIsSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __LOCKID_ISSET_ID, value); + } + + public long getTxnid() { + return this.txnid; + } + + public void setTxnid(long txnid) { + this.txnid = txnid; + setTxnidIsSet(true); + } + + public void unsetTxnid() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __TXNID_ISSET_ID); + } + + /** Returns true if field txnid is set (has been assigned a value) and false otherwise */ + public boolean isSetTxnid() { + return EncodingUtils.testBit(__isset_bitfield, __TXNID_ISSET_ID); + } + + public void setTxnidIsSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __TXNID_ISSET_ID, value); + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case LOCKID: + if (value == null) { + unsetLockid(); + } else { + setLockid((Long)value); + } + break; + + case TXNID: + if (value == null) { + unsetTxnid(); + } else { + setTxnid((Long)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case LOCKID: + return Long.valueOf(getLockid()); + + case TXNID: + return Long.valueOf(getTxnid()); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case LOCKID: + return isSetLockid(); + case TXNID: + return isSetTxnid(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof HeartbeatRequest) + return this.equals((HeartbeatRequest)that); + return false; + } + + public boolean equals(HeartbeatRequest that) { + if (that == null) + return false; + + boolean this_present_lockid = true && this.isSetLockid(); + boolean that_present_lockid = true && that.isSetLockid(); + if (this_present_lockid || that_present_lockid) { + if (!(this_present_lockid && that_present_lockid)) + return false; + if (this.lockid != that.lockid) + return false; + } + + boolean this_present_txnid = true && this.isSetTxnid(); + boolean that_present_txnid = true && that.isSetTxnid(); + if (this_present_txnid || that_present_txnid) { + if (!(this_present_txnid && that_present_txnid)) + return false; + if (this.txnid != that.txnid) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_lockid = true && (isSetLockid()); + builder.append(present_lockid); + if (present_lockid) + builder.append(lockid); + + boolean present_txnid = true && (isSetTxnid()); + builder.append(present_txnid); + if (present_txnid) + builder.append(txnid); + + return builder.toHashCode(); + } + + public int compareTo(HeartbeatRequest other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + HeartbeatRequest typedOther = (HeartbeatRequest)other; + + lastComparison = Boolean.valueOf(isSetLockid()).compareTo(typedOther.isSetLockid()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetLockid()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.lockid, typedOther.lockid); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetTxnid()).compareTo(typedOther.isSetTxnid()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetTxnid()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.txnid, typedOther.txnid); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("HeartbeatRequest("); + boolean first = true; + + if (isSetLockid()) { + sb.append("lockid:"); + sb.append(this.lockid); + first = false; + } + if (isSetTxnid()) { + if (!first) sb.append(", "); + sb.append("txnid:"); + sb.append(this.txnid); + first = false; + } + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. + __isset_bitfield = 0; + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class HeartbeatRequestStandardSchemeFactory implements SchemeFactory { + public HeartbeatRequestStandardScheme getScheme() { + return new HeartbeatRequestStandardScheme(); + } + } + + private static class HeartbeatRequestStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, HeartbeatRequest struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // LOCKID + if (schemeField.type == org.apache.thrift.protocol.TType.I64) { + struct.lockid = iprot.readI64(); + struct.setLockidIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // TXNID + if (schemeField.type == org.apache.thrift.protocol.TType.I64) { + struct.txnid = iprot.readI64(); + struct.setTxnidIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, HeartbeatRequest struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.isSetLockid()) { + oprot.writeFieldBegin(LOCKID_FIELD_DESC); + oprot.writeI64(struct.lockid); + oprot.writeFieldEnd(); + } + if (struct.isSetTxnid()) { + oprot.writeFieldBegin(TXNID_FIELD_DESC); + oprot.writeI64(struct.txnid); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class HeartbeatRequestTupleSchemeFactory implements SchemeFactory { + public HeartbeatRequestTupleScheme getScheme() { + return new HeartbeatRequestTupleScheme(); + } + } + + private static class HeartbeatRequestTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, HeartbeatRequest struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetLockid()) { + optionals.set(0); + } + if (struct.isSetTxnid()) { + optionals.set(1); + } + oprot.writeBitSet(optionals, 2); + if (struct.isSetLockid()) { + oprot.writeI64(struct.lockid); + } + if (struct.isSetTxnid()) { + oprot.writeI64(struct.txnid); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, HeartbeatRequest struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(2); + if (incoming.get(0)) { + struct.lockid = iprot.readI64(); + struct.setLockidIsSet(true); + } + if (incoming.get(1)) { + struct.txnid = iprot.readI64(); + struct.setTxnidIsSet(true); + } + } + } + +} + diff --git metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/HiveObjectRef.java metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/HiveObjectRef.java index b22b211..997060f 100644 --- metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/HiveObjectRef.java +++ metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/HiveObjectRef.java @@ -710,7 +710,7 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, HiveObjectRef struc struct.partValues = new ArrayList(_list8.size); for (int _i9 = 0; _i9 < _list8.size; ++_i9) { - String _elem10; // optional + String _elem10; // required _elem10 = iprot.readString(); struct.partValues.add(_elem10); } @@ -853,7 +853,7 @@ public void read(org.apache.thrift.protocol.TProtocol prot, HiveObjectRef struct struct.partValues = new ArrayList(_list13.size); for (int _i14 = 0; _i14 < _list13.size; ++_i14) { - String _elem15; // optional + String _elem15; // required _elem15 = iprot.readString(); struct.partValues.add(_elem15); } diff --git metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/LockComponent.java metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/LockComponent.java new file mode 100644 index 0000000..59c8199 --- /dev/null +++ metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/LockComponent.java @@ -0,0 +1,827 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hadoop.hive.metastore.api; + +import org.apache.commons.lang.builder.HashCodeBuilder; +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class LockComponent implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("LockComponent"); + + private static final org.apache.thrift.protocol.TField TYPE_FIELD_DESC = new org.apache.thrift.protocol.TField("type", org.apache.thrift.protocol.TType.I32, (short)1); + private static final org.apache.thrift.protocol.TField LEVEL_FIELD_DESC = new org.apache.thrift.protocol.TField("level", org.apache.thrift.protocol.TType.I32, (short)2); + private static final org.apache.thrift.protocol.TField DBNAME_FIELD_DESC = new org.apache.thrift.protocol.TField("dbname", org.apache.thrift.protocol.TType.STRING, (short)3); + private static final org.apache.thrift.protocol.TField TABLENAME_FIELD_DESC = new org.apache.thrift.protocol.TField("tablename", org.apache.thrift.protocol.TType.STRING, (short)4); + private static final org.apache.thrift.protocol.TField PARTITIONNAME_FIELD_DESC = new org.apache.thrift.protocol.TField("partitionname", org.apache.thrift.protocol.TType.STRING, (short)5); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new LockComponentStandardSchemeFactory()); + schemes.put(TupleScheme.class, new LockComponentTupleSchemeFactory()); + } + + private LockType type; // required + private LockLevel level; // required + private String dbname; // required + private String tablename; // optional + private String partitionname; // optional + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + /** + * + * @see LockType + */ + TYPE((short)1, "type"), + /** + * + * @see LockLevel + */ + LEVEL((short)2, "level"), + DBNAME((short)3, "dbname"), + TABLENAME((short)4, "tablename"), + PARTITIONNAME((short)5, "partitionname"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // TYPE + return TYPE; + case 2: // LEVEL + return LEVEL; + case 3: // DBNAME + return DBNAME; + case 4: // TABLENAME + return TABLENAME; + case 5: // PARTITIONNAME + return PARTITIONNAME; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + private _Fields optionals[] = {_Fields.TABLENAME,_Fields.PARTITIONNAME}; + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.TYPE, new org.apache.thrift.meta_data.FieldMetaData("type", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, LockType.class))); + tmpMap.put(_Fields.LEVEL, new org.apache.thrift.meta_data.FieldMetaData("level", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, LockLevel.class))); + tmpMap.put(_Fields.DBNAME, new org.apache.thrift.meta_data.FieldMetaData("dbname", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + tmpMap.put(_Fields.TABLENAME, new org.apache.thrift.meta_data.FieldMetaData("tablename", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + tmpMap.put(_Fields.PARTITIONNAME, new org.apache.thrift.meta_data.FieldMetaData("partitionname", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(LockComponent.class, metaDataMap); + } + + public LockComponent() { + } + + public LockComponent( + LockType type, + LockLevel level, + String dbname) + { + this(); + this.type = type; + this.level = level; + this.dbname = dbname; + } + + /** + * Performs a deep copy on other. + */ + public LockComponent(LockComponent other) { + if (other.isSetType()) { + this.type = other.type; + } + if (other.isSetLevel()) { + this.level = other.level; + } + if (other.isSetDbname()) { + this.dbname = other.dbname; + } + if (other.isSetTablename()) { + this.tablename = other.tablename; + } + if (other.isSetPartitionname()) { + this.partitionname = other.partitionname; + } + } + + public LockComponent deepCopy() { + return new LockComponent(this); + } + + @Override + public void clear() { + this.type = null; + this.level = null; + this.dbname = null; + this.tablename = null; + this.partitionname = null; + } + + /** + * + * @see LockType + */ + public LockType getType() { + return this.type; + } + + /** + * + * @see LockType + */ + public void setType(LockType type) { + this.type = type; + } + + public void unsetType() { + this.type = null; + } + + /** Returns true if field type is set (has been assigned a value) and false otherwise */ + public boolean isSetType() { + return this.type != null; + } + + public void setTypeIsSet(boolean value) { + if (!value) { + this.type = null; + } + } + + /** + * + * @see LockLevel + */ + public LockLevel getLevel() { + return this.level; + } + + /** + * + * @see LockLevel + */ + public void setLevel(LockLevel level) { + this.level = level; + } + + public void unsetLevel() { + this.level = null; + } + + /** Returns true if field level is set (has been assigned a value) and false otherwise */ + public boolean isSetLevel() { + return this.level != null; + } + + public void setLevelIsSet(boolean value) { + if (!value) { + this.level = null; + } + } + + public String getDbname() { + return this.dbname; + } + + public void setDbname(String dbname) { + this.dbname = dbname; + } + + public void unsetDbname() { + this.dbname = null; + } + + /** Returns true if field dbname is set (has been assigned a value) and false otherwise */ + public boolean isSetDbname() { + return this.dbname != null; + } + + public void setDbnameIsSet(boolean value) { + if (!value) { + this.dbname = null; + } + } + + public String getTablename() { + return this.tablename; + } + + public void setTablename(String tablename) { + this.tablename = tablename; + } + + public void unsetTablename() { + this.tablename = null; + } + + /** Returns true if field tablename is set (has been assigned a value) and false otherwise */ + public boolean isSetTablename() { + return this.tablename != null; + } + + public void setTablenameIsSet(boolean value) { + if (!value) { + this.tablename = null; + } + } + + public String getPartitionname() { + return this.partitionname; + } + + public void setPartitionname(String partitionname) { + this.partitionname = partitionname; + } + + public void unsetPartitionname() { + this.partitionname = null; + } + + /** Returns true if field partitionname is set (has been assigned a value) and false otherwise */ + public boolean isSetPartitionname() { + return this.partitionname != null; + } + + public void setPartitionnameIsSet(boolean value) { + if (!value) { + this.partitionname = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case TYPE: + if (value == null) { + unsetType(); + } else { + setType((LockType)value); + } + break; + + case LEVEL: + if (value == null) { + unsetLevel(); + } else { + setLevel((LockLevel)value); + } + break; + + case DBNAME: + if (value == null) { + unsetDbname(); + } else { + setDbname((String)value); + } + break; + + case TABLENAME: + if (value == null) { + unsetTablename(); + } else { + setTablename((String)value); + } + break; + + case PARTITIONNAME: + if (value == null) { + unsetPartitionname(); + } else { + setPartitionname((String)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case TYPE: + return getType(); + + case LEVEL: + return getLevel(); + + case DBNAME: + return getDbname(); + + case TABLENAME: + return getTablename(); + + case PARTITIONNAME: + return getPartitionname(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case TYPE: + return isSetType(); + case LEVEL: + return isSetLevel(); + case DBNAME: + return isSetDbname(); + case TABLENAME: + return isSetTablename(); + case PARTITIONNAME: + return isSetPartitionname(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof LockComponent) + return this.equals((LockComponent)that); + return false; + } + + public boolean equals(LockComponent that) { + if (that == null) + return false; + + boolean this_present_type = true && this.isSetType(); + boolean that_present_type = true && that.isSetType(); + if (this_present_type || that_present_type) { + if (!(this_present_type && that_present_type)) + return false; + if (!this.type.equals(that.type)) + return false; + } + + boolean this_present_level = true && this.isSetLevel(); + boolean that_present_level = true && that.isSetLevel(); + if (this_present_level || that_present_level) { + if (!(this_present_level && that_present_level)) + return false; + if (!this.level.equals(that.level)) + return false; + } + + boolean this_present_dbname = true && this.isSetDbname(); + boolean that_present_dbname = true && that.isSetDbname(); + if (this_present_dbname || that_present_dbname) { + if (!(this_present_dbname && that_present_dbname)) + return false; + if (!this.dbname.equals(that.dbname)) + return false; + } + + boolean this_present_tablename = true && this.isSetTablename(); + boolean that_present_tablename = true && that.isSetTablename(); + if (this_present_tablename || that_present_tablename) { + if (!(this_present_tablename && that_present_tablename)) + return false; + if (!this.tablename.equals(that.tablename)) + return false; + } + + boolean this_present_partitionname = true && this.isSetPartitionname(); + boolean that_present_partitionname = true && that.isSetPartitionname(); + if (this_present_partitionname || that_present_partitionname) { + if (!(this_present_partitionname && that_present_partitionname)) + return false; + if (!this.partitionname.equals(that.partitionname)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_type = true && (isSetType()); + builder.append(present_type); + if (present_type) + builder.append(type.getValue()); + + boolean present_level = true && (isSetLevel()); + builder.append(present_level); + if (present_level) + builder.append(level.getValue()); + + boolean present_dbname = true && (isSetDbname()); + builder.append(present_dbname); + if (present_dbname) + builder.append(dbname); + + boolean present_tablename = true && (isSetTablename()); + builder.append(present_tablename); + if (present_tablename) + builder.append(tablename); + + boolean present_partitionname = true && (isSetPartitionname()); + builder.append(present_partitionname); + if (present_partitionname) + builder.append(partitionname); + + return builder.toHashCode(); + } + + public int compareTo(LockComponent other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + LockComponent typedOther = (LockComponent)other; + + lastComparison = Boolean.valueOf(isSetType()).compareTo(typedOther.isSetType()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetType()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.type, typedOther.type); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetLevel()).compareTo(typedOther.isSetLevel()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetLevel()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.level, typedOther.level); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetDbname()).compareTo(typedOther.isSetDbname()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetDbname()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.dbname, typedOther.dbname); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetTablename()).compareTo(typedOther.isSetTablename()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetTablename()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tablename, typedOther.tablename); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetPartitionname()).compareTo(typedOther.isSetPartitionname()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetPartitionname()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.partitionname, typedOther.partitionname); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("LockComponent("); + boolean first = true; + + sb.append("type:"); + if (this.type == null) { + sb.append("null"); + } else { + sb.append(this.type); + } + first = false; + if (!first) sb.append(", "); + sb.append("level:"); + if (this.level == null) { + sb.append("null"); + } else { + sb.append(this.level); + } + first = false; + if (!first) sb.append(", "); + sb.append("dbname:"); + if (this.dbname == null) { + sb.append("null"); + } else { + sb.append(this.dbname); + } + first = false; + if (isSetTablename()) { + if (!first) sb.append(", "); + sb.append("tablename:"); + if (this.tablename == null) { + sb.append("null"); + } else { + sb.append(this.tablename); + } + first = false; + } + if (isSetPartitionname()) { + if (!first) sb.append(", "); + sb.append("partitionname:"); + if (this.partitionname == null) { + sb.append("null"); + } else { + sb.append(this.partitionname); + } + first = false; + } + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetType()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'type' is unset! Struct:" + toString()); + } + + if (!isSetLevel()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'level' is unset! Struct:" + toString()); + } + + if (!isSetDbname()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'dbname' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class LockComponentStandardSchemeFactory implements SchemeFactory { + public LockComponentStandardScheme getScheme() { + return new LockComponentStandardScheme(); + } + } + + private static class LockComponentStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, LockComponent struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // TYPE + if (schemeField.type == org.apache.thrift.protocol.TType.I32) { + struct.type = LockType.findByValue(iprot.readI32()); + struct.setTypeIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // LEVEL + if (schemeField.type == org.apache.thrift.protocol.TType.I32) { + struct.level = LockLevel.findByValue(iprot.readI32()); + struct.setLevelIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 3: // DBNAME + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.dbname = iprot.readString(); + struct.setDbnameIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 4: // TABLENAME + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.tablename = iprot.readString(); + struct.setTablenameIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 5: // PARTITIONNAME + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.partitionname = iprot.readString(); + struct.setPartitionnameIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, LockComponent struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.type != null) { + oprot.writeFieldBegin(TYPE_FIELD_DESC); + oprot.writeI32(struct.type.getValue()); + oprot.writeFieldEnd(); + } + if (struct.level != null) { + oprot.writeFieldBegin(LEVEL_FIELD_DESC); + oprot.writeI32(struct.level.getValue()); + oprot.writeFieldEnd(); + } + if (struct.dbname != null) { + oprot.writeFieldBegin(DBNAME_FIELD_DESC); + oprot.writeString(struct.dbname); + oprot.writeFieldEnd(); + } + if (struct.tablename != null) { + if (struct.isSetTablename()) { + oprot.writeFieldBegin(TABLENAME_FIELD_DESC); + oprot.writeString(struct.tablename); + oprot.writeFieldEnd(); + } + } + if (struct.partitionname != null) { + if (struct.isSetPartitionname()) { + oprot.writeFieldBegin(PARTITIONNAME_FIELD_DESC); + oprot.writeString(struct.partitionname); + oprot.writeFieldEnd(); + } + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class LockComponentTupleSchemeFactory implements SchemeFactory { + public LockComponentTupleScheme getScheme() { + return new LockComponentTupleScheme(); + } + } + + private static class LockComponentTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, LockComponent struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + oprot.writeI32(struct.type.getValue()); + oprot.writeI32(struct.level.getValue()); + oprot.writeString(struct.dbname); + BitSet optionals = new BitSet(); + if (struct.isSetTablename()) { + optionals.set(0); + } + if (struct.isSetPartitionname()) { + optionals.set(1); + } + oprot.writeBitSet(optionals, 2); + if (struct.isSetTablename()) { + oprot.writeString(struct.tablename); + } + if (struct.isSetPartitionname()) { + oprot.writeString(struct.partitionname); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, LockComponent struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.type = LockType.findByValue(iprot.readI32()); + struct.setTypeIsSet(true); + struct.level = LockLevel.findByValue(iprot.readI32()); + struct.setLevelIsSet(true); + struct.dbname = iprot.readString(); + struct.setDbnameIsSet(true); + BitSet incoming = iprot.readBitSet(2); + if (incoming.get(0)) { + struct.tablename = iprot.readString(); + struct.setTablenameIsSet(true); + } + if (incoming.get(1)) { + struct.partitionname = iprot.readString(); + struct.setPartitionnameIsSet(true); + } + } + } + +} + diff --git metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/LockLevel.java metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/LockLevel.java new file mode 100644 index 0000000..ca5d30a --- /dev/null +++ metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/LockLevel.java @@ -0,0 +1,48 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hadoop.hive.metastore.api; + + +import java.util.Map; +import java.util.HashMap; +import org.apache.thrift.TEnum; + +public enum LockLevel implements org.apache.thrift.TEnum { + DB(1), + TABLE(2), + PARTITION(3); + + private final int value; + + private LockLevel(int value) { + this.value = value; + } + + /** + * Get the integer value of this enum value, as defined in the Thrift IDL. + */ + public int getValue() { + return value; + } + + /** + * Find a the enum type by its integer value, as defined in the Thrift IDL. + * @return null if the value is not found. + */ + public static LockLevel findByValue(int value) { + switch (value) { + case 1: + return DB; + case 2: + return TABLE; + case 3: + return PARTITION; + default: + return null; + } + } +} diff --git metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/LockRequest.java metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/LockRequest.java new file mode 100644 index 0000000..9b858d6 --- /dev/null +++ metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/LockRequest.java @@ -0,0 +1,748 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hadoop.hive.metastore.api; + +import org.apache.commons.lang.builder.HashCodeBuilder; +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class LockRequest implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("LockRequest"); + + private static final org.apache.thrift.protocol.TField COMPONENT_FIELD_DESC = new org.apache.thrift.protocol.TField("component", org.apache.thrift.protocol.TType.LIST, (short)1); + private static final org.apache.thrift.protocol.TField TXNID_FIELD_DESC = new org.apache.thrift.protocol.TField("txnid", org.apache.thrift.protocol.TType.I64, (short)2); + private static final org.apache.thrift.protocol.TField USER_FIELD_DESC = new org.apache.thrift.protocol.TField("user", org.apache.thrift.protocol.TType.STRING, (short)3); + private static final org.apache.thrift.protocol.TField HOSTNAME_FIELD_DESC = new org.apache.thrift.protocol.TField("hostname", org.apache.thrift.protocol.TType.STRING, (short)4); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new LockRequestStandardSchemeFactory()); + schemes.put(TupleScheme.class, new LockRequestTupleSchemeFactory()); + } + + private List component; // required + private long txnid; // optional + private String user; // required + private String hostname; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + COMPONENT((short)1, "component"), + TXNID((short)2, "txnid"), + USER((short)3, "user"), + HOSTNAME((short)4, "hostname"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // COMPONENT + return COMPONENT; + case 2: // TXNID + return TXNID; + case 3: // USER + return USER; + case 4: // HOSTNAME + return HOSTNAME; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + private static final int __TXNID_ISSET_ID = 0; + private byte __isset_bitfield = 0; + private _Fields optionals[] = {_Fields.TXNID}; + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.COMPONENT, new org.apache.thrift.meta_data.FieldMetaData("component", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, LockComponent.class)))); + tmpMap.put(_Fields.TXNID, new org.apache.thrift.meta_data.FieldMetaData("txnid", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64))); + tmpMap.put(_Fields.USER, new org.apache.thrift.meta_data.FieldMetaData("user", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + tmpMap.put(_Fields.HOSTNAME, new org.apache.thrift.meta_data.FieldMetaData("hostname", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(LockRequest.class, metaDataMap); + } + + public LockRequest() { + } + + public LockRequest( + List component, + String user, + String hostname) + { + this(); + this.component = component; + this.user = user; + this.hostname = hostname; + } + + /** + * Performs a deep copy on other. + */ + public LockRequest(LockRequest other) { + __isset_bitfield = other.__isset_bitfield; + if (other.isSetComponent()) { + List __this__component = new ArrayList(); + for (LockComponent other_element : other.component) { + __this__component.add(new LockComponent(other_element)); + } + this.component = __this__component; + } + this.txnid = other.txnid; + if (other.isSetUser()) { + this.user = other.user; + } + if (other.isSetHostname()) { + this.hostname = other.hostname; + } + } + + public LockRequest deepCopy() { + return new LockRequest(this); + } + + @Override + public void clear() { + this.component = null; + setTxnidIsSet(false); + this.txnid = 0; + this.user = null; + this.hostname = null; + } + + public int getComponentSize() { + return (this.component == null) ? 0 : this.component.size(); + } + + public java.util.Iterator getComponentIterator() { + return (this.component == null) ? null : this.component.iterator(); + } + + public void addToComponent(LockComponent elem) { + if (this.component == null) { + this.component = new ArrayList(); + } + this.component.add(elem); + } + + public List getComponent() { + return this.component; + } + + public void setComponent(List component) { + this.component = component; + } + + public void unsetComponent() { + this.component = null; + } + + /** Returns true if field component is set (has been assigned a value) and false otherwise */ + public boolean isSetComponent() { + return this.component != null; + } + + public void setComponentIsSet(boolean value) { + if (!value) { + this.component = null; + } + } + + public long getTxnid() { + return this.txnid; + } + + public void setTxnid(long txnid) { + this.txnid = txnid; + setTxnidIsSet(true); + } + + public void unsetTxnid() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __TXNID_ISSET_ID); + } + + /** Returns true if field txnid is set (has been assigned a value) and false otherwise */ + public boolean isSetTxnid() { + return EncodingUtils.testBit(__isset_bitfield, __TXNID_ISSET_ID); + } + + public void setTxnidIsSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __TXNID_ISSET_ID, value); + } + + public String getUser() { + return this.user; + } + + public void setUser(String user) { + this.user = user; + } + + public void unsetUser() { + this.user = null; + } + + /** Returns true if field user is set (has been assigned a value) and false otherwise */ + public boolean isSetUser() { + return this.user != null; + } + + public void setUserIsSet(boolean value) { + if (!value) { + this.user = null; + } + } + + public String getHostname() { + return this.hostname; + } + + public void setHostname(String hostname) { + this.hostname = hostname; + } + + public void unsetHostname() { + this.hostname = null; + } + + /** Returns true if field hostname is set (has been assigned a value) and false otherwise */ + public boolean isSetHostname() { + return this.hostname != null; + } + + public void setHostnameIsSet(boolean value) { + if (!value) { + this.hostname = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case COMPONENT: + if (value == null) { + unsetComponent(); + } else { + setComponent((List)value); + } + break; + + case TXNID: + if (value == null) { + unsetTxnid(); + } else { + setTxnid((Long)value); + } + break; + + case USER: + if (value == null) { + unsetUser(); + } else { + setUser((String)value); + } + break; + + case HOSTNAME: + if (value == null) { + unsetHostname(); + } else { + setHostname((String)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case COMPONENT: + return getComponent(); + + case TXNID: + return Long.valueOf(getTxnid()); + + case USER: + return getUser(); + + case HOSTNAME: + return getHostname(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case COMPONENT: + return isSetComponent(); + case TXNID: + return isSetTxnid(); + case USER: + return isSetUser(); + case HOSTNAME: + return isSetHostname(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof LockRequest) + return this.equals((LockRequest)that); + return false; + } + + public boolean equals(LockRequest that) { + if (that == null) + return false; + + boolean this_present_component = true && this.isSetComponent(); + boolean that_present_component = true && that.isSetComponent(); + if (this_present_component || that_present_component) { + if (!(this_present_component && that_present_component)) + return false; + if (!this.component.equals(that.component)) + return false; + } + + boolean this_present_txnid = true && this.isSetTxnid(); + boolean that_present_txnid = true && that.isSetTxnid(); + if (this_present_txnid || that_present_txnid) { + if (!(this_present_txnid && that_present_txnid)) + return false; + if (this.txnid != that.txnid) + return false; + } + + boolean this_present_user = true && this.isSetUser(); + boolean that_present_user = true && that.isSetUser(); + if (this_present_user || that_present_user) { + if (!(this_present_user && that_present_user)) + return false; + if (!this.user.equals(that.user)) + return false; + } + + boolean this_present_hostname = true && this.isSetHostname(); + boolean that_present_hostname = true && that.isSetHostname(); + if (this_present_hostname || that_present_hostname) { + if (!(this_present_hostname && that_present_hostname)) + return false; + if (!this.hostname.equals(that.hostname)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_component = true && (isSetComponent()); + builder.append(present_component); + if (present_component) + builder.append(component); + + boolean present_txnid = true && (isSetTxnid()); + builder.append(present_txnid); + if (present_txnid) + builder.append(txnid); + + boolean present_user = true && (isSetUser()); + builder.append(present_user); + if (present_user) + builder.append(user); + + boolean present_hostname = true && (isSetHostname()); + builder.append(present_hostname); + if (present_hostname) + builder.append(hostname); + + return builder.toHashCode(); + } + + public int compareTo(LockRequest other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + LockRequest typedOther = (LockRequest)other; + + lastComparison = Boolean.valueOf(isSetComponent()).compareTo(typedOther.isSetComponent()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetComponent()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.component, typedOther.component); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetTxnid()).compareTo(typedOther.isSetTxnid()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetTxnid()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.txnid, typedOther.txnid); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetUser()).compareTo(typedOther.isSetUser()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetUser()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.user, typedOther.user); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetHostname()).compareTo(typedOther.isSetHostname()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetHostname()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.hostname, typedOther.hostname); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("LockRequest("); + boolean first = true; + + sb.append("component:"); + if (this.component == null) { + sb.append("null"); + } else { + sb.append(this.component); + } + first = false; + if (isSetTxnid()) { + if (!first) sb.append(", "); + sb.append("txnid:"); + sb.append(this.txnid); + first = false; + } + if (!first) sb.append(", "); + sb.append("user:"); + if (this.user == null) { + sb.append("null"); + } else { + sb.append(this.user); + } + first = false; + if (!first) sb.append(", "); + sb.append("hostname:"); + if (this.hostname == null) { + sb.append("null"); + } else { + sb.append(this.hostname); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetComponent()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'component' is unset! Struct:" + toString()); + } + + if (!isSetUser()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'user' is unset! Struct:" + toString()); + } + + if (!isSetHostname()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'hostname' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. + __isset_bitfield = 0; + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class LockRequestStandardSchemeFactory implements SchemeFactory { + public LockRequestStandardScheme getScheme() { + return new LockRequestStandardScheme(); + } + } + + private static class LockRequestStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, LockRequest struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // COMPONENT + if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { + { + org.apache.thrift.protocol.TList _list386 = iprot.readListBegin(); + struct.component = new ArrayList(_list386.size); + for (int _i387 = 0; _i387 < _list386.size; ++_i387) + { + LockComponent _elem388; // required + _elem388 = new LockComponent(); + _elem388.read(iprot); + struct.component.add(_elem388); + } + iprot.readListEnd(); + } + struct.setComponentIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // TXNID + if (schemeField.type == org.apache.thrift.protocol.TType.I64) { + struct.txnid = iprot.readI64(); + struct.setTxnidIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 3: // USER + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.user = iprot.readString(); + struct.setUserIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 4: // HOSTNAME + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.hostname = iprot.readString(); + struct.setHostnameIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, LockRequest struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.component != null) { + oprot.writeFieldBegin(COMPONENT_FIELD_DESC); + { + oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.component.size())); + for (LockComponent _iter389 : struct.component) + { + _iter389.write(oprot); + } + oprot.writeListEnd(); + } + oprot.writeFieldEnd(); + } + if (struct.isSetTxnid()) { + oprot.writeFieldBegin(TXNID_FIELD_DESC); + oprot.writeI64(struct.txnid); + oprot.writeFieldEnd(); + } + if (struct.user != null) { + oprot.writeFieldBegin(USER_FIELD_DESC); + oprot.writeString(struct.user); + oprot.writeFieldEnd(); + } + if (struct.hostname != null) { + oprot.writeFieldBegin(HOSTNAME_FIELD_DESC); + oprot.writeString(struct.hostname); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class LockRequestTupleSchemeFactory implements SchemeFactory { + public LockRequestTupleScheme getScheme() { + return new LockRequestTupleScheme(); + } + } + + private static class LockRequestTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, LockRequest struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + { + oprot.writeI32(struct.component.size()); + for (LockComponent _iter390 : struct.component) + { + _iter390.write(oprot); + } + } + oprot.writeString(struct.user); + oprot.writeString(struct.hostname); + BitSet optionals = new BitSet(); + if (struct.isSetTxnid()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetTxnid()) { + oprot.writeI64(struct.txnid); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, LockRequest struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + { + org.apache.thrift.protocol.TList _list391 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); + struct.component = new ArrayList(_list391.size); + for (int _i392 = 0; _i392 < _list391.size; ++_i392) + { + LockComponent _elem393; // required + _elem393 = new LockComponent(); + _elem393.read(iprot); + struct.component.add(_elem393); + } + } + struct.setComponentIsSet(true); + struct.user = iprot.readString(); + struct.setUserIsSet(true); + struct.hostname = iprot.readString(); + struct.setHostnameIsSet(true); + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.txnid = iprot.readI64(); + struct.setTxnidIsSet(true); + } + } + } + +} + diff --git metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/LockResponse.java metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/LockResponse.java new file mode 100644 index 0000000..c9ab465 --- /dev/null +++ metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/LockResponse.java @@ -0,0 +1,496 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hadoop.hive.metastore.api; + +import org.apache.commons.lang.builder.HashCodeBuilder; +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class LockResponse implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("LockResponse"); + + private static final org.apache.thrift.protocol.TField LOCKID_FIELD_DESC = new org.apache.thrift.protocol.TField("lockid", org.apache.thrift.protocol.TType.I64, (short)1); + private static final org.apache.thrift.protocol.TField STATE_FIELD_DESC = new org.apache.thrift.protocol.TField("state", org.apache.thrift.protocol.TType.I32, (short)2); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new LockResponseStandardSchemeFactory()); + schemes.put(TupleScheme.class, new LockResponseTupleSchemeFactory()); + } + + private long lockid; // required + private LockState state; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + LOCKID((short)1, "lockid"), + /** + * + * @see LockState + */ + STATE((short)2, "state"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // LOCKID + return LOCKID; + case 2: // STATE + return STATE; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + private static final int __LOCKID_ISSET_ID = 0; + private byte __isset_bitfield = 0; + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.LOCKID, new org.apache.thrift.meta_data.FieldMetaData("lockid", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64))); + tmpMap.put(_Fields.STATE, new org.apache.thrift.meta_data.FieldMetaData("state", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, LockState.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(LockResponse.class, metaDataMap); + } + + public LockResponse() { + } + + public LockResponse( + long lockid, + LockState state) + { + this(); + this.lockid = lockid; + setLockidIsSet(true); + this.state = state; + } + + /** + * Performs a deep copy on other. + */ + public LockResponse(LockResponse other) { + __isset_bitfield = other.__isset_bitfield; + this.lockid = other.lockid; + if (other.isSetState()) { + this.state = other.state; + } + } + + public LockResponse deepCopy() { + return new LockResponse(this); + } + + @Override + public void clear() { + setLockidIsSet(false); + this.lockid = 0; + this.state = null; + } + + public long getLockid() { + return this.lockid; + } + + public void setLockid(long lockid) { + this.lockid = lockid; + setLockidIsSet(true); + } + + public void unsetLockid() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __LOCKID_ISSET_ID); + } + + /** Returns true if field lockid is set (has been assigned a value) and false otherwise */ + public boolean isSetLockid() { + return EncodingUtils.testBit(__isset_bitfield, __LOCKID_ISSET_ID); + } + + public void setLockidIsSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __LOCKID_ISSET_ID, value); + } + + /** + * + * @see LockState + */ + public LockState getState() { + return this.state; + } + + /** + * + * @see LockState + */ + public void setState(LockState state) { + this.state = state; + } + + public void unsetState() { + this.state = null; + } + + /** Returns true if field state is set (has been assigned a value) and false otherwise */ + public boolean isSetState() { + return this.state != null; + } + + public void setStateIsSet(boolean value) { + if (!value) { + this.state = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case LOCKID: + if (value == null) { + unsetLockid(); + } else { + setLockid((Long)value); + } + break; + + case STATE: + if (value == null) { + unsetState(); + } else { + setState((LockState)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case LOCKID: + return Long.valueOf(getLockid()); + + case STATE: + return getState(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case LOCKID: + return isSetLockid(); + case STATE: + return isSetState(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof LockResponse) + return this.equals((LockResponse)that); + return false; + } + + public boolean equals(LockResponse that) { + if (that == null) + return false; + + boolean this_present_lockid = true; + boolean that_present_lockid = true; + if (this_present_lockid || that_present_lockid) { + if (!(this_present_lockid && that_present_lockid)) + return false; + if (this.lockid != that.lockid) + return false; + } + + boolean this_present_state = true && this.isSetState(); + boolean that_present_state = true && that.isSetState(); + if (this_present_state || that_present_state) { + if (!(this_present_state && that_present_state)) + return false; + if (!this.state.equals(that.state)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_lockid = true; + builder.append(present_lockid); + if (present_lockid) + builder.append(lockid); + + boolean present_state = true && (isSetState()); + builder.append(present_state); + if (present_state) + builder.append(state.getValue()); + + return builder.toHashCode(); + } + + public int compareTo(LockResponse other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + LockResponse typedOther = (LockResponse)other; + + lastComparison = Boolean.valueOf(isSetLockid()).compareTo(typedOther.isSetLockid()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetLockid()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.lockid, typedOther.lockid); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetState()).compareTo(typedOther.isSetState()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetState()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.state, typedOther.state); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("LockResponse("); + boolean first = true; + + sb.append("lockid:"); + sb.append(this.lockid); + first = false; + if (!first) sb.append(", "); + sb.append("state:"); + if (this.state == null) { + sb.append("null"); + } else { + sb.append(this.state); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetLockid()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'lockid' is unset! Struct:" + toString()); + } + + if (!isSetState()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'state' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. + __isset_bitfield = 0; + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class LockResponseStandardSchemeFactory implements SchemeFactory { + public LockResponseStandardScheme getScheme() { + return new LockResponseStandardScheme(); + } + } + + private static class LockResponseStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, LockResponse struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // LOCKID + if (schemeField.type == org.apache.thrift.protocol.TType.I64) { + struct.lockid = iprot.readI64(); + struct.setLockidIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // STATE + if (schemeField.type == org.apache.thrift.protocol.TType.I32) { + struct.state = LockState.findByValue(iprot.readI32()); + struct.setStateIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, LockResponse struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + oprot.writeFieldBegin(LOCKID_FIELD_DESC); + oprot.writeI64(struct.lockid); + oprot.writeFieldEnd(); + if (struct.state != null) { + oprot.writeFieldBegin(STATE_FIELD_DESC); + oprot.writeI32(struct.state.getValue()); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class LockResponseTupleSchemeFactory implements SchemeFactory { + public LockResponseTupleScheme getScheme() { + return new LockResponseTupleScheme(); + } + } + + private static class LockResponseTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, LockResponse struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + oprot.writeI64(struct.lockid); + oprot.writeI32(struct.state.getValue()); + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, LockResponse struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.lockid = iprot.readI64(); + struct.setLockidIsSet(true); + struct.state = LockState.findByValue(iprot.readI32()); + struct.setStateIsSet(true); + } + } + +} + diff --git metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/LockState.java metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/LockState.java new file mode 100644 index 0000000..57bc3c5 --- /dev/null +++ metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/LockState.java @@ -0,0 +1,51 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hadoop.hive.metastore.api; + + +import java.util.Map; +import java.util.HashMap; +import org.apache.thrift.TEnum; + +public enum LockState implements org.apache.thrift.TEnum { + ACQUIRED(1), + WAITING(2), + ABORT(3), + NOT_ACQUIRED(4); + + private final int value; + + private LockState(int value) { + this.value = value; + } + + /** + * Get the integer value of this enum value, as defined in the Thrift IDL. + */ + public int getValue() { + return value; + } + + /** + * Find a the enum type by its integer value, as defined in the Thrift IDL. + * @return null if the value is not found. + */ + public static LockState findByValue(int value) { + switch (value) { + case 1: + return ACQUIRED; + case 2: + return WAITING; + case 3: + return ABORT; + case 4: + return NOT_ACQUIRED; + default: + return null; + } + } +} diff --git metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/LockType.java metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/LockType.java new file mode 100644 index 0000000..ee57883 --- /dev/null +++ metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/LockType.java @@ -0,0 +1,48 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hadoop.hive.metastore.api; + + +import java.util.Map; +import java.util.HashMap; +import org.apache.thrift.TEnum; + +public enum LockType implements org.apache.thrift.TEnum { + SHARED_READ(1), + SHARED_WRITE(2), + EXCLUSIVE(3); + + private final int value; + + private LockType(int value) { + this.value = value; + } + + /** + * Get the integer value of this enum value, as defined in the Thrift IDL. + */ + public int getValue() { + return value; + } + + /** + * Find a the enum type by its integer value, as defined in the Thrift IDL. + * @return null if the value is not found. + */ + public static LockType findByValue(int value) { + switch (value) { + case 1: + return SHARED_READ; + case 2: + return SHARED_WRITE; + case 3: + return EXCLUSIVE; + default: + return null; + } + } +} diff --git metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NoSuchLockException.java metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NoSuchLockException.java new file mode 100644 index 0000000..9c8bd0b --- /dev/null +++ metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NoSuchLockException.java @@ -0,0 +1,391 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hadoop.hive.metastore.api; + +import org.apache.commons.lang.builder.HashCodeBuilder; +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class NoSuchLockException extends TException implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("NoSuchLockException"); + + private static final org.apache.thrift.protocol.TField MESSAGE_FIELD_DESC = new org.apache.thrift.protocol.TField("message", org.apache.thrift.protocol.TType.STRING, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new NoSuchLockExceptionStandardSchemeFactory()); + schemes.put(TupleScheme.class, new NoSuchLockExceptionTupleSchemeFactory()); + } + + private String message; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + MESSAGE((short)1, "message"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // MESSAGE + return MESSAGE; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.MESSAGE, new org.apache.thrift.meta_data.FieldMetaData("message", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(NoSuchLockException.class, metaDataMap); + } + + public NoSuchLockException() { + } + + public NoSuchLockException( + String message) + { + this(); + this.message = message; + } + + /** + * Performs a deep copy on other. + */ + public NoSuchLockException(NoSuchLockException other) { + if (other.isSetMessage()) { + this.message = other.message; + } + } + + public NoSuchLockException deepCopy() { + return new NoSuchLockException(this); + } + + @Override + public void clear() { + this.message = null; + } + + public String getMessage() { + return this.message; + } + + public void setMessage(String message) { + this.message = message; + } + + public void unsetMessage() { + this.message = null; + } + + /** Returns true if field message is set (has been assigned a value) and false otherwise */ + public boolean isSetMessage() { + return this.message != null; + } + + public void setMessageIsSet(boolean value) { + if (!value) { + this.message = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case MESSAGE: + if (value == null) { + unsetMessage(); + } else { + setMessage((String)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case MESSAGE: + return getMessage(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case MESSAGE: + return isSetMessage(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof NoSuchLockException) + return this.equals((NoSuchLockException)that); + return false; + } + + public boolean equals(NoSuchLockException that) { + if (that == null) + return false; + + boolean this_present_message = true && this.isSetMessage(); + boolean that_present_message = true && that.isSetMessage(); + if (this_present_message || that_present_message) { + if (!(this_present_message && that_present_message)) + return false; + if (!this.message.equals(that.message)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_message = true && (isSetMessage()); + builder.append(present_message); + if (present_message) + builder.append(message); + + return builder.toHashCode(); + } + + public int compareTo(NoSuchLockException other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + NoSuchLockException typedOther = (NoSuchLockException)other; + + lastComparison = Boolean.valueOf(isSetMessage()).compareTo(typedOther.isSetMessage()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetMessage()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.message, typedOther.message); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("NoSuchLockException("); + boolean first = true; + + sb.append("message:"); + if (this.message == null) { + sb.append("null"); + } else { + sb.append(this.message); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class NoSuchLockExceptionStandardSchemeFactory implements SchemeFactory { + public NoSuchLockExceptionStandardScheme getScheme() { + return new NoSuchLockExceptionStandardScheme(); + } + } + + private static class NoSuchLockExceptionStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, NoSuchLockException struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // MESSAGE + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.message = iprot.readString(); + struct.setMessageIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, NoSuchLockException struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.message != null) { + oprot.writeFieldBegin(MESSAGE_FIELD_DESC); + oprot.writeString(struct.message); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class NoSuchLockExceptionTupleSchemeFactory implements SchemeFactory { + public NoSuchLockExceptionTupleScheme getScheme() { + return new NoSuchLockExceptionTupleScheme(); + } + } + + private static class NoSuchLockExceptionTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, NoSuchLockException struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetMessage()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetMessage()) { + oprot.writeString(struct.message); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, NoSuchLockException struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.message = iprot.readString(); + struct.setMessageIsSet(true); + } + } + } + +} + diff --git metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NoSuchTxnException.java metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NoSuchTxnException.java new file mode 100644 index 0000000..92dbb7f --- /dev/null +++ metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NoSuchTxnException.java @@ -0,0 +1,391 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hadoop.hive.metastore.api; + +import org.apache.commons.lang.builder.HashCodeBuilder; +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class NoSuchTxnException extends TException implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("NoSuchTxnException"); + + private static final org.apache.thrift.protocol.TField MESSAGE_FIELD_DESC = new org.apache.thrift.protocol.TField("message", org.apache.thrift.protocol.TType.STRING, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new NoSuchTxnExceptionStandardSchemeFactory()); + schemes.put(TupleScheme.class, new NoSuchTxnExceptionTupleSchemeFactory()); + } + + private String message; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + MESSAGE((short)1, "message"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // MESSAGE + return MESSAGE; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.MESSAGE, new org.apache.thrift.meta_data.FieldMetaData("message", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(NoSuchTxnException.class, metaDataMap); + } + + public NoSuchTxnException() { + } + + public NoSuchTxnException( + String message) + { + this(); + this.message = message; + } + + /** + * Performs a deep copy on other. + */ + public NoSuchTxnException(NoSuchTxnException other) { + if (other.isSetMessage()) { + this.message = other.message; + } + } + + public NoSuchTxnException deepCopy() { + return new NoSuchTxnException(this); + } + + @Override + public void clear() { + this.message = null; + } + + public String getMessage() { + return this.message; + } + + public void setMessage(String message) { + this.message = message; + } + + public void unsetMessage() { + this.message = null; + } + + /** Returns true if field message is set (has been assigned a value) and false otherwise */ + public boolean isSetMessage() { + return this.message != null; + } + + public void setMessageIsSet(boolean value) { + if (!value) { + this.message = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case MESSAGE: + if (value == null) { + unsetMessage(); + } else { + setMessage((String)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case MESSAGE: + return getMessage(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case MESSAGE: + return isSetMessage(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof NoSuchTxnException) + return this.equals((NoSuchTxnException)that); + return false; + } + + public boolean equals(NoSuchTxnException that) { + if (that == null) + return false; + + boolean this_present_message = true && this.isSetMessage(); + boolean that_present_message = true && that.isSetMessage(); + if (this_present_message || that_present_message) { + if (!(this_present_message && that_present_message)) + return false; + if (!this.message.equals(that.message)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_message = true && (isSetMessage()); + builder.append(present_message); + if (present_message) + builder.append(message); + + return builder.toHashCode(); + } + + public int compareTo(NoSuchTxnException other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + NoSuchTxnException typedOther = (NoSuchTxnException)other; + + lastComparison = Boolean.valueOf(isSetMessage()).compareTo(typedOther.isSetMessage()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetMessage()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.message, typedOther.message); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("NoSuchTxnException("); + boolean first = true; + + sb.append("message:"); + if (this.message == null) { + sb.append("null"); + } else { + sb.append(this.message); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class NoSuchTxnExceptionStandardSchemeFactory implements SchemeFactory { + public NoSuchTxnExceptionStandardScheme getScheme() { + return new NoSuchTxnExceptionStandardScheme(); + } + } + + private static class NoSuchTxnExceptionStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, NoSuchTxnException struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // MESSAGE + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.message = iprot.readString(); + struct.setMessageIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, NoSuchTxnException struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.message != null) { + oprot.writeFieldBegin(MESSAGE_FIELD_DESC); + oprot.writeString(struct.message); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class NoSuchTxnExceptionTupleSchemeFactory implements SchemeFactory { + public NoSuchTxnExceptionTupleScheme getScheme() { + return new NoSuchTxnExceptionTupleScheme(); + } + } + + private static class NoSuchTxnExceptionTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, NoSuchTxnException struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetMessage()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetMessage()) { + oprot.writeString(struct.message); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, NoSuchTxnException struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.message = iprot.readString(); + struct.setMessageIsSet(true); + } + } + } + +} + diff --git metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/OpenTxnRequest.java metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/OpenTxnRequest.java new file mode 100644 index 0000000..0d05378 --- /dev/null +++ metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/OpenTxnRequest.java @@ -0,0 +1,585 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hadoop.hive.metastore.api; + +import org.apache.commons.lang.builder.HashCodeBuilder; +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class OpenTxnRequest implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("OpenTxnRequest"); + + private static final org.apache.thrift.protocol.TField NUM_TXNS_FIELD_DESC = new org.apache.thrift.protocol.TField("num_txns", org.apache.thrift.protocol.TType.I32, (short)1); + private static final org.apache.thrift.protocol.TField USER_FIELD_DESC = new org.apache.thrift.protocol.TField("user", org.apache.thrift.protocol.TType.STRING, (short)2); + private static final org.apache.thrift.protocol.TField HOSTNAME_FIELD_DESC = new org.apache.thrift.protocol.TField("hostname", org.apache.thrift.protocol.TType.STRING, (short)3); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new OpenTxnRequestStandardSchemeFactory()); + schemes.put(TupleScheme.class, new OpenTxnRequestTupleSchemeFactory()); + } + + private int num_txns; // required + private String user; // required + private String hostname; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + NUM_TXNS((short)1, "num_txns"), + USER((short)2, "user"), + HOSTNAME((short)3, "hostname"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // NUM_TXNS + return NUM_TXNS; + case 2: // USER + return USER; + case 3: // HOSTNAME + return HOSTNAME; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + private static final int __NUM_TXNS_ISSET_ID = 0; + private byte __isset_bitfield = 0; + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.NUM_TXNS, new org.apache.thrift.meta_data.FieldMetaData("num_txns", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32))); + tmpMap.put(_Fields.USER, new org.apache.thrift.meta_data.FieldMetaData("user", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + tmpMap.put(_Fields.HOSTNAME, new org.apache.thrift.meta_data.FieldMetaData("hostname", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(OpenTxnRequest.class, metaDataMap); + } + + public OpenTxnRequest() { + } + + public OpenTxnRequest( + int num_txns, + String user, + String hostname) + { + this(); + this.num_txns = num_txns; + setNum_txnsIsSet(true); + this.user = user; + this.hostname = hostname; + } + + /** + * Performs a deep copy on other. + */ + public OpenTxnRequest(OpenTxnRequest other) { + __isset_bitfield = other.__isset_bitfield; + this.num_txns = other.num_txns; + if (other.isSetUser()) { + this.user = other.user; + } + if (other.isSetHostname()) { + this.hostname = other.hostname; + } + } + + public OpenTxnRequest deepCopy() { + return new OpenTxnRequest(this); + } + + @Override + public void clear() { + setNum_txnsIsSet(false); + this.num_txns = 0; + this.user = null; + this.hostname = null; + } + + public int getNum_txns() { + return this.num_txns; + } + + public void setNum_txns(int num_txns) { + this.num_txns = num_txns; + setNum_txnsIsSet(true); + } + + public void unsetNum_txns() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __NUM_TXNS_ISSET_ID); + } + + /** Returns true if field num_txns is set (has been assigned a value) and false otherwise */ + public boolean isSetNum_txns() { + return EncodingUtils.testBit(__isset_bitfield, __NUM_TXNS_ISSET_ID); + } + + public void setNum_txnsIsSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __NUM_TXNS_ISSET_ID, value); + } + + public String getUser() { + return this.user; + } + + public void setUser(String user) { + this.user = user; + } + + public void unsetUser() { + this.user = null; + } + + /** Returns true if field user is set (has been assigned a value) and false otherwise */ + public boolean isSetUser() { + return this.user != null; + } + + public void setUserIsSet(boolean value) { + if (!value) { + this.user = null; + } + } + + public String getHostname() { + return this.hostname; + } + + public void setHostname(String hostname) { + this.hostname = hostname; + } + + public void unsetHostname() { + this.hostname = null; + } + + /** Returns true if field hostname is set (has been assigned a value) and false otherwise */ + public boolean isSetHostname() { + return this.hostname != null; + } + + public void setHostnameIsSet(boolean value) { + if (!value) { + this.hostname = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case NUM_TXNS: + if (value == null) { + unsetNum_txns(); + } else { + setNum_txns((Integer)value); + } + break; + + case USER: + if (value == null) { + unsetUser(); + } else { + setUser((String)value); + } + break; + + case HOSTNAME: + if (value == null) { + unsetHostname(); + } else { + setHostname((String)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case NUM_TXNS: + return Integer.valueOf(getNum_txns()); + + case USER: + return getUser(); + + case HOSTNAME: + return getHostname(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case NUM_TXNS: + return isSetNum_txns(); + case USER: + return isSetUser(); + case HOSTNAME: + return isSetHostname(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof OpenTxnRequest) + return this.equals((OpenTxnRequest)that); + return false; + } + + public boolean equals(OpenTxnRequest that) { + if (that == null) + return false; + + boolean this_present_num_txns = true; + boolean that_present_num_txns = true; + if (this_present_num_txns || that_present_num_txns) { + if (!(this_present_num_txns && that_present_num_txns)) + return false; + if (this.num_txns != that.num_txns) + return false; + } + + boolean this_present_user = true && this.isSetUser(); + boolean that_present_user = true && that.isSetUser(); + if (this_present_user || that_present_user) { + if (!(this_present_user && that_present_user)) + return false; + if (!this.user.equals(that.user)) + return false; + } + + boolean this_present_hostname = true && this.isSetHostname(); + boolean that_present_hostname = true && that.isSetHostname(); + if (this_present_hostname || that_present_hostname) { + if (!(this_present_hostname && that_present_hostname)) + return false; + if (!this.hostname.equals(that.hostname)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_num_txns = true; + builder.append(present_num_txns); + if (present_num_txns) + builder.append(num_txns); + + boolean present_user = true && (isSetUser()); + builder.append(present_user); + if (present_user) + builder.append(user); + + boolean present_hostname = true && (isSetHostname()); + builder.append(present_hostname); + if (present_hostname) + builder.append(hostname); + + return builder.toHashCode(); + } + + public int compareTo(OpenTxnRequest other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + OpenTxnRequest typedOther = (OpenTxnRequest)other; + + lastComparison = Boolean.valueOf(isSetNum_txns()).compareTo(typedOther.isSetNum_txns()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetNum_txns()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.num_txns, typedOther.num_txns); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetUser()).compareTo(typedOther.isSetUser()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetUser()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.user, typedOther.user); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetHostname()).compareTo(typedOther.isSetHostname()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetHostname()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.hostname, typedOther.hostname); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("OpenTxnRequest("); + boolean first = true; + + sb.append("num_txns:"); + sb.append(this.num_txns); + first = false; + if (!first) sb.append(", "); + sb.append("user:"); + if (this.user == null) { + sb.append("null"); + } else { + sb.append(this.user); + } + first = false; + if (!first) sb.append(", "); + sb.append("hostname:"); + if (this.hostname == null) { + sb.append("null"); + } else { + sb.append(this.hostname); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetNum_txns()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'num_txns' is unset! Struct:" + toString()); + } + + if (!isSetUser()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'user' is unset! Struct:" + toString()); + } + + if (!isSetHostname()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'hostname' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. + __isset_bitfield = 0; + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class OpenTxnRequestStandardSchemeFactory implements SchemeFactory { + public OpenTxnRequestStandardScheme getScheme() { + return new OpenTxnRequestStandardScheme(); + } + } + + private static class OpenTxnRequestStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, OpenTxnRequest struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // NUM_TXNS + if (schemeField.type == org.apache.thrift.protocol.TType.I32) { + struct.num_txns = iprot.readI32(); + struct.setNum_txnsIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // USER + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.user = iprot.readString(); + struct.setUserIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 3: // HOSTNAME + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.hostname = iprot.readString(); + struct.setHostnameIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, OpenTxnRequest struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + oprot.writeFieldBegin(NUM_TXNS_FIELD_DESC); + oprot.writeI32(struct.num_txns); + oprot.writeFieldEnd(); + if (struct.user != null) { + oprot.writeFieldBegin(USER_FIELD_DESC); + oprot.writeString(struct.user); + oprot.writeFieldEnd(); + } + if (struct.hostname != null) { + oprot.writeFieldBegin(HOSTNAME_FIELD_DESC); + oprot.writeString(struct.hostname); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class OpenTxnRequestTupleSchemeFactory implements SchemeFactory { + public OpenTxnRequestTupleScheme getScheme() { + return new OpenTxnRequestTupleScheme(); + } + } + + private static class OpenTxnRequestTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, OpenTxnRequest struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + oprot.writeI32(struct.num_txns); + oprot.writeString(struct.user); + oprot.writeString(struct.hostname); + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, OpenTxnRequest struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.num_txns = iprot.readI32(); + struct.setNum_txnsIsSet(true); + struct.user = iprot.readString(); + struct.setUserIsSet(true); + struct.hostname = iprot.readString(); + struct.setHostnameIsSet(true); + } + } + +} + diff --git metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/OpenTxnsResponse.java metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/OpenTxnsResponse.java new file mode 100644 index 0000000..416379a --- /dev/null +++ metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/OpenTxnsResponse.java @@ -0,0 +1,437 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hadoop.hive.metastore.api; + +import org.apache.commons.lang.builder.HashCodeBuilder; +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class OpenTxnsResponse implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("OpenTxnsResponse"); + + private static final org.apache.thrift.protocol.TField TXN_IDS_FIELD_DESC = new org.apache.thrift.protocol.TField("txn_ids", org.apache.thrift.protocol.TType.LIST, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new OpenTxnsResponseStandardSchemeFactory()); + schemes.put(TupleScheme.class, new OpenTxnsResponseTupleSchemeFactory()); + } + + private List txn_ids; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + TXN_IDS((short)1, "txn_ids"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // TXN_IDS + return TXN_IDS; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.TXN_IDS, new org.apache.thrift.meta_data.FieldMetaData("txn_ids", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(OpenTxnsResponse.class, metaDataMap); + } + + public OpenTxnsResponse() { + } + + public OpenTxnsResponse( + List txn_ids) + { + this(); + this.txn_ids = txn_ids; + } + + /** + * Performs a deep copy on other. + */ + public OpenTxnsResponse(OpenTxnsResponse other) { + if (other.isSetTxn_ids()) { + List __this__txn_ids = new ArrayList(); + for (Long other_element : other.txn_ids) { + __this__txn_ids.add(other_element); + } + this.txn_ids = __this__txn_ids; + } + } + + public OpenTxnsResponse deepCopy() { + return new OpenTxnsResponse(this); + } + + @Override + public void clear() { + this.txn_ids = null; + } + + public int getTxn_idsSize() { + return (this.txn_ids == null) ? 0 : this.txn_ids.size(); + } + + public java.util.Iterator getTxn_idsIterator() { + return (this.txn_ids == null) ? null : this.txn_ids.iterator(); + } + + public void addToTxn_ids(long elem) { + if (this.txn_ids == null) { + this.txn_ids = new ArrayList(); + } + this.txn_ids.add(elem); + } + + public List getTxn_ids() { + return this.txn_ids; + } + + public void setTxn_ids(List txn_ids) { + this.txn_ids = txn_ids; + } + + public void unsetTxn_ids() { + this.txn_ids = null; + } + + /** Returns true if field txn_ids is set (has been assigned a value) and false otherwise */ + public boolean isSetTxn_ids() { + return this.txn_ids != null; + } + + public void setTxn_idsIsSet(boolean value) { + if (!value) { + this.txn_ids = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case TXN_IDS: + if (value == null) { + unsetTxn_ids(); + } else { + setTxn_ids((List)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case TXN_IDS: + return getTxn_ids(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case TXN_IDS: + return isSetTxn_ids(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof OpenTxnsResponse) + return this.equals((OpenTxnsResponse)that); + return false; + } + + public boolean equals(OpenTxnsResponse that) { + if (that == null) + return false; + + boolean this_present_txn_ids = true && this.isSetTxn_ids(); + boolean that_present_txn_ids = true && that.isSetTxn_ids(); + if (this_present_txn_ids || that_present_txn_ids) { + if (!(this_present_txn_ids && that_present_txn_ids)) + return false; + if (!this.txn_ids.equals(that.txn_ids)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_txn_ids = true && (isSetTxn_ids()); + builder.append(present_txn_ids); + if (present_txn_ids) + builder.append(txn_ids); + + return builder.toHashCode(); + } + + public int compareTo(OpenTxnsResponse other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + OpenTxnsResponse typedOther = (OpenTxnsResponse)other; + + lastComparison = Boolean.valueOf(isSetTxn_ids()).compareTo(typedOther.isSetTxn_ids()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetTxn_ids()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.txn_ids, typedOther.txn_ids); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("OpenTxnsResponse("); + boolean first = true; + + sb.append("txn_ids:"); + if (this.txn_ids == null) { + sb.append("null"); + } else { + sb.append(this.txn_ids); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetTxn_ids()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'txn_ids' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class OpenTxnsResponseStandardSchemeFactory implements SchemeFactory { + public OpenTxnsResponseStandardScheme getScheme() { + return new OpenTxnsResponseStandardScheme(); + } + } + + private static class OpenTxnsResponseStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, OpenTxnsResponse struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // TXN_IDS + if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { + { + org.apache.thrift.protocol.TList _list378 = iprot.readListBegin(); + struct.txn_ids = new ArrayList(_list378.size); + for (int _i379 = 0; _i379 < _list378.size; ++_i379) + { + long _elem380; // required + _elem380 = iprot.readI64(); + struct.txn_ids.add(_elem380); + } + iprot.readListEnd(); + } + struct.setTxn_idsIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, OpenTxnsResponse struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.txn_ids != null) { + oprot.writeFieldBegin(TXN_IDS_FIELD_DESC); + { + oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, struct.txn_ids.size())); + for (long _iter381 : struct.txn_ids) + { + oprot.writeI64(_iter381); + } + oprot.writeListEnd(); + } + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class OpenTxnsResponseTupleSchemeFactory implements SchemeFactory { + public OpenTxnsResponseTupleScheme getScheme() { + return new OpenTxnsResponseTupleScheme(); + } + } + + private static class OpenTxnsResponseTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, OpenTxnsResponse struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + { + oprot.writeI32(struct.txn_ids.size()); + for (long _iter382 : struct.txn_ids) + { + oprot.writeI64(_iter382); + } + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, OpenTxnsResponse struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + { + org.apache.thrift.protocol.TList _list383 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32()); + struct.txn_ids = new ArrayList(_list383.size); + for (int _i384 = 0; _i384 < _list383.size; ++_i384) + { + long _elem385; // required + _elem385 = iprot.readI64(); + struct.txn_ids.add(_elem385); + } + } + struct.setTxn_idsIsSet(true); + } + } + +} + diff --git metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Partition.java metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Partition.java index 45589aa..4329d34 100644 --- metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Partition.java +++ metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Partition.java @@ -945,7 +945,7 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, Partition struct) t struct.values = new ArrayList(_list192.size); for (int _i193 = 0; _i193 < _list192.size; ++_i193) { - String _elem194; // optional + String _elem194; // required _elem194 = iprot.readString(); struct.values.add(_elem194); } @@ -1184,7 +1184,7 @@ public void read(org.apache.thrift.protocol.TProtocol prot, Partition struct) th struct.values = new ArrayList(_list203.size); for (int _i204 = 0; _i204 < _list203.size; ++_i204) { - String _elem205; // optional + String _elem205; // required _elem205 = iprot.readString(); struct.values.add(_elem205); } diff --git metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionsByExprResult.java metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionsByExprResult.java index 04229d5..a4bdeaa 100644 --- metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionsByExprResult.java +++ metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionsByExprResult.java @@ -439,7 +439,7 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, PartitionsByExprRes struct.partitions = new ArrayList(_list256.size); for (int _i257 = 0; _i257 < _list256.size; ++_i257) { - Partition _elem258; // optional + Partition _elem258; // required _elem258 = new Partition(); _elem258.read(iprot); struct.partitions.add(_elem258); @@ -522,7 +522,7 @@ public void read(org.apache.thrift.protocol.TProtocol prot, PartitionsByExprResu struct.partitions = new ArrayList(_list261.size); for (int _i262 = 0; _i262 < _list261.size; ++_i262) { - Partition _elem263; // optional + Partition _elem263; // required _elem263 = new Partition(); _elem263.read(iprot); struct.partitions.add(_elem263); diff --git metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionsStatsRequest.java metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionsStatsRequest.java index d24539d..e7a90f4 100644 --- metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionsStatsRequest.java +++ metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionsStatsRequest.java @@ -645,7 +645,7 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, PartitionsStatsRequ struct.colNames = new ArrayList(_list298.size); for (int _i299 = 0; _i299 < _list298.size; ++_i299) { - String _elem300; // optional + String _elem300; // required _elem300 = iprot.readString(); struct.colNames.add(_elem300); } @@ -663,7 +663,7 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, PartitionsStatsRequ struct.partNames = new ArrayList(_list301.size); for (int _i302 = 0; _i302 < _list301.size; ++_i302) { - String _elem303; // optional + String _elem303; // required _elem303 = iprot.readString(); struct.partNames.add(_elem303); } @@ -768,7 +768,7 @@ public void read(org.apache.thrift.protocol.TProtocol prot, PartitionsStatsReque struct.colNames = new ArrayList(_list308.size); for (int _i309 = 0; _i309 < _list308.size; ++_i309) { - String _elem310; // optional + String _elem310; // required _elem310 = iprot.readString(); struct.colNames.add(_elem310); } @@ -779,7 +779,7 @@ public void read(org.apache.thrift.protocol.TProtocol prot, PartitionsStatsReque struct.partNames = new ArrayList(_list311.size); for (int _i312 = 0; _i312 < _list311.size; ++_i312) { - String _elem313; // optional + String _elem313; // required _elem313 = iprot.readString(); struct.partNames.add(_elem313); } diff --git metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionsStatsResult.java metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionsStatsResult.java index a0df5f6..bd8881d 100644 --- metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionsStatsResult.java +++ metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionsStatsResult.java @@ -371,7 +371,7 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, PartitionsStatsResu _val275 = new ArrayList(_list276.size); for (int _i277 = 0; _i277 < _list276.size; ++_i277) { - ColumnStatisticsObj _elem278; // optional + ColumnStatisticsObj _elem278; // required _elem278 = new ColumnStatisticsObj(); _elem278.read(iprot); _val275.add(_elem278); @@ -469,7 +469,7 @@ public void read(org.apache.thrift.protocol.TProtocol prot, PartitionsStatsResul _val286 = new ArrayList(_list287.size); for (int _i288 = 0; _i288 < _list287.size; ++_i288) { - ColumnStatisticsObj _elem289; // optional + ColumnStatisticsObj _elem289; // required _elem289 = new ColumnStatisticsObj(); _elem289.read(iprot); _val286.add(_elem289); diff --git metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PrincipalPrivilegeSet.java metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PrincipalPrivilegeSet.java index 0c9518a..eea86e5 100644 --- metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PrincipalPrivilegeSet.java +++ metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PrincipalPrivilegeSet.java @@ -587,7 +587,7 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, PrincipalPrivilegeS _val27 = new ArrayList(_list28.size); for (int _i29 = 0; _i29 < _list28.size; ++_i29) { - PrivilegeGrantInfo _elem30; // optional + PrivilegeGrantInfo _elem30; // required _elem30 = new PrivilegeGrantInfo(); _elem30.read(iprot); _val27.add(_elem30); @@ -618,7 +618,7 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, PrincipalPrivilegeS _val34 = new ArrayList(_list35.size); for (int _i36 = 0; _i36 < _list35.size; ++_i36) { - PrivilegeGrantInfo _elem37; // optional + PrivilegeGrantInfo _elem37; // required _elem37 = new PrivilegeGrantInfo(); _elem37.read(iprot); _val34.add(_elem37); @@ -649,7 +649,7 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, PrincipalPrivilegeS _val41 = new ArrayList(_list42.size); for (int _i43 = 0; _i43 < _list42.size; ++_i43) { - PrivilegeGrantInfo _elem44; // optional + PrivilegeGrantInfo _elem44; // required _elem44 = new PrivilegeGrantInfo(); _elem44.read(iprot); _val41.add(_elem44); @@ -834,7 +834,7 @@ public void read(org.apache.thrift.protocol.TProtocol prot, PrincipalPrivilegeSe _val60 = new ArrayList(_list61.size); for (int _i62 = 0; _i62 < _list61.size; ++_i62) { - PrivilegeGrantInfo _elem63; // optional + PrivilegeGrantInfo _elem63; // required _elem63 = new PrivilegeGrantInfo(); _elem63.read(iprot); _val60.add(_elem63); @@ -859,7 +859,7 @@ public void read(org.apache.thrift.protocol.TProtocol prot, PrincipalPrivilegeSe _val67 = new ArrayList(_list68.size); for (int _i69 = 0; _i69 < _list68.size; ++_i69) { - PrivilegeGrantInfo _elem70; // optional + PrivilegeGrantInfo _elem70; // required _elem70 = new PrivilegeGrantInfo(); _elem70.read(iprot); _val67.add(_elem70); @@ -884,7 +884,7 @@ public void read(org.apache.thrift.protocol.TProtocol prot, PrincipalPrivilegeSe _val74 = new ArrayList(_list75.size); for (int _i76 = 0; _i76 < _list75.size; ++_i76) { - PrivilegeGrantInfo _elem77; // optional + PrivilegeGrantInfo _elem77; // required _elem77 = new PrivilegeGrantInfo(); _elem77.read(iprot); _val74.add(_elem77); diff --git metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PrivilegeBag.java metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PrivilegeBag.java index 4285ed8..a4687ad 100644 --- metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PrivilegeBag.java +++ metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PrivilegeBag.java @@ -350,7 +350,7 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, PrivilegeBag struct struct.privileges = new ArrayList(_list16.size); for (int _i17 = 0; _i17 < _list16.size; ++_i17) { - HiveObjectPrivilege _elem18; // optional + HiveObjectPrivilege _elem18; // required _elem18 = new HiveObjectPrivilege(); _elem18.read(iprot); struct.privileges.add(_elem18); @@ -430,7 +430,7 @@ public void read(org.apache.thrift.protocol.TProtocol prot, PrivilegeBag struct) struct.privileges = new ArrayList(_list21.size); for (int _i22 = 0; _i22 < _list21.size; ++_i22) { - HiveObjectPrivilege _elem23; // optional + HiveObjectPrivilege _elem23; // required _elem23 = new HiveObjectPrivilege(); _elem23.read(iprot); struct.privileges.add(_elem23); diff --git metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/RequestPartsSpec.java metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/RequestPartsSpec.java index 08d3ef8..2577059 100644 --- metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/RequestPartsSpec.java +++ metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/RequestPartsSpec.java @@ -169,7 +169,7 @@ protected Object standardSchemeReadValue(org.apache.thrift.protocol.TProtocol ip names = new ArrayList(_list338.size); for (int _i339 = 0; _i339 < _list338.size; ++_i339) { - String _elem340; // optional + String _elem340; // required _elem340 = iprot.readString(); names.add(_elem340); } @@ -188,7 +188,7 @@ protected Object standardSchemeReadValue(org.apache.thrift.protocol.TProtocol ip exprs = new ArrayList(_list341.size); for (int _i342 = 0; _i342 < _list341.size; ++_i342) { - DropPartitionsExpr _elem343; // optional + DropPartitionsExpr _elem343; // required _elem343 = new DropPartitionsExpr(); _elem343.read(iprot); exprs.add(_elem343); @@ -250,7 +250,7 @@ protected Object tupleSchemeReadValue(org.apache.thrift.protocol.TProtocol iprot names = new ArrayList(_list346.size); for (int _i347 = 0; _i347 < _list346.size; ++_i347) { - String _elem348; // optional + String _elem348; // required _elem348 = iprot.readString(); names.add(_elem348); } @@ -264,7 +264,7 @@ protected Object tupleSchemeReadValue(org.apache.thrift.protocol.TProtocol iprot exprs = new ArrayList(_list349.size); for (int _i350 = 0; _i350 < _list349.size; ++_i350) { - DropPartitionsExpr _elem351; // optional + DropPartitionsExpr _elem351; // required _elem351 = new DropPartitionsExpr(); _elem351.read(iprot); exprs.add(_elem351); diff --git metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Schema.java metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Schema.java index c62a828..f6af8d9 100644 --- metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Schema.java +++ metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Schema.java @@ -456,7 +456,7 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, Schema struct) thro struct.fieldSchemas = new ArrayList(_list228.size); for (int _i229 = 0; _i229 < _list228.size; ++_i229) { - FieldSchema _elem230; // optional + FieldSchema _elem230; // required _elem230 = new FieldSchema(); _elem230.read(iprot); struct.fieldSchemas.add(_elem230); @@ -582,7 +582,7 @@ public void read(org.apache.thrift.protocol.TProtocol prot, Schema struct) throw struct.fieldSchemas = new ArrayList(_list239.size); for (int _i240 = 0; _i240 < _list239.size; ++_i240) { - FieldSchema _elem241; // optional + FieldSchema _elem241; // required _elem241 = new FieldSchema(); _elem241.read(iprot); struct.fieldSchemas.add(_elem241); diff --git metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowCompactRequest.java metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowCompactRequest.java new file mode 100644 index 0000000..28ad1c9 --- /dev/null +++ metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowCompactRequest.java @@ -0,0 +1,279 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hadoop.hive.metastore.api; + +import org.apache.commons.lang.builder.HashCodeBuilder; +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class ShowCompactRequest implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("ShowCompactRequest"); + + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new ShowCompactRequestStandardSchemeFactory()); + schemes.put(TupleScheme.class, new ShowCompactRequestTupleSchemeFactory()); + } + + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { +; + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(ShowCompactRequest.class, metaDataMap); + } + + public ShowCompactRequest() { + } + + /** + * Performs a deep copy on other. + */ + public ShowCompactRequest(ShowCompactRequest other) { + } + + public ShowCompactRequest deepCopy() { + return new ShowCompactRequest(this); + } + + @Override + public void clear() { + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof ShowCompactRequest) + return this.equals((ShowCompactRequest)that); + return false; + } + + public boolean equals(ShowCompactRequest that) { + if (that == null) + return false; + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + return builder.toHashCode(); + } + + public int compareTo(ShowCompactRequest other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + ShowCompactRequest typedOther = (ShowCompactRequest)other; + + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("ShowCompactRequest("); + boolean first = true; + + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class ShowCompactRequestStandardSchemeFactory implements SchemeFactory { + public ShowCompactRequestStandardScheme getScheme() { + return new ShowCompactRequestStandardScheme(); + } + } + + private static class ShowCompactRequestStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, ShowCompactRequest struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, ShowCompactRequest struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class ShowCompactRequestTupleSchemeFactory implements SchemeFactory { + public ShowCompactRequestTupleScheme getScheme() { + return new ShowCompactRequestTupleScheme(); + } + } + + private static class ShowCompactRequestTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, ShowCompactRequest struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, ShowCompactRequest struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + } + } + +} + diff --git metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowCompactResponse.java metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowCompactResponse.java new file mode 100644 index 0000000..6da28dd --- /dev/null +++ metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowCompactResponse.java @@ -0,0 +1,439 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hadoop.hive.metastore.api; + +import org.apache.commons.lang.builder.HashCodeBuilder; +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class ShowCompactResponse implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("ShowCompactResponse"); + + private static final org.apache.thrift.protocol.TField COMPACTS_FIELD_DESC = new org.apache.thrift.protocol.TField("compacts", org.apache.thrift.protocol.TType.LIST, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new ShowCompactResponseStandardSchemeFactory()); + schemes.put(TupleScheme.class, new ShowCompactResponseTupleSchemeFactory()); + } + + private List compacts; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + COMPACTS((short)1, "compacts"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // COMPACTS + return COMPACTS; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.COMPACTS, new org.apache.thrift.meta_data.FieldMetaData("compacts", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ShowCompactResponseElement.class)))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(ShowCompactResponse.class, metaDataMap); + } + + public ShowCompactResponse() { + } + + public ShowCompactResponse( + List compacts) + { + this(); + this.compacts = compacts; + } + + /** + * Performs a deep copy on other. + */ + public ShowCompactResponse(ShowCompactResponse other) { + if (other.isSetCompacts()) { + List __this__compacts = new ArrayList(); + for (ShowCompactResponseElement other_element : other.compacts) { + __this__compacts.add(new ShowCompactResponseElement(other_element)); + } + this.compacts = __this__compacts; + } + } + + public ShowCompactResponse deepCopy() { + return new ShowCompactResponse(this); + } + + @Override + public void clear() { + this.compacts = null; + } + + public int getCompactsSize() { + return (this.compacts == null) ? 0 : this.compacts.size(); + } + + public java.util.Iterator getCompactsIterator() { + return (this.compacts == null) ? null : this.compacts.iterator(); + } + + public void addToCompacts(ShowCompactResponseElement elem) { + if (this.compacts == null) { + this.compacts = new ArrayList(); + } + this.compacts.add(elem); + } + + public List getCompacts() { + return this.compacts; + } + + public void setCompacts(List compacts) { + this.compacts = compacts; + } + + public void unsetCompacts() { + this.compacts = null; + } + + /** Returns true if field compacts is set (has been assigned a value) and false otherwise */ + public boolean isSetCompacts() { + return this.compacts != null; + } + + public void setCompactsIsSet(boolean value) { + if (!value) { + this.compacts = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case COMPACTS: + if (value == null) { + unsetCompacts(); + } else { + setCompacts((List)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case COMPACTS: + return getCompacts(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case COMPACTS: + return isSetCompacts(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof ShowCompactResponse) + return this.equals((ShowCompactResponse)that); + return false; + } + + public boolean equals(ShowCompactResponse that) { + if (that == null) + return false; + + boolean this_present_compacts = true && this.isSetCompacts(); + boolean that_present_compacts = true && that.isSetCompacts(); + if (this_present_compacts || that_present_compacts) { + if (!(this_present_compacts && that_present_compacts)) + return false; + if (!this.compacts.equals(that.compacts)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_compacts = true && (isSetCompacts()); + builder.append(present_compacts); + if (present_compacts) + builder.append(compacts); + + return builder.toHashCode(); + } + + public int compareTo(ShowCompactResponse other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + ShowCompactResponse typedOther = (ShowCompactResponse)other; + + lastComparison = Boolean.valueOf(isSetCompacts()).compareTo(typedOther.isSetCompacts()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetCompacts()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.compacts, typedOther.compacts); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("ShowCompactResponse("); + boolean first = true; + + sb.append("compacts:"); + if (this.compacts == null) { + sb.append("null"); + } else { + sb.append(this.compacts); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetCompacts()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'compacts' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class ShowCompactResponseStandardSchemeFactory implements SchemeFactory { + public ShowCompactResponseStandardScheme getScheme() { + return new ShowCompactResponseStandardScheme(); + } + } + + private static class ShowCompactResponseStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, ShowCompactResponse struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // COMPACTS + if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { + { + org.apache.thrift.protocol.TList _list402 = iprot.readListBegin(); + struct.compacts = new ArrayList(_list402.size); + for (int _i403 = 0; _i403 < _list402.size; ++_i403) + { + ShowCompactResponseElement _elem404; // required + _elem404 = new ShowCompactResponseElement(); + _elem404.read(iprot); + struct.compacts.add(_elem404); + } + iprot.readListEnd(); + } + struct.setCompactsIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, ShowCompactResponse struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.compacts != null) { + oprot.writeFieldBegin(COMPACTS_FIELD_DESC); + { + oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.compacts.size())); + for (ShowCompactResponseElement _iter405 : struct.compacts) + { + _iter405.write(oprot); + } + oprot.writeListEnd(); + } + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class ShowCompactResponseTupleSchemeFactory implements SchemeFactory { + public ShowCompactResponseTupleScheme getScheme() { + return new ShowCompactResponseTupleScheme(); + } + } + + private static class ShowCompactResponseTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, ShowCompactResponse struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + { + oprot.writeI32(struct.compacts.size()); + for (ShowCompactResponseElement _iter406 : struct.compacts) + { + _iter406.write(oprot); + } + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, ShowCompactResponse struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + { + org.apache.thrift.protocol.TList _list407 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); + struct.compacts = new ArrayList(_list407.size); + for (int _i408 = 0; _i408 < _list407.size; ++_i408) + { + ShowCompactResponseElement _elem409; // required + _elem409 = new ShowCompactResponseElement(); + _elem409.read(iprot); + struct.compacts.add(_elem409); + } + } + struct.setCompactsIsSet(true); + } + } + +} + diff --git metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowCompactResponseElement.java metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowCompactResponseElement.java new file mode 100644 index 0000000..47da9b3 --- /dev/null +++ metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowCompactResponseElement.java @@ -0,0 +1,1102 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hadoop.hive.metastore.api; + +import org.apache.commons.lang.builder.HashCodeBuilder; +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class ShowCompactResponseElement implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("ShowCompactResponseElement"); + + private static final org.apache.thrift.protocol.TField DBNAME_FIELD_DESC = new org.apache.thrift.protocol.TField("dbname", org.apache.thrift.protocol.TType.STRING, (short)1); + private static final org.apache.thrift.protocol.TField TABLENAME_FIELD_DESC = new org.apache.thrift.protocol.TField("tablename", org.apache.thrift.protocol.TType.STRING, (short)2); + private static final org.apache.thrift.protocol.TField PARTITIONNAME_FIELD_DESC = new org.apache.thrift.protocol.TField("partitionname", org.apache.thrift.protocol.TType.STRING, (short)3); + private static final org.apache.thrift.protocol.TField TYPE_FIELD_DESC = new org.apache.thrift.protocol.TField("type", org.apache.thrift.protocol.TType.I32, (short)4); + private static final org.apache.thrift.protocol.TField STATE_FIELD_DESC = new org.apache.thrift.protocol.TField("state", org.apache.thrift.protocol.TType.STRING, (short)5); + private static final org.apache.thrift.protocol.TField WORKERID_FIELD_DESC = new org.apache.thrift.protocol.TField("workerid", org.apache.thrift.protocol.TType.STRING, (short)6); + private static final org.apache.thrift.protocol.TField START_FIELD_DESC = new org.apache.thrift.protocol.TField("start", org.apache.thrift.protocol.TType.I64, (short)7); + private static final org.apache.thrift.protocol.TField RUN_AS_FIELD_DESC = new org.apache.thrift.protocol.TField("runAs", org.apache.thrift.protocol.TType.STRING, (short)8); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new ShowCompactResponseElementStandardSchemeFactory()); + schemes.put(TupleScheme.class, new ShowCompactResponseElementTupleSchemeFactory()); + } + + private String dbname; // required + private String tablename; // required + private String partitionname; // required + private CompactionType type; // required + private String state; // required + private String workerid; // required + private long start; // required + private String runAs; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + DBNAME((short)1, "dbname"), + TABLENAME((short)2, "tablename"), + PARTITIONNAME((short)3, "partitionname"), + /** + * + * @see CompactionType + */ + TYPE((short)4, "type"), + STATE((short)5, "state"), + WORKERID((short)6, "workerid"), + START((short)7, "start"), + RUN_AS((short)8, "runAs"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // DBNAME + return DBNAME; + case 2: // TABLENAME + return TABLENAME; + case 3: // PARTITIONNAME + return PARTITIONNAME; + case 4: // TYPE + return TYPE; + case 5: // STATE + return STATE; + case 6: // WORKERID + return WORKERID; + case 7: // START + return START; + case 8: // RUN_AS + return RUN_AS; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + private static final int __START_ISSET_ID = 0; + private byte __isset_bitfield = 0; + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.DBNAME, new org.apache.thrift.meta_data.FieldMetaData("dbname", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + tmpMap.put(_Fields.TABLENAME, new org.apache.thrift.meta_data.FieldMetaData("tablename", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + tmpMap.put(_Fields.PARTITIONNAME, new org.apache.thrift.meta_data.FieldMetaData("partitionname", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + tmpMap.put(_Fields.TYPE, new org.apache.thrift.meta_data.FieldMetaData("type", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, CompactionType.class))); + tmpMap.put(_Fields.STATE, new org.apache.thrift.meta_data.FieldMetaData("state", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + tmpMap.put(_Fields.WORKERID, new org.apache.thrift.meta_data.FieldMetaData("workerid", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + tmpMap.put(_Fields.START, new org.apache.thrift.meta_data.FieldMetaData("start", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64))); + tmpMap.put(_Fields.RUN_AS, new org.apache.thrift.meta_data.FieldMetaData("runAs", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(ShowCompactResponseElement.class, metaDataMap); + } + + public ShowCompactResponseElement() { + } + + public ShowCompactResponseElement( + String dbname, + String tablename, + String partitionname, + CompactionType type, + String state, + String workerid, + long start, + String runAs) + { + this(); + this.dbname = dbname; + this.tablename = tablename; + this.partitionname = partitionname; + this.type = type; + this.state = state; + this.workerid = workerid; + this.start = start; + setStartIsSet(true); + this.runAs = runAs; + } + + /** + * Performs a deep copy on other. + */ + public ShowCompactResponseElement(ShowCompactResponseElement other) { + __isset_bitfield = other.__isset_bitfield; + if (other.isSetDbname()) { + this.dbname = other.dbname; + } + if (other.isSetTablename()) { + this.tablename = other.tablename; + } + if (other.isSetPartitionname()) { + this.partitionname = other.partitionname; + } + if (other.isSetType()) { + this.type = other.type; + } + if (other.isSetState()) { + this.state = other.state; + } + if (other.isSetWorkerid()) { + this.workerid = other.workerid; + } + this.start = other.start; + if (other.isSetRunAs()) { + this.runAs = other.runAs; + } + } + + public ShowCompactResponseElement deepCopy() { + return new ShowCompactResponseElement(this); + } + + @Override + public void clear() { + this.dbname = null; + this.tablename = null; + this.partitionname = null; + this.type = null; + this.state = null; + this.workerid = null; + setStartIsSet(false); + this.start = 0; + this.runAs = null; + } + + public String getDbname() { + return this.dbname; + } + + public void setDbname(String dbname) { + this.dbname = dbname; + } + + public void unsetDbname() { + this.dbname = null; + } + + /** Returns true if field dbname is set (has been assigned a value) and false otherwise */ + public boolean isSetDbname() { + return this.dbname != null; + } + + public void setDbnameIsSet(boolean value) { + if (!value) { + this.dbname = null; + } + } + + public String getTablename() { + return this.tablename; + } + + public void setTablename(String tablename) { + this.tablename = tablename; + } + + public void unsetTablename() { + this.tablename = null; + } + + /** Returns true if field tablename is set (has been assigned a value) and false otherwise */ + public boolean isSetTablename() { + return this.tablename != null; + } + + public void setTablenameIsSet(boolean value) { + if (!value) { + this.tablename = null; + } + } + + public String getPartitionname() { + return this.partitionname; + } + + public void setPartitionname(String partitionname) { + this.partitionname = partitionname; + } + + public void unsetPartitionname() { + this.partitionname = null; + } + + /** Returns true if field partitionname is set (has been assigned a value) and false otherwise */ + public boolean isSetPartitionname() { + return this.partitionname != null; + } + + public void setPartitionnameIsSet(boolean value) { + if (!value) { + this.partitionname = null; + } + } + + /** + * + * @see CompactionType + */ + public CompactionType getType() { + return this.type; + } + + /** + * + * @see CompactionType + */ + public void setType(CompactionType type) { + this.type = type; + } + + public void unsetType() { + this.type = null; + } + + /** Returns true if field type is set (has been assigned a value) and false otherwise */ + public boolean isSetType() { + return this.type != null; + } + + public void setTypeIsSet(boolean value) { + if (!value) { + this.type = null; + } + } + + public String getState() { + return this.state; + } + + public void setState(String state) { + this.state = state; + } + + public void unsetState() { + this.state = null; + } + + /** Returns true if field state is set (has been assigned a value) and false otherwise */ + public boolean isSetState() { + return this.state != null; + } + + public void setStateIsSet(boolean value) { + if (!value) { + this.state = null; + } + } + + public String getWorkerid() { + return this.workerid; + } + + public void setWorkerid(String workerid) { + this.workerid = workerid; + } + + public void unsetWorkerid() { + this.workerid = null; + } + + /** Returns true if field workerid is set (has been assigned a value) and false otherwise */ + public boolean isSetWorkerid() { + return this.workerid != null; + } + + public void setWorkeridIsSet(boolean value) { + if (!value) { + this.workerid = null; + } + } + + public long getStart() { + return this.start; + } + + public void setStart(long start) { + this.start = start; + setStartIsSet(true); + } + + public void unsetStart() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __START_ISSET_ID); + } + + /** Returns true if field start is set (has been assigned a value) and false otherwise */ + public boolean isSetStart() { + return EncodingUtils.testBit(__isset_bitfield, __START_ISSET_ID); + } + + public void setStartIsSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __START_ISSET_ID, value); + } + + public String getRunAs() { + return this.runAs; + } + + public void setRunAs(String runAs) { + this.runAs = runAs; + } + + public void unsetRunAs() { + this.runAs = null; + } + + /** Returns true if field runAs is set (has been assigned a value) and false otherwise */ + public boolean isSetRunAs() { + return this.runAs != null; + } + + public void setRunAsIsSet(boolean value) { + if (!value) { + this.runAs = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case DBNAME: + if (value == null) { + unsetDbname(); + } else { + setDbname((String)value); + } + break; + + case TABLENAME: + if (value == null) { + unsetTablename(); + } else { + setTablename((String)value); + } + break; + + case PARTITIONNAME: + if (value == null) { + unsetPartitionname(); + } else { + setPartitionname((String)value); + } + break; + + case TYPE: + if (value == null) { + unsetType(); + } else { + setType((CompactionType)value); + } + break; + + case STATE: + if (value == null) { + unsetState(); + } else { + setState((String)value); + } + break; + + case WORKERID: + if (value == null) { + unsetWorkerid(); + } else { + setWorkerid((String)value); + } + break; + + case START: + if (value == null) { + unsetStart(); + } else { + setStart((Long)value); + } + break; + + case RUN_AS: + if (value == null) { + unsetRunAs(); + } else { + setRunAs((String)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case DBNAME: + return getDbname(); + + case TABLENAME: + return getTablename(); + + case PARTITIONNAME: + return getPartitionname(); + + case TYPE: + return getType(); + + case STATE: + return getState(); + + case WORKERID: + return getWorkerid(); + + case START: + return Long.valueOf(getStart()); + + case RUN_AS: + return getRunAs(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case DBNAME: + return isSetDbname(); + case TABLENAME: + return isSetTablename(); + case PARTITIONNAME: + return isSetPartitionname(); + case TYPE: + return isSetType(); + case STATE: + return isSetState(); + case WORKERID: + return isSetWorkerid(); + case START: + return isSetStart(); + case RUN_AS: + return isSetRunAs(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof ShowCompactResponseElement) + return this.equals((ShowCompactResponseElement)that); + return false; + } + + public boolean equals(ShowCompactResponseElement that) { + if (that == null) + return false; + + boolean this_present_dbname = true && this.isSetDbname(); + boolean that_present_dbname = true && that.isSetDbname(); + if (this_present_dbname || that_present_dbname) { + if (!(this_present_dbname && that_present_dbname)) + return false; + if (!this.dbname.equals(that.dbname)) + return false; + } + + boolean this_present_tablename = true && this.isSetTablename(); + boolean that_present_tablename = true && that.isSetTablename(); + if (this_present_tablename || that_present_tablename) { + if (!(this_present_tablename && that_present_tablename)) + return false; + if (!this.tablename.equals(that.tablename)) + return false; + } + + boolean this_present_partitionname = true && this.isSetPartitionname(); + boolean that_present_partitionname = true && that.isSetPartitionname(); + if (this_present_partitionname || that_present_partitionname) { + if (!(this_present_partitionname && that_present_partitionname)) + return false; + if (!this.partitionname.equals(that.partitionname)) + return false; + } + + boolean this_present_type = true && this.isSetType(); + boolean that_present_type = true && that.isSetType(); + if (this_present_type || that_present_type) { + if (!(this_present_type && that_present_type)) + return false; + if (!this.type.equals(that.type)) + return false; + } + + boolean this_present_state = true && this.isSetState(); + boolean that_present_state = true && that.isSetState(); + if (this_present_state || that_present_state) { + if (!(this_present_state && that_present_state)) + return false; + if (!this.state.equals(that.state)) + return false; + } + + boolean this_present_workerid = true && this.isSetWorkerid(); + boolean that_present_workerid = true && that.isSetWorkerid(); + if (this_present_workerid || that_present_workerid) { + if (!(this_present_workerid && that_present_workerid)) + return false; + if (!this.workerid.equals(that.workerid)) + return false; + } + + boolean this_present_start = true; + boolean that_present_start = true; + if (this_present_start || that_present_start) { + if (!(this_present_start && that_present_start)) + return false; + if (this.start != that.start) + return false; + } + + boolean this_present_runAs = true && this.isSetRunAs(); + boolean that_present_runAs = true && that.isSetRunAs(); + if (this_present_runAs || that_present_runAs) { + if (!(this_present_runAs && that_present_runAs)) + return false; + if (!this.runAs.equals(that.runAs)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_dbname = true && (isSetDbname()); + builder.append(present_dbname); + if (present_dbname) + builder.append(dbname); + + boolean present_tablename = true && (isSetTablename()); + builder.append(present_tablename); + if (present_tablename) + builder.append(tablename); + + boolean present_partitionname = true && (isSetPartitionname()); + builder.append(present_partitionname); + if (present_partitionname) + builder.append(partitionname); + + boolean present_type = true && (isSetType()); + builder.append(present_type); + if (present_type) + builder.append(type.getValue()); + + boolean present_state = true && (isSetState()); + builder.append(present_state); + if (present_state) + builder.append(state); + + boolean present_workerid = true && (isSetWorkerid()); + builder.append(present_workerid); + if (present_workerid) + builder.append(workerid); + + boolean present_start = true; + builder.append(present_start); + if (present_start) + builder.append(start); + + boolean present_runAs = true && (isSetRunAs()); + builder.append(present_runAs); + if (present_runAs) + builder.append(runAs); + + return builder.toHashCode(); + } + + public int compareTo(ShowCompactResponseElement other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + ShowCompactResponseElement typedOther = (ShowCompactResponseElement)other; + + lastComparison = Boolean.valueOf(isSetDbname()).compareTo(typedOther.isSetDbname()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetDbname()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.dbname, typedOther.dbname); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetTablename()).compareTo(typedOther.isSetTablename()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetTablename()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tablename, typedOther.tablename); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetPartitionname()).compareTo(typedOther.isSetPartitionname()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetPartitionname()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.partitionname, typedOther.partitionname); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetType()).compareTo(typedOther.isSetType()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetType()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.type, typedOther.type); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetState()).compareTo(typedOther.isSetState()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetState()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.state, typedOther.state); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetWorkerid()).compareTo(typedOther.isSetWorkerid()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetWorkerid()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.workerid, typedOther.workerid); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetStart()).compareTo(typedOther.isSetStart()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetStart()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.start, typedOther.start); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetRunAs()).compareTo(typedOther.isSetRunAs()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetRunAs()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.runAs, typedOther.runAs); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("ShowCompactResponseElement("); + boolean first = true; + + sb.append("dbname:"); + if (this.dbname == null) { + sb.append("null"); + } else { + sb.append(this.dbname); + } + first = false; + if (!first) sb.append(", "); + sb.append("tablename:"); + if (this.tablename == null) { + sb.append("null"); + } else { + sb.append(this.tablename); + } + first = false; + if (!first) sb.append(", "); + sb.append("partitionname:"); + if (this.partitionname == null) { + sb.append("null"); + } else { + sb.append(this.partitionname); + } + first = false; + if (!first) sb.append(", "); + sb.append("type:"); + if (this.type == null) { + sb.append("null"); + } else { + sb.append(this.type); + } + first = false; + if (!first) sb.append(", "); + sb.append("state:"); + if (this.state == null) { + sb.append("null"); + } else { + sb.append(this.state); + } + first = false; + if (!first) sb.append(", "); + sb.append("workerid:"); + if (this.workerid == null) { + sb.append("null"); + } else { + sb.append(this.workerid); + } + first = false; + if (!first) sb.append(", "); + sb.append("start:"); + sb.append(this.start); + first = false; + if (!first) sb.append(", "); + sb.append("runAs:"); + if (this.runAs == null) { + sb.append("null"); + } else { + sb.append(this.runAs); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetDbname()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'dbname' is unset! Struct:" + toString()); + } + + if (!isSetTablename()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'tablename' is unset! Struct:" + toString()); + } + + if (!isSetPartitionname()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'partitionname' is unset! Struct:" + toString()); + } + + if (!isSetType()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'type' is unset! Struct:" + toString()); + } + + if (!isSetState()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'state' is unset! Struct:" + toString()); + } + + if (!isSetWorkerid()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'workerid' is unset! Struct:" + toString()); + } + + if (!isSetStart()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'start' is unset! Struct:" + toString()); + } + + if (!isSetRunAs()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'runAs' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. + __isset_bitfield = 0; + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class ShowCompactResponseElementStandardSchemeFactory implements SchemeFactory { + public ShowCompactResponseElementStandardScheme getScheme() { + return new ShowCompactResponseElementStandardScheme(); + } + } + + private static class ShowCompactResponseElementStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, ShowCompactResponseElement struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // DBNAME + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.dbname = iprot.readString(); + struct.setDbnameIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // TABLENAME + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.tablename = iprot.readString(); + struct.setTablenameIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 3: // PARTITIONNAME + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.partitionname = iprot.readString(); + struct.setPartitionnameIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 4: // TYPE + if (schemeField.type == org.apache.thrift.protocol.TType.I32) { + struct.type = CompactionType.findByValue(iprot.readI32()); + struct.setTypeIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 5: // STATE + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.state = iprot.readString(); + struct.setStateIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 6: // WORKERID + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.workerid = iprot.readString(); + struct.setWorkeridIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 7: // START + if (schemeField.type == org.apache.thrift.protocol.TType.I64) { + struct.start = iprot.readI64(); + struct.setStartIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 8: // RUN_AS + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.runAs = iprot.readString(); + struct.setRunAsIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, ShowCompactResponseElement struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.dbname != null) { + oprot.writeFieldBegin(DBNAME_FIELD_DESC); + oprot.writeString(struct.dbname); + oprot.writeFieldEnd(); + } + if (struct.tablename != null) { + oprot.writeFieldBegin(TABLENAME_FIELD_DESC); + oprot.writeString(struct.tablename); + oprot.writeFieldEnd(); + } + if (struct.partitionname != null) { + oprot.writeFieldBegin(PARTITIONNAME_FIELD_DESC); + oprot.writeString(struct.partitionname); + oprot.writeFieldEnd(); + } + if (struct.type != null) { + oprot.writeFieldBegin(TYPE_FIELD_DESC); + oprot.writeI32(struct.type.getValue()); + oprot.writeFieldEnd(); + } + if (struct.state != null) { + oprot.writeFieldBegin(STATE_FIELD_DESC); + oprot.writeString(struct.state); + oprot.writeFieldEnd(); + } + if (struct.workerid != null) { + oprot.writeFieldBegin(WORKERID_FIELD_DESC); + oprot.writeString(struct.workerid); + oprot.writeFieldEnd(); + } + oprot.writeFieldBegin(START_FIELD_DESC); + oprot.writeI64(struct.start); + oprot.writeFieldEnd(); + if (struct.runAs != null) { + oprot.writeFieldBegin(RUN_AS_FIELD_DESC); + oprot.writeString(struct.runAs); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class ShowCompactResponseElementTupleSchemeFactory implements SchemeFactory { + public ShowCompactResponseElementTupleScheme getScheme() { + return new ShowCompactResponseElementTupleScheme(); + } + } + + private static class ShowCompactResponseElementTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, ShowCompactResponseElement struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + oprot.writeString(struct.dbname); + oprot.writeString(struct.tablename); + oprot.writeString(struct.partitionname); + oprot.writeI32(struct.type.getValue()); + oprot.writeString(struct.state); + oprot.writeString(struct.workerid); + oprot.writeI64(struct.start); + oprot.writeString(struct.runAs); + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, ShowCompactResponseElement struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.dbname = iprot.readString(); + struct.setDbnameIsSet(true); + struct.tablename = iprot.readString(); + struct.setTablenameIsSet(true); + struct.partitionname = iprot.readString(); + struct.setPartitionnameIsSet(true); + struct.type = CompactionType.findByValue(iprot.readI32()); + struct.setTypeIsSet(true); + struct.state = iprot.readString(); + struct.setStateIsSet(true); + struct.workerid = iprot.readString(); + struct.setWorkeridIsSet(true); + struct.start = iprot.readI64(); + struct.setStartIsSet(true); + struct.runAs = iprot.readString(); + struct.setRunAsIsSet(true); + } + } + +} + diff --git metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowLocksRequest.java metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowLocksRequest.java new file mode 100644 index 0000000..7a1fd6f --- /dev/null +++ metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowLocksRequest.java @@ -0,0 +1,279 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hadoop.hive.metastore.api; + +import org.apache.commons.lang.builder.HashCodeBuilder; +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class ShowLocksRequest implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("ShowLocksRequest"); + + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new ShowLocksRequestStandardSchemeFactory()); + schemes.put(TupleScheme.class, new ShowLocksRequestTupleSchemeFactory()); + } + + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { +; + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(ShowLocksRequest.class, metaDataMap); + } + + public ShowLocksRequest() { + } + + /** + * Performs a deep copy on other. + */ + public ShowLocksRequest(ShowLocksRequest other) { + } + + public ShowLocksRequest deepCopy() { + return new ShowLocksRequest(this); + } + + @Override + public void clear() { + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof ShowLocksRequest) + return this.equals((ShowLocksRequest)that); + return false; + } + + public boolean equals(ShowLocksRequest that) { + if (that == null) + return false; + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + return builder.toHashCode(); + } + + public int compareTo(ShowLocksRequest other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + ShowLocksRequest typedOther = (ShowLocksRequest)other; + + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("ShowLocksRequest("); + boolean first = true; + + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class ShowLocksRequestStandardSchemeFactory implements SchemeFactory { + public ShowLocksRequestStandardScheme getScheme() { + return new ShowLocksRequestStandardScheme(); + } + } + + private static class ShowLocksRequestStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, ShowLocksRequest struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, ShowLocksRequest struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class ShowLocksRequestTupleSchemeFactory implements SchemeFactory { + public ShowLocksRequestTupleScheme getScheme() { + return new ShowLocksRequestTupleScheme(); + } + } + + private static class ShowLocksRequestTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, ShowLocksRequest struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, ShowLocksRequest struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + } + } + +} + diff --git metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowLocksResponse.java metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowLocksResponse.java new file mode 100644 index 0000000..95948c4 --- /dev/null +++ metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowLocksResponse.java @@ -0,0 +1,445 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hadoop.hive.metastore.api; + +import org.apache.commons.lang.builder.HashCodeBuilder; +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class ShowLocksResponse implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("ShowLocksResponse"); + + private static final org.apache.thrift.protocol.TField LOCKS_FIELD_DESC = new org.apache.thrift.protocol.TField("locks", org.apache.thrift.protocol.TType.LIST, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new ShowLocksResponseStandardSchemeFactory()); + schemes.put(TupleScheme.class, new ShowLocksResponseTupleSchemeFactory()); + } + + private List locks; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + LOCKS((short)1, "locks"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // LOCKS + return LOCKS; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.LOCKS, new org.apache.thrift.meta_data.FieldMetaData("locks", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ShowLocksResponseElement.class)))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(ShowLocksResponse.class, metaDataMap); + } + + public ShowLocksResponse() { + } + + public ShowLocksResponse( + List locks) + { + this(); + this.locks = locks; + } + + /** + * Performs a deep copy on other. + */ + public ShowLocksResponse(ShowLocksResponse other) { + if (other.isSetLocks()) { + List __this__locks = new ArrayList(); + for (ShowLocksResponseElement other_element : other.locks) { + __this__locks.add(new ShowLocksResponseElement(other_element)); + } + this.locks = __this__locks; + } + } + + public ShowLocksResponse deepCopy() { + return new ShowLocksResponse(this); + } + + @Override + public void clear() { + this.locks = null; + } + + public int getLocksSize() { + return (this.locks == null) ? 0 : this.locks.size(); + } + + public java.util.Iterator getLocksIterator() { + return (this.locks == null) ? null : this.locks.iterator(); + } + + public void addToLocks(ShowLocksResponseElement elem) { + if (this.locks == null) { + this.locks = new ArrayList(); + } + this.locks.add(elem); + } + + public List getLocks() { + return this.locks; + } + + public void setLocks(List locks) { + this.locks = locks; + } + + public void unsetLocks() { + this.locks = null; + } + + /** Returns true if field locks is set (has been assigned a value) and false otherwise */ + public boolean isSetLocks() { + return this.locks != null; + } + + public void setLocksIsSet(boolean value) { + if (!value) { + this.locks = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case LOCKS: + if (value == null) { + unsetLocks(); + } else { + setLocks((List)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case LOCKS: + return getLocks(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case LOCKS: + return isSetLocks(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof ShowLocksResponse) + return this.equals((ShowLocksResponse)that); + return false; + } + + public boolean equals(ShowLocksResponse that) { + if (that == null) + return false; + + boolean this_present_locks = true && this.isSetLocks(); + boolean that_present_locks = true && that.isSetLocks(); + if (this_present_locks || that_present_locks) { + if (!(this_present_locks && that_present_locks)) + return false; + if (!this.locks.equals(that.locks)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_locks = true && (isSetLocks()); + builder.append(present_locks); + if (present_locks) + builder.append(locks); + + return builder.toHashCode(); + } + + public int compareTo(ShowLocksResponse other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + ShowLocksResponse typedOther = (ShowLocksResponse)other; + + lastComparison = Boolean.valueOf(isSetLocks()).compareTo(typedOther.isSetLocks()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetLocks()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.locks, typedOther.locks); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("ShowLocksResponse("); + boolean first = true; + + sb.append("locks:"); + if (this.locks == null) { + sb.append("null"); + } else { + sb.append(this.locks); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class ShowLocksResponseStandardSchemeFactory implements SchemeFactory { + public ShowLocksResponseStandardScheme getScheme() { + return new ShowLocksResponseStandardScheme(); + } + } + + private static class ShowLocksResponseStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, ShowLocksResponse struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // LOCKS + if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { + { + org.apache.thrift.protocol.TList _list394 = iprot.readListBegin(); + struct.locks = new ArrayList(_list394.size); + for (int _i395 = 0; _i395 < _list394.size; ++_i395) + { + ShowLocksResponseElement _elem396; // required + _elem396 = new ShowLocksResponseElement(); + _elem396.read(iprot); + struct.locks.add(_elem396); + } + iprot.readListEnd(); + } + struct.setLocksIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, ShowLocksResponse struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.locks != null) { + oprot.writeFieldBegin(LOCKS_FIELD_DESC); + { + oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.locks.size())); + for (ShowLocksResponseElement _iter397 : struct.locks) + { + _iter397.write(oprot); + } + oprot.writeListEnd(); + } + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class ShowLocksResponseTupleSchemeFactory implements SchemeFactory { + public ShowLocksResponseTupleScheme getScheme() { + return new ShowLocksResponseTupleScheme(); + } + } + + private static class ShowLocksResponseTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, ShowLocksResponse struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetLocks()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetLocks()) { + { + oprot.writeI32(struct.locks.size()); + for (ShowLocksResponseElement _iter398 : struct.locks) + { + _iter398.write(oprot); + } + } + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, ShowLocksResponse struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + { + org.apache.thrift.protocol.TList _list399 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); + struct.locks = new ArrayList(_list399.size); + for (int _i400 = 0; _i400 < _list399.size; ++_i400) + { + ShowLocksResponseElement _elem401; // required + _elem401 = new ShowLocksResponseElement(); + _elem401.read(iprot); + struct.locks.add(_elem401); + } + } + struct.setLocksIsSet(true); + } + } + } + +} + diff --git metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowLocksResponseElement.java metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowLocksResponseElement.java new file mode 100644 index 0000000..2f7c24f --- /dev/null +++ metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowLocksResponseElement.java @@ -0,0 +1,1421 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hadoop.hive.metastore.api; + +import org.apache.commons.lang.builder.HashCodeBuilder; +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class ShowLocksResponseElement implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("ShowLocksResponseElement"); + + private static final org.apache.thrift.protocol.TField LOCKID_FIELD_DESC = new org.apache.thrift.protocol.TField("lockid", org.apache.thrift.protocol.TType.I64, (short)1); + private static final org.apache.thrift.protocol.TField DBNAME_FIELD_DESC = new org.apache.thrift.protocol.TField("dbname", org.apache.thrift.protocol.TType.STRING, (short)2); + private static final org.apache.thrift.protocol.TField TABLENAME_FIELD_DESC = new org.apache.thrift.protocol.TField("tablename", org.apache.thrift.protocol.TType.STRING, (short)3); + private static final org.apache.thrift.protocol.TField PARTNAME_FIELD_DESC = new org.apache.thrift.protocol.TField("partname", org.apache.thrift.protocol.TType.STRING, (short)4); + private static final org.apache.thrift.protocol.TField STATE_FIELD_DESC = new org.apache.thrift.protocol.TField("state", org.apache.thrift.protocol.TType.I32, (short)5); + private static final org.apache.thrift.protocol.TField TYPE_FIELD_DESC = new org.apache.thrift.protocol.TField("type", org.apache.thrift.protocol.TType.I32, (short)6); + private static final org.apache.thrift.protocol.TField TXNID_FIELD_DESC = new org.apache.thrift.protocol.TField("txnid", org.apache.thrift.protocol.TType.I64, (short)7); + private static final org.apache.thrift.protocol.TField LASTHEARTBEAT_FIELD_DESC = new org.apache.thrift.protocol.TField("lastheartbeat", org.apache.thrift.protocol.TType.I64, (short)8); + private static final org.apache.thrift.protocol.TField ACQUIREDAT_FIELD_DESC = new org.apache.thrift.protocol.TField("acquiredat", org.apache.thrift.protocol.TType.I64, (short)9); + private static final org.apache.thrift.protocol.TField USER_FIELD_DESC = new org.apache.thrift.protocol.TField("user", org.apache.thrift.protocol.TType.STRING, (short)10); + private static final org.apache.thrift.protocol.TField HOSTNAME_FIELD_DESC = new org.apache.thrift.protocol.TField("hostname", org.apache.thrift.protocol.TType.STRING, (short)11); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new ShowLocksResponseElementStandardSchemeFactory()); + schemes.put(TupleScheme.class, new ShowLocksResponseElementTupleSchemeFactory()); + } + + private long lockid; // required + private String dbname; // required + private String tablename; // optional + private String partname; // optional + private LockState state; // required + private LockType type; // required + private long txnid; // optional + private long lastheartbeat; // required + private long acquiredat; // optional + private String user; // required + private String hostname; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + LOCKID((short)1, "lockid"), + DBNAME((short)2, "dbname"), + TABLENAME((short)3, "tablename"), + PARTNAME((short)4, "partname"), + /** + * + * @see LockState + */ + STATE((short)5, "state"), + /** + * + * @see LockType + */ + TYPE((short)6, "type"), + TXNID((short)7, "txnid"), + LASTHEARTBEAT((short)8, "lastheartbeat"), + ACQUIREDAT((short)9, "acquiredat"), + USER((short)10, "user"), + HOSTNAME((short)11, "hostname"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // LOCKID + return LOCKID; + case 2: // DBNAME + return DBNAME; + case 3: // TABLENAME + return TABLENAME; + case 4: // PARTNAME + return PARTNAME; + case 5: // STATE + return STATE; + case 6: // TYPE + return TYPE; + case 7: // TXNID + return TXNID; + case 8: // LASTHEARTBEAT + return LASTHEARTBEAT; + case 9: // ACQUIREDAT + return ACQUIREDAT; + case 10: // USER + return USER; + case 11: // HOSTNAME + return HOSTNAME; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + private static final int __LOCKID_ISSET_ID = 0; + private static final int __TXNID_ISSET_ID = 1; + private static final int __LASTHEARTBEAT_ISSET_ID = 2; + private static final int __ACQUIREDAT_ISSET_ID = 3; + private byte __isset_bitfield = 0; + private _Fields optionals[] = {_Fields.TABLENAME,_Fields.PARTNAME,_Fields.TXNID,_Fields.ACQUIREDAT}; + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.LOCKID, new org.apache.thrift.meta_data.FieldMetaData("lockid", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64))); + tmpMap.put(_Fields.DBNAME, new org.apache.thrift.meta_data.FieldMetaData("dbname", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + tmpMap.put(_Fields.TABLENAME, new org.apache.thrift.meta_data.FieldMetaData("tablename", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + tmpMap.put(_Fields.PARTNAME, new org.apache.thrift.meta_data.FieldMetaData("partname", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + tmpMap.put(_Fields.STATE, new org.apache.thrift.meta_data.FieldMetaData("state", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, LockState.class))); + tmpMap.put(_Fields.TYPE, new org.apache.thrift.meta_data.FieldMetaData("type", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, LockType.class))); + tmpMap.put(_Fields.TXNID, new org.apache.thrift.meta_data.FieldMetaData("txnid", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64))); + tmpMap.put(_Fields.LASTHEARTBEAT, new org.apache.thrift.meta_data.FieldMetaData("lastheartbeat", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64))); + tmpMap.put(_Fields.ACQUIREDAT, new org.apache.thrift.meta_data.FieldMetaData("acquiredat", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64))); + tmpMap.put(_Fields.USER, new org.apache.thrift.meta_data.FieldMetaData("user", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + tmpMap.put(_Fields.HOSTNAME, new org.apache.thrift.meta_data.FieldMetaData("hostname", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(ShowLocksResponseElement.class, metaDataMap); + } + + public ShowLocksResponseElement() { + } + + public ShowLocksResponseElement( + long lockid, + String dbname, + LockState state, + LockType type, + long lastheartbeat, + String user, + String hostname) + { + this(); + this.lockid = lockid; + setLockidIsSet(true); + this.dbname = dbname; + this.state = state; + this.type = type; + this.lastheartbeat = lastheartbeat; + setLastheartbeatIsSet(true); + this.user = user; + this.hostname = hostname; + } + + /** + * Performs a deep copy on other. + */ + public ShowLocksResponseElement(ShowLocksResponseElement other) { + __isset_bitfield = other.__isset_bitfield; + this.lockid = other.lockid; + if (other.isSetDbname()) { + this.dbname = other.dbname; + } + if (other.isSetTablename()) { + this.tablename = other.tablename; + } + if (other.isSetPartname()) { + this.partname = other.partname; + } + if (other.isSetState()) { + this.state = other.state; + } + if (other.isSetType()) { + this.type = other.type; + } + this.txnid = other.txnid; + this.lastheartbeat = other.lastheartbeat; + this.acquiredat = other.acquiredat; + if (other.isSetUser()) { + this.user = other.user; + } + if (other.isSetHostname()) { + this.hostname = other.hostname; + } + } + + public ShowLocksResponseElement deepCopy() { + return new ShowLocksResponseElement(this); + } + + @Override + public void clear() { + setLockidIsSet(false); + this.lockid = 0; + this.dbname = null; + this.tablename = null; + this.partname = null; + this.state = null; + this.type = null; + setTxnidIsSet(false); + this.txnid = 0; + setLastheartbeatIsSet(false); + this.lastheartbeat = 0; + setAcquiredatIsSet(false); + this.acquiredat = 0; + this.user = null; + this.hostname = null; + } + + public long getLockid() { + return this.lockid; + } + + public void setLockid(long lockid) { + this.lockid = lockid; + setLockidIsSet(true); + } + + public void unsetLockid() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __LOCKID_ISSET_ID); + } + + /** Returns true if field lockid is set (has been assigned a value) and false otherwise */ + public boolean isSetLockid() { + return EncodingUtils.testBit(__isset_bitfield, __LOCKID_ISSET_ID); + } + + public void setLockidIsSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __LOCKID_ISSET_ID, value); + } + + public String getDbname() { + return this.dbname; + } + + public void setDbname(String dbname) { + this.dbname = dbname; + } + + public void unsetDbname() { + this.dbname = null; + } + + /** Returns true if field dbname is set (has been assigned a value) and false otherwise */ + public boolean isSetDbname() { + return this.dbname != null; + } + + public void setDbnameIsSet(boolean value) { + if (!value) { + this.dbname = null; + } + } + + public String getTablename() { + return this.tablename; + } + + public void setTablename(String tablename) { + this.tablename = tablename; + } + + public void unsetTablename() { + this.tablename = null; + } + + /** Returns true if field tablename is set (has been assigned a value) and false otherwise */ + public boolean isSetTablename() { + return this.tablename != null; + } + + public void setTablenameIsSet(boolean value) { + if (!value) { + this.tablename = null; + } + } + + public String getPartname() { + return this.partname; + } + + public void setPartname(String partname) { + this.partname = partname; + } + + public void unsetPartname() { + this.partname = null; + } + + /** Returns true if field partname is set (has been assigned a value) and false otherwise */ + public boolean isSetPartname() { + return this.partname != null; + } + + public void setPartnameIsSet(boolean value) { + if (!value) { + this.partname = null; + } + } + + /** + * + * @see LockState + */ + public LockState getState() { + return this.state; + } + + /** + * + * @see LockState + */ + public void setState(LockState state) { + this.state = state; + } + + public void unsetState() { + this.state = null; + } + + /** Returns true if field state is set (has been assigned a value) and false otherwise */ + public boolean isSetState() { + return this.state != null; + } + + public void setStateIsSet(boolean value) { + if (!value) { + this.state = null; + } + } + + /** + * + * @see LockType + */ + public LockType getType() { + return this.type; + } + + /** + * + * @see LockType + */ + public void setType(LockType type) { + this.type = type; + } + + public void unsetType() { + this.type = null; + } + + /** Returns true if field type is set (has been assigned a value) and false otherwise */ + public boolean isSetType() { + return this.type != null; + } + + public void setTypeIsSet(boolean value) { + if (!value) { + this.type = null; + } + } + + public long getTxnid() { + return this.txnid; + } + + public void setTxnid(long txnid) { + this.txnid = txnid; + setTxnidIsSet(true); + } + + public void unsetTxnid() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __TXNID_ISSET_ID); + } + + /** Returns true if field txnid is set (has been assigned a value) and false otherwise */ + public boolean isSetTxnid() { + return EncodingUtils.testBit(__isset_bitfield, __TXNID_ISSET_ID); + } + + public void setTxnidIsSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __TXNID_ISSET_ID, value); + } + + public long getLastheartbeat() { + return this.lastheartbeat; + } + + public void setLastheartbeat(long lastheartbeat) { + this.lastheartbeat = lastheartbeat; + setLastheartbeatIsSet(true); + } + + public void unsetLastheartbeat() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __LASTHEARTBEAT_ISSET_ID); + } + + /** Returns true if field lastheartbeat is set (has been assigned a value) and false otherwise */ + public boolean isSetLastheartbeat() { + return EncodingUtils.testBit(__isset_bitfield, __LASTHEARTBEAT_ISSET_ID); + } + + public void setLastheartbeatIsSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __LASTHEARTBEAT_ISSET_ID, value); + } + + public long getAcquiredat() { + return this.acquiredat; + } + + public void setAcquiredat(long acquiredat) { + this.acquiredat = acquiredat; + setAcquiredatIsSet(true); + } + + public void unsetAcquiredat() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __ACQUIREDAT_ISSET_ID); + } + + /** Returns true if field acquiredat is set (has been assigned a value) and false otherwise */ + public boolean isSetAcquiredat() { + return EncodingUtils.testBit(__isset_bitfield, __ACQUIREDAT_ISSET_ID); + } + + public void setAcquiredatIsSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __ACQUIREDAT_ISSET_ID, value); + } + + public String getUser() { + return this.user; + } + + public void setUser(String user) { + this.user = user; + } + + public void unsetUser() { + this.user = null; + } + + /** Returns true if field user is set (has been assigned a value) and false otherwise */ + public boolean isSetUser() { + return this.user != null; + } + + public void setUserIsSet(boolean value) { + if (!value) { + this.user = null; + } + } + + public String getHostname() { + return this.hostname; + } + + public void setHostname(String hostname) { + this.hostname = hostname; + } + + public void unsetHostname() { + this.hostname = null; + } + + /** Returns true if field hostname is set (has been assigned a value) and false otherwise */ + public boolean isSetHostname() { + return this.hostname != null; + } + + public void setHostnameIsSet(boolean value) { + if (!value) { + this.hostname = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case LOCKID: + if (value == null) { + unsetLockid(); + } else { + setLockid((Long)value); + } + break; + + case DBNAME: + if (value == null) { + unsetDbname(); + } else { + setDbname((String)value); + } + break; + + case TABLENAME: + if (value == null) { + unsetTablename(); + } else { + setTablename((String)value); + } + break; + + case PARTNAME: + if (value == null) { + unsetPartname(); + } else { + setPartname((String)value); + } + break; + + case STATE: + if (value == null) { + unsetState(); + } else { + setState((LockState)value); + } + break; + + case TYPE: + if (value == null) { + unsetType(); + } else { + setType((LockType)value); + } + break; + + case TXNID: + if (value == null) { + unsetTxnid(); + } else { + setTxnid((Long)value); + } + break; + + case LASTHEARTBEAT: + if (value == null) { + unsetLastheartbeat(); + } else { + setLastheartbeat((Long)value); + } + break; + + case ACQUIREDAT: + if (value == null) { + unsetAcquiredat(); + } else { + setAcquiredat((Long)value); + } + break; + + case USER: + if (value == null) { + unsetUser(); + } else { + setUser((String)value); + } + break; + + case HOSTNAME: + if (value == null) { + unsetHostname(); + } else { + setHostname((String)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case LOCKID: + return Long.valueOf(getLockid()); + + case DBNAME: + return getDbname(); + + case TABLENAME: + return getTablename(); + + case PARTNAME: + return getPartname(); + + case STATE: + return getState(); + + case TYPE: + return getType(); + + case TXNID: + return Long.valueOf(getTxnid()); + + case LASTHEARTBEAT: + return Long.valueOf(getLastheartbeat()); + + case ACQUIREDAT: + return Long.valueOf(getAcquiredat()); + + case USER: + return getUser(); + + case HOSTNAME: + return getHostname(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case LOCKID: + return isSetLockid(); + case DBNAME: + return isSetDbname(); + case TABLENAME: + return isSetTablename(); + case PARTNAME: + return isSetPartname(); + case STATE: + return isSetState(); + case TYPE: + return isSetType(); + case TXNID: + return isSetTxnid(); + case LASTHEARTBEAT: + return isSetLastheartbeat(); + case ACQUIREDAT: + return isSetAcquiredat(); + case USER: + return isSetUser(); + case HOSTNAME: + return isSetHostname(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof ShowLocksResponseElement) + return this.equals((ShowLocksResponseElement)that); + return false; + } + + public boolean equals(ShowLocksResponseElement that) { + if (that == null) + return false; + + boolean this_present_lockid = true; + boolean that_present_lockid = true; + if (this_present_lockid || that_present_lockid) { + if (!(this_present_lockid && that_present_lockid)) + return false; + if (this.lockid != that.lockid) + return false; + } + + boolean this_present_dbname = true && this.isSetDbname(); + boolean that_present_dbname = true && that.isSetDbname(); + if (this_present_dbname || that_present_dbname) { + if (!(this_present_dbname && that_present_dbname)) + return false; + if (!this.dbname.equals(that.dbname)) + return false; + } + + boolean this_present_tablename = true && this.isSetTablename(); + boolean that_present_tablename = true && that.isSetTablename(); + if (this_present_tablename || that_present_tablename) { + if (!(this_present_tablename && that_present_tablename)) + return false; + if (!this.tablename.equals(that.tablename)) + return false; + } + + boolean this_present_partname = true && this.isSetPartname(); + boolean that_present_partname = true && that.isSetPartname(); + if (this_present_partname || that_present_partname) { + if (!(this_present_partname && that_present_partname)) + return false; + if (!this.partname.equals(that.partname)) + return false; + } + + boolean this_present_state = true && this.isSetState(); + boolean that_present_state = true && that.isSetState(); + if (this_present_state || that_present_state) { + if (!(this_present_state && that_present_state)) + return false; + if (!this.state.equals(that.state)) + return false; + } + + boolean this_present_type = true && this.isSetType(); + boolean that_present_type = true && that.isSetType(); + if (this_present_type || that_present_type) { + if (!(this_present_type && that_present_type)) + return false; + if (!this.type.equals(that.type)) + return false; + } + + boolean this_present_txnid = true && this.isSetTxnid(); + boolean that_present_txnid = true && that.isSetTxnid(); + if (this_present_txnid || that_present_txnid) { + if (!(this_present_txnid && that_present_txnid)) + return false; + if (this.txnid != that.txnid) + return false; + } + + boolean this_present_lastheartbeat = true; + boolean that_present_lastheartbeat = true; + if (this_present_lastheartbeat || that_present_lastheartbeat) { + if (!(this_present_lastheartbeat && that_present_lastheartbeat)) + return false; + if (this.lastheartbeat != that.lastheartbeat) + return false; + } + + boolean this_present_acquiredat = true && this.isSetAcquiredat(); + boolean that_present_acquiredat = true && that.isSetAcquiredat(); + if (this_present_acquiredat || that_present_acquiredat) { + if (!(this_present_acquiredat && that_present_acquiredat)) + return false; + if (this.acquiredat != that.acquiredat) + return false; + } + + boolean this_present_user = true && this.isSetUser(); + boolean that_present_user = true && that.isSetUser(); + if (this_present_user || that_present_user) { + if (!(this_present_user && that_present_user)) + return false; + if (!this.user.equals(that.user)) + return false; + } + + boolean this_present_hostname = true && this.isSetHostname(); + boolean that_present_hostname = true && that.isSetHostname(); + if (this_present_hostname || that_present_hostname) { + if (!(this_present_hostname && that_present_hostname)) + return false; + if (!this.hostname.equals(that.hostname)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_lockid = true; + builder.append(present_lockid); + if (present_lockid) + builder.append(lockid); + + boolean present_dbname = true && (isSetDbname()); + builder.append(present_dbname); + if (present_dbname) + builder.append(dbname); + + boolean present_tablename = true && (isSetTablename()); + builder.append(present_tablename); + if (present_tablename) + builder.append(tablename); + + boolean present_partname = true && (isSetPartname()); + builder.append(present_partname); + if (present_partname) + builder.append(partname); + + boolean present_state = true && (isSetState()); + builder.append(present_state); + if (present_state) + builder.append(state.getValue()); + + boolean present_type = true && (isSetType()); + builder.append(present_type); + if (present_type) + builder.append(type.getValue()); + + boolean present_txnid = true && (isSetTxnid()); + builder.append(present_txnid); + if (present_txnid) + builder.append(txnid); + + boolean present_lastheartbeat = true; + builder.append(present_lastheartbeat); + if (present_lastheartbeat) + builder.append(lastheartbeat); + + boolean present_acquiredat = true && (isSetAcquiredat()); + builder.append(present_acquiredat); + if (present_acquiredat) + builder.append(acquiredat); + + boolean present_user = true && (isSetUser()); + builder.append(present_user); + if (present_user) + builder.append(user); + + boolean present_hostname = true && (isSetHostname()); + builder.append(present_hostname); + if (present_hostname) + builder.append(hostname); + + return builder.toHashCode(); + } + + public int compareTo(ShowLocksResponseElement other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + ShowLocksResponseElement typedOther = (ShowLocksResponseElement)other; + + lastComparison = Boolean.valueOf(isSetLockid()).compareTo(typedOther.isSetLockid()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetLockid()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.lockid, typedOther.lockid); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetDbname()).compareTo(typedOther.isSetDbname()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetDbname()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.dbname, typedOther.dbname); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetTablename()).compareTo(typedOther.isSetTablename()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetTablename()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tablename, typedOther.tablename); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetPartname()).compareTo(typedOther.isSetPartname()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetPartname()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.partname, typedOther.partname); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetState()).compareTo(typedOther.isSetState()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetState()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.state, typedOther.state); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetType()).compareTo(typedOther.isSetType()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetType()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.type, typedOther.type); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetTxnid()).compareTo(typedOther.isSetTxnid()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetTxnid()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.txnid, typedOther.txnid); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetLastheartbeat()).compareTo(typedOther.isSetLastheartbeat()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetLastheartbeat()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.lastheartbeat, typedOther.lastheartbeat); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetAcquiredat()).compareTo(typedOther.isSetAcquiredat()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetAcquiredat()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.acquiredat, typedOther.acquiredat); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetUser()).compareTo(typedOther.isSetUser()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetUser()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.user, typedOther.user); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetHostname()).compareTo(typedOther.isSetHostname()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetHostname()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.hostname, typedOther.hostname); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("ShowLocksResponseElement("); + boolean first = true; + + sb.append("lockid:"); + sb.append(this.lockid); + first = false; + if (!first) sb.append(", "); + sb.append("dbname:"); + if (this.dbname == null) { + sb.append("null"); + } else { + sb.append(this.dbname); + } + first = false; + if (isSetTablename()) { + if (!first) sb.append(", "); + sb.append("tablename:"); + if (this.tablename == null) { + sb.append("null"); + } else { + sb.append(this.tablename); + } + first = false; + } + if (isSetPartname()) { + if (!first) sb.append(", "); + sb.append("partname:"); + if (this.partname == null) { + sb.append("null"); + } else { + sb.append(this.partname); + } + first = false; + } + if (!first) sb.append(", "); + sb.append("state:"); + if (this.state == null) { + sb.append("null"); + } else { + sb.append(this.state); + } + first = false; + if (!first) sb.append(", "); + sb.append("type:"); + if (this.type == null) { + sb.append("null"); + } else { + sb.append(this.type); + } + first = false; + if (isSetTxnid()) { + if (!first) sb.append(", "); + sb.append("txnid:"); + sb.append(this.txnid); + first = false; + } + if (!first) sb.append(", "); + sb.append("lastheartbeat:"); + sb.append(this.lastheartbeat); + first = false; + if (isSetAcquiredat()) { + if (!first) sb.append(", "); + sb.append("acquiredat:"); + sb.append(this.acquiredat); + first = false; + } + if (!first) sb.append(", "); + sb.append("user:"); + if (this.user == null) { + sb.append("null"); + } else { + sb.append(this.user); + } + first = false; + if (!first) sb.append(", "); + sb.append("hostname:"); + if (this.hostname == null) { + sb.append("null"); + } else { + sb.append(this.hostname); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetLockid()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'lockid' is unset! Struct:" + toString()); + } + + if (!isSetDbname()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'dbname' is unset! Struct:" + toString()); + } + + if (!isSetState()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'state' is unset! Struct:" + toString()); + } + + if (!isSetType()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'type' is unset! Struct:" + toString()); + } + + if (!isSetLastheartbeat()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'lastheartbeat' is unset! Struct:" + toString()); + } + + if (!isSetUser()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'user' is unset! Struct:" + toString()); + } + + if (!isSetHostname()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'hostname' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. + __isset_bitfield = 0; + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class ShowLocksResponseElementStandardSchemeFactory implements SchemeFactory { + public ShowLocksResponseElementStandardScheme getScheme() { + return new ShowLocksResponseElementStandardScheme(); + } + } + + private static class ShowLocksResponseElementStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, ShowLocksResponseElement struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // LOCKID + if (schemeField.type == org.apache.thrift.protocol.TType.I64) { + struct.lockid = iprot.readI64(); + struct.setLockidIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // DBNAME + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.dbname = iprot.readString(); + struct.setDbnameIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 3: // TABLENAME + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.tablename = iprot.readString(); + struct.setTablenameIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 4: // PARTNAME + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.partname = iprot.readString(); + struct.setPartnameIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 5: // STATE + if (schemeField.type == org.apache.thrift.protocol.TType.I32) { + struct.state = LockState.findByValue(iprot.readI32()); + struct.setStateIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 6: // TYPE + if (schemeField.type == org.apache.thrift.protocol.TType.I32) { + struct.type = LockType.findByValue(iprot.readI32()); + struct.setTypeIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 7: // TXNID + if (schemeField.type == org.apache.thrift.protocol.TType.I64) { + struct.txnid = iprot.readI64(); + struct.setTxnidIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 8: // LASTHEARTBEAT + if (schemeField.type == org.apache.thrift.protocol.TType.I64) { + struct.lastheartbeat = iprot.readI64(); + struct.setLastheartbeatIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 9: // ACQUIREDAT + if (schemeField.type == org.apache.thrift.protocol.TType.I64) { + struct.acquiredat = iprot.readI64(); + struct.setAcquiredatIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 10: // USER + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.user = iprot.readString(); + struct.setUserIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 11: // HOSTNAME + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.hostname = iprot.readString(); + struct.setHostnameIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, ShowLocksResponseElement struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + oprot.writeFieldBegin(LOCKID_FIELD_DESC); + oprot.writeI64(struct.lockid); + oprot.writeFieldEnd(); + if (struct.dbname != null) { + oprot.writeFieldBegin(DBNAME_FIELD_DESC); + oprot.writeString(struct.dbname); + oprot.writeFieldEnd(); + } + if (struct.tablename != null) { + if (struct.isSetTablename()) { + oprot.writeFieldBegin(TABLENAME_FIELD_DESC); + oprot.writeString(struct.tablename); + oprot.writeFieldEnd(); + } + } + if (struct.partname != null) { + if (struct.isSetPartname()) { + oprot.writeFieldBegin(PARTNAME_FIELD_DESC); + oprot.writeString(struct.partname); + oprot.writeFieldEnd(); + } + } + if (struct.state != null) { + oprot.writeFieldBegin(STATE_FIELD_DESC); + oprot.writeI32(struct.state.getValue()); + oprot.writeFieldEnd(); + } + if (struct.type != null) { + oprot.writeFieldBegin(TYPE_FIELD_DESC); + oprot.writeI32(struct.type.getValue()); + oprot.writeFieldEnd(); + } + if (struct.isSetTxnid()) { + oprot.writeFieldBegin(TXNID_FIELD_DESC); + oprot.writeI64(struct.txnid); + oprot.writeFieldEnd(); + } + oprot.writeFieldBegin(LASTHEARTBEAT_FIELD_DESC); + oprot.writeI64(struct.lastheartbeat); + oprot.writeFieldEnd(); + if (struct.isSetAcquiredat()) { + oprot.writeFieldBegin(ACQUIREDAT_FIELD_DESC); + oprot.writeI64(struct.acquiredat); + oprot.writeFieldEnd(); + } + if (struct.user != null) { + oprot.writeFieldBegin(USER_FIELD_DESC); + oprot.writeString(struct.user); + oprot.writeFieldEnd(); + } + if (struct.hostname != null) { + oprot.writeFieldBegin(HOSTNAME_FIELD_DESC); + oprot.writeString(struct.hostname); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class ShowLocksResponseElementTupleSchemeFactory implements SchemeFactory { + public ShowLocksResponseElementTupleScheme getScheme() { + return new ShowLocksResponseElementTupleScheme(); + } + } + + private static class ShowLocksResponseElementTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, ShowLocksResponseElement struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + oprot.writeI64(struct.lockid); + oprot.writeString(struct.dbname); + oprot.writeI32(struct.state.getValue()); + oprot.writeI32(struct.type.getValue()); + oprot.writeI64(struct.lastheartbeat); + oprot.writeString(struct.user); + oprot.writeString(struct.hostname); + BitSet optionals = new BitSet(); + if (struct.isSetTablename()) { + optionals.set(0); + } + if (struct.isSetPartname()) { + optionals.set(1); + } + if (struct.isSetTxnid()) { + optionals.set(2); + } + if (struct.isSetAcquiredat()) { + optionals.set(3); + } + oprot.writeBitSet(optionals, 4); + if (struct.isSetTablename()) { + oprot.writeString(struct.tablename); + } + if (struct.isSetPartname()) { + oprot.writeString(struct.partname); + } + if (struct.isSetTxnid()) { + oprot.writeI64(struct.txnid); + } + if (struct.isSetAcquiredat()) { + oprot.writeI64(struct.acquiredat); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, ShowLocksResponseElement struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.lockid = iprot.readI64(); + struct.setLockidIsSet(true); + struct.dbname = iprot.readString(); + struct.setDbnameIsSet(true); + struct.state = LockState.findByValue(iprot.readI32()); + struct.setStateIsSet(true); + struct.type = LockType.findByValue(iprot.readI32()); + struct.setTypeIsSet(true); + struct.lastheartbeat = iprot.readI64(); + struct.setLastheartbeatIsSet(true); + struct.user = iprot.readString(); + struct.setUserIsSet(true); + struct.hostname = iprot.readString(); + struct.setHostnameIsSet(true); + BitSet incoming = iprot.readBitSet(4); + if (incoming.get(0)) { + struct.tablename = iprot.readString(); + struct.setTablenameIsSet(true); + } + if (incoming.get(1)) { + struct.partname = iprot.readString(); + struct.setPartnameIsSet(true); + } + if (incoming.get(2)) { + struct.txnid = iprot.readI64(); + struct.setTxnidIsSet(true); + } + if (incoming.get(3)) { + struct.acquiredat = iprot.readI64(); + struct.setAcquiredatIsSet(true); + } + } + } + +} + diff --git metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SkewedInfo.java metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SkewedInfo.java index 98d6f81..2ad42a2 100644 --- metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SkewedInfo.java +++ metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SkewedInfo.java @@ -566,7 +566,7 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, SkewedInfo struct) struct.skewedColNames = new ArrayList(_list98.size); for (int _i99 = 0; _i99 < _list98.size; ++_i99) { - String _elem100; // optional + String _elem100; // required _elem100 = iprot.readString(); struct.skewedColNames.add(_elem100); } @@ -584,13 +584,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, SkewedInfo struct) struct.skewedColValues = new ArrayList>(_list101.size); for (int _i102 = 0; _i102 < _list101.size; ++_i102) { - List _elem103; // optional + List _elem103; // required { org.apache.thrift.protocol.TList _list104 = iprot.readListBegin(); _elem103 = new ArrayList(_list104.size); for (int _i105 = 0; _i105 < _list104.size; ++_i105) { - String _elem106; // optional + String _elem106; // required _elem106 = iprot.readString(); _elem103.add(_elem106); } @@ -619,7 +619,7 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, SkewedInfo struct) _key109 = new ArrayList(_list111.size); for (int _i112 = 0; _i112 < _list111.size; ++_i112) { - String _elem113; // optional + String _elem113; // required _elem113 = iprot.readString(); _key109.add(_elem113); } @@ -779,7 +779,7 @@ public void read(org.apache.thrift.protocol.TProtocol prot, SkewedInfo struct) t struct.skewedColNames = new ArrayList(_list124.size); for (int _i125 = 0; _i125 < _list124.size; ++_i125) { - String _elem126; // optional + String _elem126; // required _elem126 = iprot.readString(); struct.skewedColNames.add(_elem126); } @@ -792,13 +792,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, SkewedInfo struct) t struct.skewedColValues = new ArrayList>(_list127.size); for (int _i128 = 0; _i128 < _list127.size; ++_i128) { - List _elem129; // optional + List _elem129; // required { org.apache.thrift.protocol.TList _list130 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); _elem129 = new ArrayList(_list130.size); for (int _i131 = 0; _i131 < _list130.size; ++_i131) { - String _elem132; // optional + String _elem132; // required _elem132 = iprot.readString(); _elem129.add(_elem132); } @@ -821,7 +821,7 @@ public void read(org.apache.thrift.protocol.TProtocol prot, SkewedInfo struct) t _key135 = new ArrayList(_list137.size); for (int _i138 = 0; _i138 < _list137.size; ++_i138) { - String _elem139; // optional + String _elem139; // required _elem139 = iprot.readString(); _key135.add(_elem139); } diff --git metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/StorageDescriptor.java metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/StorageDescriptor.java index a2be1c5..0a2f2c2 100644 --- metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/StorageDescriptor.java +++ metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/StorageDescriptor.java @@ -1304,7 +1304,7 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, StorageDescriptor s struct.cols = new ArrayList(_list140.size); for (int _i141 = 0; _i141 < _list140.size; ++_i141) { - FieldSchema _elem142; // optional + FieldSchema _elem142; // required _elem142 = new FieldSchema(); _elem142.read(iprot); struct.cols.add(_elem142); @@ -1372,7 +1372,7 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, StorageDescriptor s struct.bucketCols = new ArrayList(_list143.size); for (int _i144 = 0; _i144 < _list143.size; ++_i144) { - String _elem145; // optional + String _elem145; // required _elem145 = iprot.readString(); struct.bucketCols.add(_elem145); } @@ -1390,7 +1390,7 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, StorageDescriptor s struct.sortCols = new ArrayList(_list146.size); for (int _i147 = 0; _i147 < _list146.size; ++_i147) { - Order _elem148; // optional + Order _elem148; // required _elem148 = new Order(); _elem148.read(iprot); struct.sortCols.add(_elem148); @@ -1667,7 +1667,7 @@ public void read(org.apache.thrift.protocol.TProtocol prot, StorageDescriptor st struct.cols = new ArrayList(_list161.size); for (int _i162 = 0; _i162 < _list161.size; ++_i162) { - FieldSchema _elem163; // optional + FieldSchema _elem163; // required _elem163 = new FieldSchema(); _elem163.read(iprot); struct.cols.add(_elem163); @@ -1706,7 +1706,7 @@ public void read(org.apache.thrift.protocol.TProtocol prot, StorageDescriptor st struct.bucketCols = new ArrayList(_list164.size); for (int _i165 = 0; _i165 < _list164.size; ++_i165) { - String _elem166; // optional + String _elem166; // required _elem166 = iprot.readString(); struct.bucketCols.add(_elem166); } @@ -1719,7 +1719,7 @@ public void read(org.apache.thrift.protocol.TProtocol prot, StorageDescriptor st struct.sortCols = new ArrayList(_list167.size); for (int _i168 = 0; _i168 < _list167.size; ++_i168) { - Order _elem169; // optional + Order _elem169; // required _elem169 = new Order(); _elem169.read(iprot); struct.sortCols.add(_elem169); diff --git metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Table.java metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Table.java index aa71e86..377cafe 100644 --- metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Table.java +++ metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Table.java @@ -1403,7 +1403,7 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, Table struct) throw struct.partitionKeys = new ArrayList(_list174.size); for (int _i175 = 0; _i175 < _list174.size; ++_i175) { - FieldSchema _elem176; // optional + FieldSchema _elem176; // required _elem176 = new FieldSchema(); _elem176.read(iprot); struct.partitionKeys.add(_elem176); @@ -1708,7 +1708,7 @@ public void read(org.apache.thrift.protocol.TProtocol prot, Table struct) throws struct.partitionKeys = new ArrayList(_list185.size); for (int _i186 = 0; _i186 < _list185.size; ++_i186) { - FieldSchema _elem187; // optional + FieldSchema _elem187; // required _elem187 = new FieldSchema(); _elem187.read(iprot); struct.partitionKeys.add(_elem187); diff --git metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TableStatsRequest.java metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TableStatsRequest.java index 464257b..9a03953 100644 --- metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TableStatsRequest.java +++ metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TableStatsRequest.java @@ -540,7 +540,7 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, TableStatsRequest s struct.colNames = new ArrayList(_list290.size); for (int _i291 = 0; _i291 < _list290.size; ++_i291) { - String _elem292; // optional + String _elem292; // required _elem292 = iprot.readString(); struct.colNames.add(_elem292); } @@ -626,7 +626,7 @@ public void read(org.apache.thrift.protocol.TProtocol prot, TableStatsRequest st struct.colNames = new ArrayList(_list295.size); for (int _i296 = 0; _i296 < _list295.size; ++_i296) { - String _elem297; // optional + String _elem297; // required _elem297 = iprot.readString(); struct.colNames.add(_elem297); } diff --git metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TableStatsResult.java metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TableStatsResult.java index 362b369..c2ed067 100644 --- metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TableStatsResult.java +++ metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TableStatsResult.java @@ -354,7 +354,7 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, TableStatsResult st struct.tableStats = new ArrayList(_list264.size); for (int _i265 = 0; _i265 < _list264.size; ++_i265) { - ColumnStatisticsObj _elem266; // optional + ColumnStatisticsObj _elem266; // required _elem266 = new ColumnStatisticsObj(); _elem266.read(iprot); struct.tableStats.add(_elem266); @@ -425,7 +425,7 @@ public void read(org.apache.thrift.protocol.TProtocol prot, TableStatsResult str struct.tableStats = new ArrayList(_list269.size); for (int _i270 = 0; _i270 < _list269.size; ++_i270) { - ColumnStatisticsObj _elem271; // optional + ColumnStatisticsObj _elem271; // required _elem271 = new ColumnStatisticsObj(); _elem271.read(iprot); struct.tableStats.add(_elem271); diff --git metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java index 00be43c..bb0205c 100644 --- metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java +++ metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java @@ -222,6 +222,30 @@ public void cancel_delegation_token(String token_str_form) throws MetaException, org.apache.thrift.TException; + public GetOpenTxnsResponse get_open_txns() throws org.apache.thrift.TException; + + public GetOpenTxnsInfoResponse get_open_txns_info() throws org.apache.thrift.TException; + + public OpenTxnsResponse open_txns(OpenTxnRequest rqst) throws org.apache.thrift.TException; + + public void abort_txn(AbortTxnRequest rqst) throws NoSuchTxnException, org.apache.thrift.TException; + + public void commit_txn(CommitTxnRequest rqst) throws NoSuchTxnException, TxnAbortedException, org.apache.thrift.TException; + + public LockResponse lock(LockRequest rqst) throws NoSuchTxnException, TxnAbortedException, org.apache.thrift.TException; + + public LockResponse check_lock(CheckLockRequest rqst) throws NoSuchTxnException, TxnAbortedException, NoSuchLockException, org.apache.thrift.TException; + + public void unlock(UnlockRequest rqst) throws NoSuchLockException, TxnOpenException, org.apache.thrift.TException; + + public ShowLocksResponse show_locks(ShowLocksRequest rqst) throws org.apache.thrift.TException; + + public void heartbeat(HeartbeatRequest ids) throws NoSuchLockException, NoSuchTxnException, TxnAbortedException, org.apache.thrift.TException; + + public void compact(CompactionRequest rqst) throws org.apache.thrift.TException; + + public ShowCompactResponse show_compact(ShowCompactRequest rqst) throws org.apache.thrift.TException; + } public interface AsyncIface extends com.facebook.fb303.FacebookService .AsyncIface { @@ -410,6 +434,30 @@ public void cancel_delegation_token(String token_str_form, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + public void get_open_txns(org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + + public void get_open_txns_info(org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + + public void open_txns(OpenTxnRequest rqst, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + + public void abort_txn(AbortTxnRequest rqst, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + + public void commit_txn(CommitTxnRequest rqst, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + + public void lock(LockRequest rqst, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + + public void check_lock(CheckLockRequest rqst, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + + public void unlock(UnlockRequest rqst, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + + public void show_locks(ShowLocksRequest rqst, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + + public void heartbeat(HeartbeatRequest ids, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + + public void compact(CompactionRequest rqst, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + + public void show_compact(ShowCompactRequest rqst, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + } public static class Client extends com.facebook.fb303.FacebookService.Client implements Iface { @@ -3237,6 +3285,304 @@ public void recv_cancel_delegation_token() throws MetaException, org.apache.thri return; } + public GetOpenTxnsResponse get_open_txns() throws org.apache.thrift.TException + { + send_get_open_txns(); + return recv_get_open_txns(); + } + + public void send_get_open_txns() throws org.apache.thrift.TException + { + get_open_txns_args args = new get_open_txns_args(); + sendBase("get_open_txns", args); + } + + public GetOpenTxnsResponse recv_get_open_txns() throws org.apache.thrift.TException + { + get_open_txns_result result = new get_open_txns_result(); + receiveBase(result, "get_open_txns"); + if (result.isSetSuccess()) { + return result.success; + } + throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "get_open_txns failed: unknown result"); + } + + public GetOpenTxnsInfoResponse get_open_txns_info() throws org.apache.thrift.TException + { + send_get_open_txns_info(); + return recv_get_open_txns_info(); + } + + public void send_get_open_txns_info() throws org.apache.thrift.TException + { + get_open_txns_info_args args = new get_open_txns_info_args(); + sendBase("get_open_txns_info", args); + } + + public GetOpenTxnsInfoResponse recv_get_open_txns_info() throws org.apache.thrift.TException + { + get_open_txns_info_result result = new get_open_txns_info_result(); + receiveBase(result, "get_open_txns_info"); + if (result.isSetSuccess()) { + return result.success; + } + throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "get_open_txns_info failed: unknown result"); + } + + public OpenTxnsResponse open_txns(OpenTxnRequest rqst) throws org.apache.thrift.TException + { + send_open_txns(rqst); + return recv_open_txns(); + } + + public void send_open_txns(OpenTxnRequest rqst) throws org.apache.thrift.TException + { + open_txns_args args = new open_txns_args(); + args.setRqst(rqst); + sendBase("open_txns", args); + } + + public OpenTxnsResponse recv_open_txns() throws org.apache.thrift.TException + { + open_txns_result result = new open_txns_result(); + receiveBase(result, "open_txns"); + if (result.isSetSuccess()) { + return result.success; + } + throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "open_txns failed: unknown result"); + } + + public void abort_txn(AbortTxnRequest rqst) throws NoSuchTxnException, org.apache.thrift.TException + { + send_abort_txn(rqst); + recv_abort_txn(); + } + + public void send_abort_txn(AbortTxnRequest rqst) throws org.apache.thrift.TException + { + abort_txn_args args = new abort_txn_args(); + args.setRqst(rqst); + sendBase("abort_txn", args); + } + + public void recv_abort_txn() throws NoSuchTxnException, org.apache.thrift.TException + { + abort_txn_result result = new abort_txn_result(); + receiveBase(result, "abort_txn"); + if (result.o1 != null) { + throw result.o1; + } + return; + } + + public void commit_txn(CommitTxnRequest rqst) throws NoSuchTxnException, TxnAbortedException, org.apache.thrift.TException + { + send_commit_txn(rqst); + recv_commit_txn(); + } + + public void send_commit_txn(CommitTxnRequest rqst) throws org.apache.thrift.TException + { + commit_txn_args args = new commit_txn_args(); + args.setRqst(rqst); + sendBase("commit_txn", args); + } + + public void recv_commit_txn() throws NoSuchTxnException, TxnAbortedException, org.apache.thrift.TException + { + commit_txn_result result = new commit_txn_result(); + receiveBase(result, "commit_txn"); + if (result.o1 != null) { + throw result.o1; + } + if (result.o2 != null) { + throw result.o2; + } + return; + } + + public LockResponse lock(LockRequest rqst) throws NoSuchTxnException, TxnAbortedException, org.apache.thrift.TException + { + send_lock(rqst); + return recv_lock(); + } + + public void send_lock(LockRequest rqst) throws org.apache.thrift.TException + { + lock_args args = new lock_args(); + args.setRqst(rqst); + sendBase("lock", args); + } + + public LockResponse recv_lock() throws NoSuchTxnException, TxnAbortedException, org.apache.thrift.TException + { + lock_result result = new lock_result(); + receiveBase(result, "lock"); + if (result.isSetSuccess()) { + return result.success; + } + if (result.o1 != null) { + throw result.o1; + } + if (result.o2 != null) { + throw result.o2; + } + throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "lock failed: unknown result"); + } + + public LockResponse check_lock(CheckLockRequest rqst) throws NoSuchTxnException, TxnAbortedException, NoSuchLockException, org.apache.thrift.TException + { + send_check_lock(rqst); + return recv_check_lock(); + } + + public void send_check_lock(CheckLockRequest rqst) throws org.apache.thrift.TException + { + check_lock_args args = new check_lock_args(); + args.setRqst(rqst); + sendBase("check_lock", args); + } + + public LockResponse recv_check_lock() throws NoSuchTxnException, TxnAbortedException, NoSuchLockException, org.apache.thrift.TException + { + check_lock_result result = new check_lock_result(); + receiveBase(result, "check_lock"); + if (result.isSetSuccess()) { + return result.success; + } + if (result.o1 != null) { + throw result.o1; + } + if (result.o2 != null) { + throw result.o2; + } + if (result.o3 != null) { + throw result.o3; + } + throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "check_lock failed: unknown result"); + } + + public void unlock(UnlockRequest rqst) throws NoSuchLockException, TxnOpenException, org.apache.thrift.TException + { + send_unlock(rqst); + recv_unlock(); + } + + public void send_unlock(UnlockRequest rqst) throws org.apache.thrift.TException + { + unlock_args args = new unlock_args(); + args.setRqst(rqst); + sendBase("unlock", args); + } + + public void recv_unlock() throws NoSuchLockException, TxnOpenException, org.apache.thrift.TException + { + unlock_result result = new unlock_result(); + receiveBase(result, "unlock"); + if (result.o1 != null) { + throw result.o1; + } + if (result.o2 != null) { + throw result.o2; + } + return; + } + + public ShowLocksResponse show_locks(ShowLocksRequest rqst) throws org.apache.thrift.TException + { + send_show_locks(rqst); + return recv_show_locks(); + } + + public void send_show_locks(ShowLocksRequest rqst) throws org.apache.thrift.TException + { + show_locks_args args = new show_locks_args(); + args.setRqst(rqst); + sendBase("show_locks", args); + } + + public ShowLocksResponse recv_show_locks() throws org.apache.thrift.TException + { + show_locks_result result = new show_locks_result(); + receiveBase(result, "show_locks"); + if (result.isSetSuccess()) { + return result.success; + } + throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "show_locks failed: unknown result"); + } + + public void heartbeat(HeartbeatRequest ids) throws NoSuchLockException, NoSuchTxnException, TxnAbortedException, org.apache.thrift.TException + { + send_heartbeat(ids); + recv_heartbeat(); + } + + public void send_heartbeat(HeartbeatRequest ids) throws org.apache.thrift.TException + { + heartbeat_args args = new heartbeat_args(); + args.setIds(ids); + sendBase("heartbeat", args); + } + + public void recv_heartbeat() throws NoSuchLockException, NoSuchTxnException, TxnAbortedException, org.apache.thrift.TException + { + heartbeat_result result = new heartbeat_result(); + receiveBase(result, "heartbeat"); + if (result.o1 != null) { + throw result.o1; + } + if (result.o2 != null) { + throw result.o2; + } + if (result.o3 != null) { + throw result.o3; + } + return; + } + + public void compact(CompactionRequest rqst) throws org.apache.thrift.TException + { + send_compact(rqst); + recv_compact(); + } + + public void send_compact(CompactionRequest rqst) throws org.apache.thrift.TException + { + compact_args args = new compact_args(); + args.setRqst(rqst); + sendBase("compact", args); + } + + public void recv_compact() throws org.apache.thrift.TException + { + compact_result result = new compact_result(); + receiveBase(result, "compact"); + return; + } + + public ShowCompactResponse show_compact(ShowCompactRequest rqst) throws org.apache.thrift.TException + { + send_show_compact(rqst); + return recv_show_compact(); + } + + public void send_show_compact(ShowCompactRequest rqst) throws org.apache.thrift.TException + { + show_compact_args args = new show_compact_args(); + args.setRqst(rqst); + sendBase("show_compact", args); + } + + public ShowCompactResponse recv_show_compact() throws org.apache.thrift.TException + { + show_compact_result result = new show_compact_result(); + receiveBase(result, "show_compact"); + if (result.isSetSuccess()) { + return result.success; + } + throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "show_compact failed: unknown result"); + } + } public static class AsyncClient extends com.facebook.fb303.FacebookService.AsyncClient implements AsyncIface { public static class Factory implements org.apache.thrift.async.TAsyncClientFactory { @@ -6619,6 +6965,384 @@ public void getResult() throws MetaException, org.apache.thrift.TException { } } + public void get_open_txns(org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + checkReady(); + get_open_txns_call method_call = new get_open_txns_call(resultHandler, this, ___protocolFactory, ___transport); + this.___currentMethod = method_call; + ___manager.call(method_call); + } + + public static class get_open_txns_call extends org.apache.thrift.async.TAsyncMethodCall { + public get_open_txns_call(org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { + super(client, protocolFactory, transport, resultHandler, false); + } + + public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { + prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("get_open_txns", org.apache.thrift.protocol.TMessageType.CALL, 0)); + get_open_txns_args args = new get_open_txns_args(); + args.write(prot); + prot.writeMessageEnd(); + } + + public GetOpenTxnsResponse getResult() throws org.apache.thrift.TException { + if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { + throw new IllegalStateException("Method call not finished!"); + } + org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); + org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); + return (new Client(prot)).recv_get_open_txns(); + } + } + + public void get_open_txns_info(org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + checkReady(); + get_open_txns_info_call method_call = new get_open_txns_info_call(resultHandler, this, ___protocolFactory, ___transport); + this.___currentMethod = method_call; + ___manager.call(method_call); + } + + public static class get_open_txns_info_call extends org.apache.thrift.async.TAsyncMethodCall { + public get_open_txns_info_call(org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { + super(client, protocolFactory, transport, resultHandler, false); + } + + public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { + prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("get_open_txns_info", org.apache.thrift.protocol.TMessageType.CALL, 0)); + get_open_txns_info_args args = new get_open_txns_info_args(); + args.write(prot); + prot.writeMessageEnd(); + } + + public GetOpenTxnsInfoResponse getResult() throws org.apache.thrift.TException { + if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { + throw new IllegalStateException("Method call not finished!"); + } + org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); + org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); + return (new Client(prot)).recv_get_open_txns_info(); + } + } + + public void open_txns(OpenTxnRequest rqst, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + checkReady(); + open_txns_call method_call = new open_txns_call(rqst, resultHandler, this, ___protocolFactory, ___transport); + this.___currentMethod = method_call; + ___manager.call(method_call); + } + + public static class open_txns_call extends org.apache.thrift.async.TAsyncMethodCall { + private OpenTxnRequest rqst; + public open_txns_call(OpenTxnRequest rqst, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { + super(client, protocolFactory, transport, resultHandler, false); + this.rqst = rqst; + } + + public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { + prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("open_txns", org.apache.thrift.protocol.TMessageType.CALL, 0)); + open_txns_args args = new open_txns_args(); + args.setRqst(rqst); + args.write(prot); + prot.writeMessageEnd(); + } + + public OpenTxnsResponse getResult() throws org.apache.thrift.TException { + if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { + throw new IllegalStateException("Method call not finished!"); + } + org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); + org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); + return (new Client(prot)).recv_open_txns(); + } + } + + public void abort_txn(AbortTxnRequest rqst, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + checkReady(); + abort_txn_call method_call = new abort_txn_call(rqst, resultHandler, this, ___protocolFactory, ___transport); + this.___currentMethod = method_call; + ___manager.call(method_call); + } + + public static class abort_txn_call extends org.apache.thrift.async.TAsyncMethodCall { + private AbortTxnRequest rqst; + public abort_txn_call(AbortTxnRequest rqst, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { + super(client, protocolFactory, transport, resultHandler, false); + this.rqst = rqst; + } + + public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { + prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("abort_txn", org.apache.thrift.protocol.TMessageType.CALL, 0)); + abort_txn_args args = new abort_txn_args(); + args.setRqst(rqst); + args.write(prot); + prot.writeMessageEnd(); + } + + public void getResult() throws NoSuchTxnException, org.apache.thrift.TException { + if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { + throw new IllegalStateException("Method call not finished!"); + } + org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); + org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); + (new Client(prot)).recv_abort_txn(); + } + } + + public void commit_txn(CommitTxnRequest rqst, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + checkReady(); + commit_txn_call method_call = new commit_txn_call(rqst, resultHandler, this, ___protocolFactory, ___transport); + this.___currentMethod = method_call; + ___manager.call(method_call); + } + + public static class commit_txn_call extends org.apache.thrift.async.TAsyncMethodCall { + private CommitTxnRequest rqst; + public commit_txn_call(CommitTxnRequest rqst, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { + super(client, protocolFactory, transport, resultHandler, false); + this.rqst = rqst; + } + + public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { + prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("commit_txn", org.apache.thrift.protocol.TMessageType.CALL, 0)); + commit_txn_args args = new commit_txn_args(); + args.setRqst(rqst); + args.write(prot); + prot.writeMessageEnd(); + } + + public void getResult() throws NoSuchTxnException, TxnAbortedException, org.apache.thrift.TException { + if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { + throw new IllegalStateException("Method call not finished!"); + } + org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); + org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); + (new Client(prot)).recv_commit_txn(); + } + } + + public void lock(LockRequest rqst, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + checkReady(); + lock_call method_call = new lock_call(rqst, resultHandler, this, ___protocolFactory, ___transport); + this.___currentMethod = method_call; + ___manager.call(method_call); + } + + public static class lock_call extends org.apache.thrift.async.TAsyncMethodCall { + private LockRequest rqst; + public lock_call(LockRequest rqst, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { + super(client, protocolFactory, transport, resultHandler, false); + this.rqst = rqst; + } + + public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { + prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("lock", org.apache.thrift.protocol.TMessageType.CALL, 0)); + lock_args args = new lock_args(); + args.setRqst(rqst); + args.write(prot); + prot.writeMessageEnd(); + } + + public LockResponse getResult() throws NoSuchTxnException, TxnAbortedException, org.apache.thrift.TException { + if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { + throw new IllegalStateException("Method call not finished!"); + } + org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); + org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); + return (new Client(prot)).recv_lock(); + } + } + + public void check_lock(CheckLockRequest rqst, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + checkReady(); + check_lock_call method_call = new check_lock_call(rqst, resultHandler, this, ___protocolFactory, ___transport); + this.___currentMethod = method_call; + ___manager.call(method_call); + } + + public static class check_lock_call extends org.apache.thrift.async.TAsyncMethodCall { + private CheckLockRequest rqst; + public check_lock_call(CheckLockRequest rqst, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { + super(client, protocolFactory, transport, resultHandler, false); + this.rqst = rqst; + } + + public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { + prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("check_lock", org.apache.thrift.protocol.TMessageType.CALL, 0)); + check_lock_args args = new check_lock_args(); + args.setRqst(rqst); + args.write(prot); + prot.writeMessageEnd(); + } + + public LockResponse getResult() throws NoSuchTxnException, TxnAbortedException, NoSuchLockException, org.apache.thrift.TException { + if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { + throw new IllegalStateException("Method call not finished!"); + } + org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); + org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); + return (new Client(prot)).recv_check_lock(); + } + } + + public void unlock(UnlockRequest rqst, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + checkReady(); + unlock_call method_call = new unlock_call(rqst, resultHandler, this, ___protocolFactory, ___transport); + this.___currentMethod = method_call; + ___manager.call(method_call); + } + + public static class unlock_call extends org.apache.thrift.async.TAsyncMethodCall { + private UnlockRequest rqst; + public unlock_call(UnlockRequest rqst, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { + super(client, protocolFactory, transport, resultHandler, false); + this.rqst = rqst; + } + + public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { + prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("unlock", org.apache.thrift.protocol.TMessageType.CALL, 0)); + unlock_args args = new unlock_args(); + args.setRqst(rqst); + args.write(prot); + prot.writeMessageEnd(); + } + + public void getResult() throws NoSuchLockException, TxnOpenException, org.apache.thrift.TException { + if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { + throw new IllegalStateException("Method call not finished!"); + } + org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); + org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); + (new Client(prot)).recv_unlock(); + } + } + + public void show_locks(ShowLocksRequest rqst, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + checkReady(); + show_locks_call method_call = new show_locks_call(rqst, resultHandler, this, ___protocolFactory, ___transport); + this.___currentMethod = method_call; + ___manager.call(method_call); + } + + public static class show_locks_call extends org.apache.thrift.async.TAsyncMethodCall { + private ShowLocksRequest rqst; + public show_locks_call(ShowLocksRequest rqst, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { + super(client, protocolFactory, transport, resultHandler, false); + this.rqst = rqst; + } + + public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { + prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("show_locks", org.apache.thrift.protocol.TMessageType.CALL, 0)); + show_locks_args args = new show_locks_args(); + args.setRqst(rqst); + args.write(prot); + prot.writeMessageEnd(); + } + + public ShowLocksResponse getResult() throws org.apache.thrift.TException { + if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { + throw new IllegalStateException("Method call not finished!"); + } + org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); + org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); + return (new Client(prot)).recv_show_locks(); + } + } + + public void heartbeat(HeartbeatRequest ids, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + checkReady(); + heartbeat_call method_call = new heartbeat_call(ids, resultHandler, this, ___protocolFactory, ___transport); + this.___currentMethod = method_call; + ___manager.call(method_call); + } + + public static class heartbeat_call extends org.apache.thrift.async.TAsyncMethodCall { + private HeartbeatRequest ids; + public heartbeat_call(HeartbeatRequest ids, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { + super(client, protocolFactory, transport, resultHandler, false); + this.ids = ids; + } + + public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { + prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("heartbeat", org.apache.thrift.protocol.TMessageType.CALL, 0)); + heartbeat_args args = new heartbeat_args(); + args.setIds(ids); + args.write(prot); + prot.writeMessageEnd(); + } + + public void getResult() throws NoSuchLockException, NoSuchTxnException, TxnAbortedException, org.apache.thrift.TException { + if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { + throw new IllegalStateException("Method call not finished!"); + } + org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); + org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); + (new Client(prot)).recv_heartbeat(); + } + } + + public void compact(CompactionRequest rqst, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + checkReady(); + compact_call method_call = new compact_call(rqst, resultHandler, this, ___protocolFactory, ___transport); + this.___currentMethod = method_call; + ___manager.call(method_call); + } + + public static class compact_call extends org.apache.thrift.async.TAsyncMethodCall { + private CompactionRequest rqst; + public compact_call(CompactionRequest rqst, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { + super(client, protocolFactory, transport, resultHandler, false); + this.rqst = rqst; + } + + public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { + prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("compact", org.apache.thrift.protocol.TMessageType.CALL, 0)); + compact_args args = new compact_args(); + args.setRqst(rqst); + args.write(prot); + prot.writeMessageEnd(); + } + + public void getResult() throws org.apache.thrift.TException { + if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { + throw new IllegalStateException("Method call not finished!"); + } + org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); + org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); + (new Client(prot)).recv_compact(); + } + } + + public void show_compact(ShowCompactRequest rqst, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + checkReady(); + show_compact_call method_call = new show_compact_call(rqst, resultHandler, this, ___protocolFactory, ___transport); + this.___currentMethod = method_call; + ___manager.call(method_call); + } + + public static class show_compact_call extends org.apache.thrift.async.TAsyncMethodCall { + private ShowCompactRequest rqst; + public show_compact_call(ShowCompactRequest rqst, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { + super(client, protocolFactory, transport, resultHandler, false); + this.rqst = rqst; + } + + public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { + prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("show_compact", org.apache.thrift.protocol.TMessageType.CALL, 0)); + show_compact_args args = new show_compact_args(); + args.setRqst(rqst); + args.write(prot); + prot.writeMessageEnd(); + } + + public ShowCompactResponse getResult() throws org.apache.thrift.TException { + if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { + throw new IllegalStateException("Method call not finished!"); + } + org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); + org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); + return (new Client(prot)).recv_show_compact(); + } + } + } public static class Processor extends com.facebook.fb303.FacebookService.Processor implements org.apache.thrift.TProcessor { @@ -6724,6 +7448,18 @@ protected Processor(I iface, Map extends org.apache.thrift.ProcessFunction { + public get_open_txns() { + super("get_open_txns"); + } + + public get_open_txns_args getEmptyArgsInstance() { + return new get_open_txns_args(); + } + + protected boolean isOneway() { + return false; + } + + public get_open_txns_result getResult(I iface, get_open_txns_args args) throws org.apache.thrift.TException { + get_open_txns_result result = new get_open_txns_result(); + result.success = iface.get_open_txns(); + return result; + } + } + + public static class get_open_txns_info extends org.apache.thrift.ProcessFunction { + public get_open_txns_info() { + super("get_open_txns_info"); + } + + public get_open_txns_info_args getEmptyArgsInstance() { + return new get_open_txns_info_args(); + } + + protected boolean isOneway() { + return false; + } + + public get_open_txns_info_result getResult(I iface, get_open_txns_info_args args) throws org.apache.thrift.TException { + get_open_txns_info_result result = new get_open_txns_info_result(); + result.success = iface.get_open_txns_info(); + return result; + } + } + + public static class open_txns extends org.apache.thrift.ProcessFunction { + public open_txns() { + super("open_txns"); + } + + public open_txns_args getEmptyArgsInstance() { + return new open_txns_args(); + } + + protected boolean isOneway() { + return false; + } + + public open_txns_result getResult(I iface, open_txns_args args) throws org.apache.thrift.TException { + open_txns_result result = new open_txns_result(); + result.success = iface.open_txns(args.rqst); + return result; + } + } + + public static class abort_txn extends org.apache.thrift.ProcessFunction { + public abort_txn() { + super("abort_txn"); + } + + public abort_txn_args getEmptyArgsInstance() { + return new abort_txn_args(); + } + + protected boolean isOneway() { + return false; + } + + public abort_txn_result getResult(I iface, abort_txn_args args) throws org.apache.thrift.TException { + abort_txn_result result = new abort_txn_result(); + try { + iface.abort_txn(args.rqst); + } catch (NoSuchTxnException o1) { + result.o1 = o1; + } + return result; + } + } + + public static class commit_txn extends org.apache.thrift.ProcessFunction { + public commit_txn() { + super("commit_txn"); + } + + public commit_txn_args getEmptyArgsInstance() { + return new commit_txn_args(); + } + + protected boolean isOneway() { + return false; + } + + public commit_txn_result getResult(I iface, commit_txn_args args) throws org.apache.thrift.TException { + commit_txn_result result = new commit_txn_result(); + try { + iface.commit_txn(args.rqst); + } catch (NoSuchTxnException o1) { + result.o1 = o1; + } catch (TxnAbortedException o2) { + result.o2 = o2; + } + return result; + } + } + + public static class lock extends org.apache.thrift.ProcessFunction { + public lock() { + super("lock"); + } + + public lock_args getEmptyArgsInstance() { + return new lock_args(); + } + + protected boolean isOneway() { + return false; + } + + public lock_result getResult(I iface, lock_args args) throws org.apache.thrift.TException { + lock_result result = new lock_result(); + try { + result.success = iface.lock(args.rqst); + } catch (NoSuchTxnException o1) { + result.o1 = o1; + } catch (TxnAbortedException o2) { + result.o2 = o2; + } + return result; + } + } + + public static class check_lock extends org.apache.thrift.ProcessFunction { + public check_lock() { + super("check_lock"); + } + + public check_lock_args getEmptyArgsInstance() { + return new check_lock_args(); + } + + protected boolean isOneway() { + return false; + } + + public check_lock_result getResult(I iface, check_lock_args args) throws org.apache.thrift.TException { + check_lock_result result = new check_lock_result(); + try { + result.success = iface.check_lock(args.rqst); + } catch (NoSuchTxnException o1) { + result.o1 = o1; + } catch (TxnAbortedException o2) { + result.o2 = o2; + } catch (NoSuchLockException o3) { + result.o3 = o3; + } + return result; + } + } + + public static class unlock extends org.apache.thrift.ProcessFunction { + public unlock() { + super("unlock"); + } + + public unlock_args getEmptyArgsInstance() { + return new unlock_args(); + } + + protected boolean isOneway() { + return false; + } + + public unlock_result getResult(I iface, unlock_args args) throws org.apache.thrift.TException { + unlock_result result = new unlock_result(); + try { + iface.unlock(args.rqst); + } catch (NoSuchLockException o1) { + result.o1 = o1; + } catch (TxnOpenException o2) { + result.o2 = o2; + } + return result; + } + } + + public static class show_locks extends org.apache.thrift.ProcessFunction { + public show_locks() { + super("show_locks"); + } + + public show_locks_args getEmptyArgsInstance() { + return new show_locks_args(); + } + + protected boolean isOneway() { + return false; + } + + public show_locks_result getResult(I iface, show_locks_args args) throws org.apache.thrift.TException { + show_locks_result result = new show_locks_result(); + result.success = iface.show_locks(args.rqst); + return result; + } + } + + public static class heartbeat extends org.apache.thrift.ProcessFunction { + public heartbeat() { + super("heartbeat"); + } + + public heartbeat_args getEmptyArgsInstance() { + return new heartbeat_args(); + } + + protected boolean isOneway() { + return false; + } + + public heartbeat_result getResult(I iface, heartbeat_args args) throws org.apache.thrift.TException { + heartbeat_result result = new heartbeat_result(); + try { + iface.heartbeat(args.ids); + } catch (NoSuchLockException o1) { + result.o1 = o1; + } catch (NoSuchTxnException o2) { + result.o2 = o2; + } catch (TxnAbortedException o3) { + result.o3 = o3; + } + return result; + } + } + + public static class compact extends org.apache.thrift.ProcessFunction { + public compact() { + super("compact"); + } + + public compact_args getEmptyArgsInstance() { + return new compact_args(); + } + + protected boolean isOneway() { + return false; + } + + public compact_result getResult(I iface, compact_args args) throws org.apache.thrift.TException { + compact_result result = new compact_result(); + iface.compact(args.rqst); + return result; + } + } + + public static class show_compact extends org.apache.thrift.ProcessFunction { + public show_compact() { + super("show_compact"); + } + + public show_compact_args getEmptyArgsInstance() { + return new show_compact_args(); + } + + protected boolean isOneway() { + return false; + } + + public show_compact_result getResult(I iface, show_compact_args args) throws org.apache.thrift.TException { + show_compact_result result = new show_compact_result(); + result.success = iface.show_compact(args.rqst); + return result; + } + } + } public static class create_database_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { @@ -12930,13 +13944,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, get_databases_resul case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list362 = iprot.readListBegin(); - struct.success = new ArrayList(_list362.size); - for (int _i363 = 0; _i363 < _list362.size; ++_i363) + org.apache.thrift.protocol.TList _list410 = iprot.readListBegin(); + struct.success = new ArrayList(_list410.size); + for (int _i411 = 0; _i411 < _list410.size; ++_i411) { - String _elem364; // optional - _elem364 = iprot.readString(); - struct.success.add(_elem364); + String _elem412; // required + _elem412 = iprot.readString(); + struct.success.add(_elem412); } iprot.readListEnd(); } @@ -12971,9 +13985,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, get_databases_resu oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size())); - for (String _iter365 : struct.success) + for (String _iter413 : struct.success) { - oprot.writeString(_iter365); + oprot.writeString(_iter413); } oprot.writeListEnd(); } @@ -13012,9 +14026,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, get_databases_resul if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (String _iter366 : struct.success) + for (String _iter414 : struct.success) { - oprot.writeString(_iter366); + oprot.writeString(_iter414); } } } @@ -13029,13 +14043,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, get_databases_result BitSet incoming = iprot.readBitSet(2); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list367 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); - struct.success = new ArrayList(_list367.size); - for (int _i368 = 0; _i368 < _list367.size; ++_i368) + org.apache.thrift.protocol.TList _list415 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); + struct.success = new ArrayList(_list415.size); + for (int _i416 = 0; _i416 < _list415.size; ++_i416) { - String _elem369; // optional - _elem369 = iprot.readString(); - struct.success.add(_elem369); + String _elem417; // required + _elem417 = iprot.readString(); + struct.success.add(_elem417); } } struct.setSuccessIsSet(true); @@ -13692,13 +14706,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, get_all_databases_r case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list370 = iprot.readListBegin(); - struct.success = new ArrayList(_list370.size); - for (int _i371 = 0; _i371 < _list370.size; ++_i371) + org.apache.thrift.protocol.TList _list418 = iprot.readListBegin(); + struct.success = new ArrayList(_list418.size); + for (int _i419 = 0; _i419 < _list418.size; ++_i419) { - String _elem372; // optional - _elem372 = iprot.readString(); - struct.success.add(_elem372); + String _elem420; // required + _elem420 = iprot.readString(); + struct.success.add(_elem420); } iprot.readListEnd(); } @@ -13733,9 +14747,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, get_all_databases_ oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size())); - for (String _iter373 : struct.success) + for (String _iter421 : struct.success) { - oprot.writeString(_iter373); + oprot.writeString(_iter421); } oprot.writeListEnd(); } @@ -13774,9 +14788,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, get_all_databases_r if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (String _iter374 : struct.success) + for (String _iter422 : struct.success) { - oprot.writeString(_iter374); + oprot.writeString(_iter422); } } } @@ -13791,13 +14805,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, get_all_databases_re BitSet incoming = iprot.readBitSet(2); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list375 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); - struct.success = new ArrayList(_list375.size); - for (int _i376 = 0; _i376 < _list375.size; ++_i376) + org.apache.thrift.protocol.TList _list423 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); + struct.success = new ArrayList(_list423.size); + for (int _i424 = 0; _i424 < _list423.size; ++_i424) { - String _elem377; // optional - _elem377 = iprot.readString(); - struct.success.add(_elem377); + String _elem425; // required + _elem425 = iprot.readString(); + struct.success.add(_elem425); } } struct.setSuccessIsSet(true); @@ -18404,16 +19418,16 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, get_type_all_result case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { { - org.apache.thrift.protocol.TMap _map378 = iprot.readMapBegin(); - struct.success = new HashMap(2*_map378.size); - for (int _i379 = 0; _i379 < _map378.size; ++_i379) + org.apache.thrift.protocol.TMap _map426 = iprot.readMapBegin(); + struct.success = new HashMap(2*_map426.size); + for (int _i427 = 0; _i427 < _map426.size; ++_i427) { - String _key380; // required - Type _val381; // required - _key380 = iprot.readString(); - _val381 = new Type(); - _val381.read(iprot); - struct.success.put(_key380, _val381); + String _key428; // required + Type _val429; // required + _key428 = iprot.readString(); + _val429 = new Type(); + _val429.read(iprot); + struct.success.put(_key428, _val429); } iprot.readMapEnd(); } @@ -18448,10 +19462,10 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, get_type_all_resul oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, struct.success.size())); - for (Map.Entry _iter382 : struct.success.entrySet()) + for (Map.Entry _iter430 : struct.success.entrySet()) { - oprot.writeString(_iter382.getKey()); - _iter382.getValue().write(oprot); + oprot.writeString(_iter430.getKey()); + _iter430.getValue().write(oprot); } oprot.writeMapEnd(); } @@ -18490,10 +19504,10 @@ public void write(org.apache.thrift.protocol.TProtocol prot, get_type_all_result if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (Map.Entry _iter383 : struct.success.entrySet()) + for (Map.Entry _iter431 : struct.success.entrySet()) { - oprot.writeString(_iter383.getKey()); - _iter383.getValue().write(oprot); + oprot.writeString(_iter431.getKey()); + _iter431.getValue().write(oprot); } } } @@ -18508,16 +19522,16 @@ public void read(org.apache.thrift.protocol.TProtocol prot, get_type_all_result BitSet incoming = iprot.readBitSet(2); if (incoming.get(0)) { { - org.apache.thrift.protocol.TMap _map384 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); - struct.success = new HashMap(2*_map384.size); - for (int _i385 = 0; _i385 < _map384.size; ++_i385) + org.apache.thrift.protocol.TMap _map432 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); + struct.success = new HashMap(2*_map432.size); + for (int _i433 = 0; _i433 < _map432.size; ++_i433) { - String _key386; // required - Type _val387; // required - _key386 = iprot.readString(); - _val387 = new Type(); - _val387.read(iprot); - struct.success.put(_key386, _val387); + String _key434; // required + Type _val435; // required + _key434 = iprot.readString(); + _val435 = new Type(); + _val435.read(iprot); + struct.success.put(_key434, _val435); } } struct.setSuccessIsSet(true); @@ -19552,14 +20566,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, get_fields_result s case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list388 = iprot.readListBegin(); - struct.success = new ArrayList(_list388.size); - for (int _i389 = 0; _i389 < _list388.size; ++_i389) + org.apache.thrift.protocol.TList _list436 = iprot.readListBegin(); + struct.success = new ArrayList(_list436.size); + for (int _i437 = 0; _i437 < _list436.size; ++_i437) { - FieldSchema _elem390; // optional - _elem390 = new FieldSchema(); - _elem390.read(iprot); - struct.success.add(_elem390); + FieldSchema _elem438; // required + _elem438 = new FieldSchema(); + _elem438.read(iprot); + struct.success.add(_elem438); } iprot.readListEnd(); } @@ -19612,9 +20626,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, get_fields_result oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size())); - for (FieldSchema _iter391 : struct.success) + for (FieldSchema _iter439 : struct.success) { - _iter391.write(oprot); + _iter439.write(oprot); } oprot.writeListEnd(); } @@ -19669,9 +20683,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, get_fields_result s if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (FieldSchema _iter392 : struct.success) + for (FieldSchema _iter440 : struct.success) { - _iter392.write(oprot); + _iter440.write(oprot); } } } @@ -19692,14 +20706,14 @@ public void read(org.apache.thrift.protocol.TProtocol prot, get_fields_result st BitSet incoming = iprot.readBitSet(4); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list393 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); - struct.success = new ArrayList(_list393.size); - for (int _i394 = 0; _i394 < _list393.size; ++_i394) + org.apache.thrift.protocol.TList _list441 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); + struct.success = new ArrayList(_list441.size); + for (int _i442 = 0; _i442 < _list441.size; ++_i442) { - FieldSchema _elem395; // optional - _elem395 = new FieldSchema(); - _elem395.read(iprot); - struct.success.add(_elem395); + FieldSchema _elem443; // required + _elem443 = new FieldSchema(); + _elem443.read(iprot); + struct.success.add(_elem443); } } struct.setSuccessIsSet(true); @@ -20744,14 +21758,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, get_schema_result s case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list396 = iprot.readListBegin(); - struct.success = new ArrayList(_list396.size); - for (int _i397 = 0; _i397 < _list396.size; ++_i397) + org.apache.thrift.protocol.TList _list444 = iprot.readListBegin(); + struct.success = new ArrayList(_list444.size); + for (int _i445 = 0; _i445 < _list444.size; ++_i445) { - FieldSchema _elem398; // optional - _elem398 = new FieldSchema(); - _elem398.read(iprot); - struct.success.add(_elem398); + FieldSchema _elem446; // required + _elem446 = new FieldSchema(); + _elem446.read(iprot); + struct.success.add(_elem446); } iprot.readListEnd(); } @@ -20804,9 +21818,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, get_schema_result oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size())); - for (FieldSchema _iter399 : struct.success) + for (FieldSchema _iter447 : struct.success) { - _iter399.write(oprot); + _iter447.write(oprot); } oprot.writeListEnd(); } @@ -20861,9 +21875,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, get_schema_result s if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (FieldSchema _iter400 : struct.success) + for (FieldSchema _iter448 : struct.success) { - _iter400.write(oprot); + _iter448.write(oprot); } } } @@ -20884,14 +21898,14 @@ public void read(org.apache.thrift.protocol.TProtocol prot, get_schema_result st BitSet incoming = iprot.readBitSet(4); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list401 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); - struct.success = new ArrayList(_list401.size); - for (int _i402 = 0; _i402 < _list401.size; ++_i402) + org.apache.thrift.protocol.TList _list449 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); + struct.success = new ArrayList(_list449.size); + for (int _i450 = 0; _i450 < _list449.size; ++_i450) { - FieldSchema _elem403; // optional - _elem403 = new FieldSchema(); - _elem403.read(iprot); - struct.success.add(_elem403); + FieldSchema _elem451; // required + _elem451 = new FieldSchema(); + _elem451.read(iprot); + struct.success.add(_elem451); } } struct.setSuccessIsSet(true); @@ -26134,13 +27148,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, get_tables_result s case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list404 = iprot.readListBegin(); - struct.success = new ArrayList(_list404.size); - for (int _i405 = 0; _i405 < _list404.size; ++_i405) + org.apache.thrift.protocol.TList _list452 = iprot.readListBegin(); + struct.success = new ArrayList(_list452.size); + for (int _i453 = 0; _i453 < _list452.size; ++_i453) { - String _elem406; // optional - _elem406 = iprot.readString(); - struct.success.add(_elem406); + String _elem454; // required + _elem454 = iprot.readString(); + struct.success.add(_elem454); } iprot.readListEnd(); } @@ -26175,9 +27189,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, get_tables_result oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size())); - for (String _iter407 : struct.success) + for (String _iter455 : struct.success) { - oprot.writeString(_iter407); + oprot.writeString(_iter455); } oprot.writeListEnd(); } @@ -26216,9 +27230,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, get_tables_result s if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (String _iter408 : struct.success) + for (String _iter456 : struct.success) { - oprot.writeString(_iter408); + oprot.writeString(_iter456); } } } @@ -26233,13 +27247,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, get_tables_result st BitSet incoming = iprot.readBitSet(2); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list409 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); - struct.success = new ArrayList(_list409.size); - for (int _i410 = 0; _i410 < _list409.size; ++_i410) + org.apache.thrift.protocol.TList _list457 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); + struct.success = new ArrayList(_list457.size); + for (int _i458 = 0; _i458 < _list457.size; ++_i458) { - String _elem411; // optional - _elem411 = iprot.readString(); - struct.success.add(_elem411); + String _elem459; // required + _elem459 = iprot.readString(); + struct.success.add(_elem459); } } struct.setSuccessIsSet(true); @@ -27008,13 +28022,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, get_all_tables_resu case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list412 = iprot.readListBegin(); - struct.success = new ArrayList(_list412.size); - for (int _i413 = 0; _i413 < _list412.size; ++_i413) + org.apache.thrift.protocol.TList _list460 = iprot.readListBegin(); + struct.success = new ArrayList(_list460.size); + for (int _i461 = 0; _i461 < _list460.size; ++_i461) { - String _elem414; // optional - _elem414 = iprot.readString(); - struct.success.add(_elem414); + String _elem462; // required + _elem462 = iprot.readString(); + struct.success.add(_elem462); } iprot.readListEnd(); } @@ -27049,9 +28063,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, get_all_tables_res oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size())); - for (String _iter415 : struct.success) + for (String _iter463 : struct.success) { - oprot.writeString(_iter415); + oprot.writeString(_iter463); } oprot.writeListEnd(); } @@ -27090,9 +28104,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, get_all_tables_resu if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (String _iter416 : struct.success) + for (String _iter464 : struct.success) { - oprot.writeString(_iter416); + oprot.writeString(_iter464); } } } @@ -27107,13 +28121,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, get_all_tables_resul BitSet incoming = iprot.readBitSet(2); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list417 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); - struct.success = new ArrayList(_list417.size); - for (int _i418 = 0; _i418 < _list417.size; ++_i418) + org.apache.thrift.protocol.TList _list465 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); + struct.success = new ArrayList(_list465.size); + for (int _i466 = 0; _i466 < _list465.size; ++_i466) { - String _elem419; // optional - _elem419 = iprot.readString(); - struct.success.add(_elem419); + String _elem467; // required + _elem467 = iprot.readString(); + struct.success.add(_elem467); } } struct.setSuccessIsSet(true); @@ -28569,13 +29583,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, get_table_objects_b case 2: // TBL_NAMES if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list420 = iprot.readListBegin(); - struct.tbl_names = new ArrayList(_list420.size); - for (int _i421 = 0; _i421 < _list420.size; ++_i421) + org.apache.thrift.protocol.TList _list468 = iprot.readListBegin(); + struct.tbl_names = new ArrayList(_list468.size); + for (int _i469 = 0; _i469 < _list468.size; ++_i469) { - String _elem422; // optional - _elem422 = iprot.readString(); - struct.tbl_names.add(_elem422); + String _elem470; // required + _elem470 = iprot.readString(); + struct.tbl_names.add(_elem470); } iprot.readListEnd(); } @@ -28606,9 +29620,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, get_table_objects_ oprot.writeFieldBegin(TBL_NAMES_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.tbl_names.size())); - for (String _iter423 : struct.tbl_names) + for (String _iter471 : struct.tbl_names) { - oprot.writeString(_iter423); + oprot.writeString(_iter471); } oprot.writeListEnd(); } @@ -28645,9 +29659,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, get_table_objects_b if (struct.isSetTbl_names()) { { oprot.writeI32(struct.tbl_names.size()); - for (String _iter424 : struct.tbl_names) + for (String _iter472 : struct.tbl_names) { - oprot.writeString(_iter424); + oprot.writeString(_iter472); } } } @@ -28663,13 +29677,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, get_table_objects_by } if (incoming.get(1)) { { - org.apache.thrift.protocol.TList _list425 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); - struct.tbl_names = new ArrayList(_list425.size); - for (int _i426 = 0; _i426 < _list425.size; ++_i426) + org.apache.thrift.protocol.TList _list473 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); + struct.tbl_names = new ArrayList(_list473.size); + for (int _i474 = 0; _i474 < _list473.size; ++_i474) { - String _elem427; // optional - _elem427 = iprot.readString(); - struct.tbl_names.add(_elem427); + String _elem475; // required + _elem475 = iprot.readString(); + struct.tbl_names.add(_elem475); } } struct.setTbl_namesIsSet(true); @@ -29237,14 +30251,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, get_table_objects_b case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list428 = iprot.readListBegin(); - struct.success = new ArrayList
(_list428.size); - for (int _i429 = 0; _i429 < _list428.size; ++_i429) + org.apache.thrift.protocol.TList _list476 = iprot.readListBegin(); + struct.success = new ArrayList
(_list476.size); + for (int _i477 = 0; _i477 < _list476.size; ++_i477) { - Table _elem430; // optional - _elem430 = new Table(); - _elem430.read(iprot); - struct.success.add(_elem430); + Table _elem478; // required + _elem478 = new Table(); + _elem478.read(iprot); + struct.success.add(_elem478); } iprot.readListEnd(); } @@ -29297,9 +30311,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, get_table_objects_ oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size())); - for (Table _iter431 : struct.success) + for (Table _iter479 : struct.success) { - _iter431.write(oprot); + _iter479.write(oprot); } oprot.writeListEnd(); } @@ -29354,9 +30368,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, get_table_objects_b if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (Table _iter432 : struct.success) + for (Table _iter480 : struct.success) { - _iter432.write(oprot); + _iter480.write(oprot); } } } @@ -29377,14 +30391,14 @@ public void read(org.apache.thrift.protocol.TProtocol prot, get_table_objects_by BitSet incoming = iprot.readBitSet(4); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list433 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); - struct.success = new ArrayList
(_list433.size); - for (int _i434 = 0; _i434 < _list433.size; ++_i434) + org.apache.thrift.protocol.TList _list481 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); + struct.success = new ArrayList
(_list481.size); + for (int _i482 = 0; _i482 < _list481.size; ++_i482) { - Table _elem435; // optional - _elem435 = new Table(); - _elem435.read(iprot); - struct.success.add(_elem435); + Table _elem483; // required + _elem483 = new Table(); + _elem483.read(iprot); + struct.success.add(_elem483); } } struct.setSuccessIsSet(true); @@ -30533,13 +31547,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, get_table_names_by_ case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list436 = iprot.readListBegin(); - struct.success = new ArrayList(_list436.size); - for (int _i437 = 0; _i437 < _list436.size; ++_i437) + org.apache.thrift.protocol.TList _list484 = iprot.readListBegin(); + struct.success = new ArrayList(_list484.size); + for (int _i485 = 0; _i485 < _list484.size; ++_i485) { - String _elem438; // optional - _elem438 = iprot.readString(); - struct.success.add(_elem438); + String _elem486; // required + _elem486 = iprot.readString(); + struct.success.add(_elem486); } iprot.readListEnd(); } @@ -30592,9 +31606,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, get_table_names_by oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size())); - for (String _iter439 : struct.success) + for (String _iter487 : struct.success) { - oprot.writeString(_iter439); + oprot.writeString(_iter487); } oprot.writeListEnd(); } @@ -30649,9 +31663,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, get_table_names_by_ if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (String _iter440 : struct.success) + for (String _iter488 : struct.success) { - oprot.writeString(_iter440); + oprot.writeString(_iter488); } } } @@ -30672,13 +31686,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, get_table_names_by_f BitSet incoming = iprot.readBitSet(4); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list441 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); - struct.success = new ArrayList(_list441.size); - for (int _i442 = 0; _i442 < _list441.size; ++_i442) + org.apache.thrift.protocol.TList _list489 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); + struct.success = new ArrayList(_list489.size); + for (int _i490 = 0; _i490 < _list489.size; ++_i490) { - String _elem443; // optional - _elem443 = iprot.readString(); - struct.success.add(_elem443); + String _elem491; // required + _elem491 = iprot.readString(); + struct.success.add(_elem491); } } struct.setSuccessIsSet(true); @@ -35398,14 +36412,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, add_partitions_args case 1: // NEW_PARTS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list444 = iprot.readListBegin(); - struct.new_parts = new ArrayList(_list444.size); - for (int _i445 = 0; _i445 < _list444.size; ++_i445) + org.apache.thrift.protocol.TList _list492 = iprot.readListBegin(); + struct.new_parts = new ArrayList(_list492.size); + for (int _i493 = 0; _i493 < _list492.size; ++_i493) { - Partition _elem446; // optional - _elem446 = new Partition(); - _elem446.read(iprot); - struct.new_parts.add(_elem446); + Partition _elem494; // required + _elem494 = new Partition(); + _elem494.read(iprot); + struct.new_parts.add(_elem494); } iprot.readListEnd(); } @@ -35431,9 +36445,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, add_partitions_arg oprot.writeFieldBegin(NEW_PARTS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.new_parts.size())); - for (Partition _iter447 : struct.new_parts) + for (Partition _iter495 : struct.new_parts) { - _iter447.write(oprot); + _iter495.write(oprot); } oprot.writeListEnd(); } @@ -35464,9 +36478,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, add_partitions_args if (struct.isSetNew_parts()) { { oprot.writeI32(struct.new_parts.size()); - for (Partition _iter448 : struct.new_parts) + for (Partition _iter496 : struct.new_parts) { - _iter448.write(oprot); + _iter496.write(oprot); } } } @@ -35478,14 +36492,14 @@ public void read(org.apache.thrift.protocol.TProtocol prot, add_partitions_args BitSet incoming = iprot.readBitSet(1); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list449 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); - struct.new_parts = new ArrayList(_list449.size); - for (int _i450 = 0; _i450 < _list449.size; ++_i450) + org.apache.thrift.protocol.TList _list497 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); + struct.new_parts = new ArrayList(_list497.size); + for (int _i498 = 0; _i498 < _list497.size; ++_i498) { - Partition _elem451; // optional - _elem451 = new Partition(); - _elem451.read(iprot); - struct.new_parts.add(_elem451); + Partition _elem499; // required + _elem499 = new Partition(); + _elem499.read(iprot); + struct.new_parts.add(_elem499); } } struct.setNew_partsIsSet(true); @@ -36664,13 +37678,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, append_partition_ar case 3: // PART_VALS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list452 = iprot.readListBegin(); - struct.part_vals = new ArrayList(_list452.size); - for (int _i453 = 0; _i453 < _list452.size; ++_i453) + org.apache.thrift.protocol.TList _list500 = iprot.readListBegin(); + struct.part_vals = new ArrayList(_list500.size); + for (int _i501 = 0; _i501 < _list500.size; ++_i501) { - String _elem454; // optional - _elem454 = iprot.readString(); - struct.part_vals.add(_elem454); + String _elem502; // required + _elem502 = iprot.readString(); + struct.part_vals.add(_elem502); } iprot.readListEnd(); } @@ -36706,9 +37720,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, append_partition_a oprot.writeFieldBegin(PART_VALS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.part_vals.size())); - for (String _iter455 : struct.part_vals) + for (String _iter503 : struct.part_vals) { - oprot.writeString(_iter455); + oprot.writeString(_iter503); } oprot.writeListEnd(); } @@ -36751,9 +37765,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, append_partition_ar if (struct.isSetPart_vals()) { { oprot.writeI32(struct.part_vals.size()); - for (String _iter456 : struct.part_vals) + for (String _iter504 : struct.part_vals) { - oprot.writeString(_iter456); + oprot.writeString(_iter504); } } } @@ -36773,13 +37787,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, append_partition_arg } if (incoming.get(2)) { { - org.apache.thrift.protocol.TList _list457 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); - struct.part_vals = new ArrayList(_list457.size); - for (int _i458 = 0; _i458 < _list457.size; ++_i458) + org.apache.thrift.protocol.TList _list505 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); + struct.part_vals = new ArrayList(_list505.size); + for (int _i506 = 0; _i506 < _list505.size; ++_i506) { - String _elem459; // optional - _elem459 = iprot.readString(); - struct.part_vals.add(_elem459); + String _elem507; // required + _elem507 = iprot.readString(); + struct.part_vals.add(_elem507); } } struct.setPart_valsIsSet(true); @@ -39091,13 +40105,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, append_partition_wi case 3: // PART_VALS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list460 = iprot.readListBegin(); - struct.part_vals = new ArrayList(_list460.size); - for (int _i461 = 0; _i461 < _list460.size; ++_i461) + org.apache.thrift.protocol.TList _list508 = iprot.readListBegin(); + struct.part_vals = new ArrayList(_list508.size); + for (int _i509 = 0; _i509 < _list508.size; ++_i509) { - String _elem462; // optional - _elem462 = iprot.readString(); - struct.part_vals.add(_elem462); + String _elem510; // required + _elem510 = iprot.readString(); + struct.part_vals.add(_elem510); } iprot.readListEnd(); } @@ -39142,9 +40156,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, append_partition_w oprot.writeFieldBegin(PART_VALS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.part_vals.size())); - for (String _iter463 : struct.part_vals) + for (String _iter511 : struct.part_vals) { - oprot.writeString(_iter463); + oprot.writeString(_iter511); } oprot.writeListEnd(); } @@ -39195,9 +40209,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, append_partition_wi if (struct.isSetPart_vals()) { { oprot.writeI32(struct.part_vals.size()); - for (String _iter464 : struct.part_vals) + for (String _iter512 : struct.part_vals) { - oprot.writeString(_iter464); + oprot.writeString(_iter512); } } } @@ -39220,13 +40234,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, append_partition_wit } if (incoming.get(2)) { { - org.apache.thrift.protocol.TList _list465 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); - struct.part_vals = new ArrayList(_list465.size); - for (int _i466 = 0; _i466 < _list465.size; ++_i466) + org.apache.thrift.protocol.TList _list513 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); + struct.part_vals = new ArrayList(_list513.size); + for (int _i514 = 0; _i514 < _list513.size; ++_i514) { - String _elem467; // optional - _elem467 = iprot.readString(); - struct.part_vals.add(_elem467); + String _elem515; // required + _elem515 = iprot.readString(); + struct.part_vals.add(_elem515); } } struct.setPart_valsIsSet(true); @@ -43099,13 +44113,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, drop_partition_args case 3: // PART_VALS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list468 = iprot.readListBegin(); - struct.part_vals = new ArrayList(_list468.size); - for (int _i469 = 0; _i469 < _list468.size; ++_i469) + org.apache.thrift.protocol.TList _list516 = iprot.readListBegin(); + struct.part_vals = new ArrayList(_list516.size); + for (int _i517 = 0; _i517 < _list516.size; ++_i517) { - String _elem470; // optional - _elem470 = iprot.readString(); - struct.part_vals.add(_elem470); + String _elem518; // required + _elem518 = iprot.readString(); + struct.part_vals.add(_elem518); } iprot.readListEnd(); } @@ -43149,9 +44163,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, drop_partition_arg oprot.writeFieldBegin(PART_VALS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.part_vals.size())); - for (String _iter471 : struct.part_vals) + for (String _iter519 : struct.part_vals) { - oprot.writeString(_iter471); + oprot.writeString(_iter519); } oprot.writeListEnd(); } @@ -43200,9 +44214,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, drop_partition_args if (struct.isSetPart_vals()) { { oprot.writeI32(struct.part_vals.size()); - for (String _iter472 : struct.part_vals) + for (String _iter520 : struct.part_vals) { - oprot.writeString(_iter472); + oprot.writeString(_iter520); } } } @@ -43225,13 +44239,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, drop_partition_args } if (incoming.get(2)) { { - org.apache.thrift.protocol.TList _list473 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); - struct.part_vals = new ArrayList(_list473.size); - for (int _i474 = 0; _i474 < _list473.size; ++_i474) + org.apache.thrift.protocol.TList _list521 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); + struct.part_vals = new ArrayList(_list521.size); + for (int _i522 = 0; _i522 < _list521.size; ++_i522) { - String _elem475; // optional - _elem475 = iprot.readString(); - struct.part_vals.add(_elem475); + String _elem523; // required + _elem523 = iprot.readString(); + struct.part_vals.add(_elem523); } } struct.setPart_valsIsSet(true); @@ -44473,13 +45487,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, drop_partition_with case 3: // PART_VALS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list476 = iprot.readListBegin(); - struct.part_vals = new ArrayList(_list476.size); - for (int _i477 = 0; _i477 < _list476.size; ++_i477) + org.apache.thrift.protocol.TList _list524 = iprot.readListBegin(); + struct.part_vals = new ArrayList(_list524.size); + for (int _i525 = 0; _i525 < _list524.size; ++_i525) { - String _elem478; // optional - _elem478 = iprot.readString(); - struct.part_vals.add(_elem478); + String _elem526; // required + _elem526 = iprot.readString(); + struct.part_vals.add(_elem526); } iprot.readListEnd(); } @@ -44532,9 +45546,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, drop_partition_wit oprot.writeFieldBegin(PART_VALS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.part_vals.size())); - for (String _iter479 : struct.part_vals) + for (String _iter527 : struct.part_vals) { - oprot.writeString(_iter479); + oprot.writeString(_iter527); } oprot.writeListEnd(); } @@ -44591,9 +45605,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, drop_partition_with if (struct.isSetPart_vals()) { { oprot.writeI32(struct.part_vals.size()); - for (String _iter480 : struct.part_vals) + for (String _iter528 : struct.part_vals) { - oprot.writeString(_iter480); + oprot.writeString(_iter528); } } } @@ -44619,13 +45633,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, drop_partition_with_ } if (incoming.get(2)) { { - org.apache.thrift.protocol.TList _list481 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); - struct.part_vals = new ArrayList(_list481.size); - for (int _i482 = 0; _i482 < _list481.size; ++_i482) + org.apache.thrift.protocol.TList _list529 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); + struct.part_vals = new ArrayList(_list529.size); + for (int _i530 = 0; _i530 < _list529.size; ++_i530) { - String _elem483; // optional - _elem483 = iprot.readString(); - struct.part_vals.add(_elem483); + String _elem531; // required + _elem531 = iprot.readString(); + struct.part_vals.add(_elem531); } } struct.setPart_valsIsSet(true); @@ -49230,13 +50244,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, get_partition_args case 3: // PART_VALS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list484 = iprot.readListBegin(); - struct.part_vals = new ArrayList(_list484.size); - for (int _i485 = 0; _i485 < _list484.size; ++_i485) + org.apache.thrift.protocol.TList _list532 = iprot.readListBegin(); + struct.part_vals = new ArrayList(_list532.size); + for (int _i533 = 0; _i533 < _list532.size; ++_i533) { - String _elem486; // optional - _elem486 = iprot.readString(); - struct.part_vals.add(_elem486); + String _elem534; // required + _elem534 = iprot.readString(); + struct.part_vals.add(_elem534); } iprot.readListEnd(); } @@ -49272,9 +50286,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, get_partition_args oprot.writeFieldBegin(PART_VALS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.part_vals.size())); - for (String _iter487 : struct.part_vals) + for (String _iter535 : struct.part_vals) { - oprot.writeString(_iter487); + oprot.writeString(_iter535); } oprot.writeListEnd(); } @@ -49317,9 +50331,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, get_partition_args if (struct.isSetPart_vals()) { { oprot.writeI32(struct.part_vals.size()); - for (String _iter488 : struct.part_vals) + for (String _iter536 : struct.part_vals) { - oprot.writeString(_iter488); + oprot.writeString(_iter536); } } } @@ -49339,13 +50353,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, get_partition_args s } if (incoming.get(2)) { { - org.apache.thrift.protocol.TList _list489 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); - struct.part_vals = new ArrayList(_list489.size); - for (int _i490 = 0; _i490 < _list489.size; ++_i490) + org.apache.thrift.protocol.TList _list537 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); + struct.part_vals = new ArrayList(_list537.size); + for (int _i538 = 0; _i538 < _list537.size; ++_i538) { - String _elem491; // optional - _elem491 = iprot.readString(); - struct.part_vals.add(_elem491); + String _elem539; // required + _elem539 = iprot.readString(); + struct.part_vals.add(_elem539); } } struct.setPart_valsIsSet(true); @@ -50574,15 +51588,15 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, exchange_partition_ case 1: // PARTITION_SPECS if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { { - org.apache.thrift.protocol.TMap _map492 = iprot.readMapBegin(); - struct.partitionSpecs = new HashMap(2*_map492.size); - for (int _i493 = 0; _i493 < _map492.size; ++_i493) + org.apache.thrift.protocol.TMap _map540 = iprot.readMapBegin(); + struct.partitionSpecs = new HashMap(2*_map540.size); + for (int _i541 = 0; _i541 < _map540.size; ++_i541) { - String _key494; // required - String _val495; // required - _key494 = iprot.readString(); - _val495 = iprot.readString(); - struct.partitionSpecs.put(_key494, _val495); + String _key542; // required + String _val543; // required + _key542 = iprot.readString(); + _val543 = iprot.readString(); + struct.partitionSpecs.put(_key542, _val543); } iprot.readMapEnd(); } @@ -50640,10 +51654,10 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, exchange_partition oprot.writeFieldBegin(PARTITION_SPECS_FIELD_DESC); { oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.partitionSpecs.size())); - for (Map.Entry _iter496 : struct.partitionSpecs.entrySet()) + for (Map.Entry _iter544 : struct.partitionSpecs.entrySet()) { - oprot.writeString(_iter496.getKey()); - oprot.writeString(_iter496.getValue()); + oprot.writeString(_iter544.getKey()); + oprot.writeString(_iter544.getValue()); } oprot.writeMapEnd(); } @@ -50706,10 +51720,10 @@ public void write(org.apache.thrift.protocol.TProtocol prot, exchange_partition_ if (struct.isSetPartitionSpecs()) { { oprot.writeI32(struct.partitionSpecs.size()); - for (Map.Entry _iter497 : struct.partitionSpecs.entrySet()) + for (Map.Entry _iter545 : struct.partitionSpecs.entrySet()) { - oprot.writeString(_iter497.getKey()); - oprot.writeString(_iter497.getValue()); + oprot.writeString(_iter545.getKey()); + oprot.writeString(_iter545.getValue()); } } } @@ -50733,15 +51747,15 @@ public void read(org.apache.thrift.protocol.TProtocol prot, exchange_partition_a BitSet incoming = iprot.readBitSet(5); if (incoming.get(0)) { { - org.apache.thrift.protocol.TMap _map498 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32()); - struct.partitionSpecs = new HashMap(2*_map498.size); - for (int _i499 = 0; _i499 < _map498.size; ++_i499) + org.apache.thrift.protocol.TMap _map546 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32()); + struct.partitionSpecs = new HashMap(2*_map546.size); + for (int _i547 = 0; _i547 < _map546.size; ++_i547) { - String _key500; // required - String _val501; // required - _key500 = iprot.readString(); - _val501 = iprot.readString(); - struct.partitionSpecs.put(_key500, _val501); + String _key548; // required + String _val549; // required + _key548 = iprot.readString(); + _val549 = iprot.readString(); + struct.partitionSpecs.put(_key548, _val549); } } struct.setPartitionSpecsIsSet(true); @@ -52229,13 +53243,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, get_partition_with_ case 3: // PART_VALS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list502 = iprot.readListBegin(); - struct.part_vals = new ArrayList(_list502.size); - for (int _i503 = 0; _i503 < _list502.size; ++_i503) + org.apache.thrift.protocol.TList _list550 = iprot.readListBegin(); + struct.part_vals = new ArrayList(_list550.size); + for (int _i551 = 0; _i551 < _list550.size; ++_i551) { - String _elem504; // optional - _elem504 = iprot.readString(); - struct.part_vals.add(_elem504); + String _elem552; // required + _elem552 = iprot.readString(); + struct.part_vals.add(_elem552); } iprot.readListEnd(); } @@ -52255,13 +53269,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, get_partition_with_ case 5: // GROUP_NAMES if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list505 = iprot.readListBegin(); - struct.group_names = new ArrayList(_list505.size); - for (int _i506 = 0; _i506 < _list505.size; ++_i506) + org.apache.thrift.protocol.TList _list553 = iprot.readListBegin(); + struct.group_names = new ArrayList(_list553.size); + for (int _i554 = 0; _i554 < _list553.size; ++_i554) { - String _elem507; // optional - _elem507 = iprot.readString(); - struct.group_names.add(_elem507); + String _elem555; // required + _elem555 = iprot.readString(); + struct.group_names.add(_elem555); } iprot.readListEnd(); } @@ -52297,9 +53311,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, get_partition_with oprot.writeFieldBegin(PART_VALS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.part_vals.size())); - for (String _iter508 : struct.part_vals) + for (String _iter556 : struct.part_vals) { - oprot.writeString(_iter508); + oprot.writeString(_iter556); } oprot.writeListEnd(); } @@ -52314,9 +53328,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, get_partition_with oprot.writeFieldBegin(GROUP_NAMES_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.group_names.size())); - for (String _iter509 : struct.group_names) + for (String _iter557 : struct.group_names) { - oprot.writeString(_iter509); + oprot.writeString(_iter557); } oprot.writeListEnd(); } @@ -52365,9 +53379,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, get_partition_with_ if (struct.isSetPart_vals()) { { oprot.writeI32(struct.part_vals.size()); - for (String _iter510 : struct.part_vals) + for (String _iter558 : struct.part_vals) { - oprot.writeString(_iter510); + oprot.writeString(_iter558); } } } @@ -52377,9 +53391,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, get_partition_with_ if (struct.isSetGroup_names()) { { oprot.writeI32(struct.group_names.size()); - for (String _iter511 : struct.group_names) + for (String _iter559 : struct.group_names) { - oprot.writeString(_iter511); + oprot.writeString(_iter559); } } } @@ -52399,13 +53413,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, get_partition_with_a } if (incoming.get(2)) { { - org.apache.thrift.protocol.TList _list512 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); - struct.part_vals = new ArrayList(_list512.size); - for (int _i513 = 0; _i513 < _list512.size; ++_i513) + org.apache.thrift.protocol.TList _list560 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); + struct.part_vals = new ArrayList(_list560.size); + for (int _i561 = 0; _i561 < _list560.size; ++_i561) { - String _elem514; // optional - _elem514 = iprot.readString(); - struct.part_vals.add(_elem514); + String _elem562; // required + _elem562 = iprot.readString(); + struct.part_vals.add(_elem562); } } struct.setPart_valsIsSet(true); @@ -52416,13 +53430,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, get_partition_with_a } if (incoming.get(4)) { { - org.apache.thrift.protocol.TList _list515 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); - struct.group_names = new ArrayList(_list515.size); - for (int _i516 = 0; _i516 < _list515.size; ++_i516) + org.apache.thrift.protocol.TList _list563 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); + struct.group_names = new ArrayList(_list563.size); + for (int _i564 = 0; _i564 < _list563.size; ++_i564) { - String _elem517; // optional - _elem517 = iprot.readString(); - struct.group_names.add(_elem517); + String _elem565; // required + _elem565 = iprot.readString(); + struct.group_names.add(_elem565); } } struct.setGroup_namesIsSet(true); @@ -55191,14 +56205,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, get_partitions_resu case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list518 = iprot.readListBegin(); - struct.success = new ArrayList(_list518.size); - for (int _i519 = 0; _i519 < _list518.size; ++_i519) + org.apache.thrift.protocol.TList _list566 = iprot.readListBegin(); + struct.success = new ArrayList(_list566.size); + for (int _i567 = 0; _i567 < _list566.size; ++_i567) { - Partition _elem520; // optional - _elem520 = new Partition(); - _elem520.read(iprot); - struct.success.add(_elem520); + Partition _elem568; // required + _elem568 = new Partition(); + _elem568.read(iprot); + struct.success.add(_elem568); } iprot.readListEnd(); } @@ -55242,9 +56256,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, get_partitions_res oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size())); - for (Partition _iter521 : struct.success) + for (Partition _iter569 : struct.success) { - _iter521.write(oprot); + _iter569.write(oprot); } oprot.writeListEnd(); } @@ -55291,9 +56305,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, get_partitions_resu if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (Partition _iter522 : struct.success) + for (Partition _iter570 : struct.success) { - _iter522.write(oprot); + _iter570.write(oprot); } } } @@ -55311,14 +56325,14 @@ public void read(org.apache.thrift.protocol.TProtocol prot, get_partitions_resul BitSet incoming = iprot.readBitSet(3); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list523 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); - struct.success = new ArrayList(_list523.size); - for (int _i524 = 0; _i524 < _list523.size; ++_i524) + org.apache.thrift.protocol.TList _list571 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); + struct.success = new ArrayList(_list571.size); + for (int _i572 = 0; _i572 < _list571.size; ++_i572) { - Partition _elem525; // optional - _elem525 = new Partition(); - _elem525.read(iprot); - struct.success.add(_elem525); + Partition _elem573; // required + _elem573 = new Partition(); + _elem573.read(iprot); + struct.success.add(_elem573); } } struct.setSuccessIsSet(true); @@ -56011,13 +57025,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, get_partitions_with case 5: // GROUP_NAMES if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list526 = iprot.readListBegin(); - struct.group_names = new ArrayList(_list526.size); - for (int _i527 = 0; _i527 < _list526.size; ++_i527) + org.apache.thrift.protocol.TList _list574 = iprot.readListBegin(); + struct.group_names = new ArrayList(_list574.size); + for (int _i575 = 0; _i575 < _list574.size; ++_i575) { - String _elem528; // optional - _elem528 = iprot.readString(); - struct.group_names.add(_elem528); + String _elem576; // required + _elem576 = iprot.readString(); + struct.group_names.add(_elem576); } iprot.readListEnd(); } @@ -56061,9 +57075,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, get_partitions_wit oprot.writeFieldBegin(GROUP_NAMES_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.group_names.size())); - for (String _iter529 : struct.group_names) + for (String _iter577 : struct.group_names) { - oprot.writeString(_iter529); + oprot.writeString(_iter577); } oprot.writeListEnd(); } @@ -56118,9 +57132,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, get_partitions_with if (struct.isSetGroup_names()) { { oprot.writeI32(struct.group_names.size()); - for (String _iter530 : struct.group_names) + for (String _iter578 : struct.group_names) { - oprot.writeString(_iter530); + oprot.writeString(_iter578); } } } @@ -56148,13 +57162,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, get_partitions_with_ } if (incoming.get(4)) { { - org.apache.thrift.protocol.TList _list531 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); - struct.group_names = new ArrayList(_list531.size); - for (int _i532 = 0; _i532 < _list531.size; ++_i532) + org.apache.thrift.protocol.TList _list579 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); + struct.group_names = new ArrayList(_list579.size); + for (int _i580 = 0; _i580 < _list579.size; ++_i580) { - String _elem533; // optional - _elem533 = iprot.readString(); - struct.group_names.add(_elem533); + String _elem581; // required + _elem581 = iprot.readString(); + struct.group_names.add(_elem581); } } struct.setGroup_namesIsSet(true); @@ -56641,14 +57655,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, get_partitions_with case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list534 = iprot.readListBegin(); - struct.success = new ArrayList(_list534.size); - for (int _i535 = 0; _i535 < _list534.size; ++_i535) + org.apache.thrift.protocol.TList _list582 = iprot.readListBegin(); + struct.success = new ArrayList(_list582.size); + for (int _i583 = 0; _i583 < _list582.size; ++_i583) { - Partition _elem536; // optional - _elem536 = new Partition(); - _elem536.read(iprot); - struct.success.add(_elem536); + Partition _elem584; // required + _elem584 = new Partition(); + _elem584.read(iprot); + struct.success.add(_elem584); } iprot.readListEnd(); } @@ -56692,9 +57706,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, get_partitions_wit oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size())); - for (Partition _iter537 : struct.success) + for (Partition _iter585 : struct.success) { - _iter537.write(oprot); + _iter585.write(oprot); } oprot.writeListEnd(); } @@ -56741,9 +57755,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, get_partitions_with if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (Partition _iter538 : struct.success) + for (Partition _iter586 : struct.success) { - _iter538.write(oprot); + _iter586.write(oprot); } } } @@ -56761,14 +57775,14 @@ public void read(org.apache.thrift.protocol.TProtocol prot, get_partitions_with_ BitSet incoming = iprot.readBitSet(3); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list539 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); - struct.success = new ArrayList(_list539.size); - for (int _i540 = 0; _i540 < _list539.size; ++_i540) + org.apache.thrift.protocol.TList _list587 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); + struct.success = new ArrayList(_list587.size); + for (int _i588 = 0; _i588 < _list587.size; ++_i588) { - Partition _elem541; // optional - _elem541 = new Partition(); - _elem541.read(iprot); - struct.success.add(_elem541); + Partition _elem589; // required + _elem589 = new Partition(); + _elem589.read(iprot); + struct.success.add(_elem589); } } struct.setSuccessIsSet(true); @@ -57750,13 +58764,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, get_partition_names case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list542 = iprot.readListBegin(); - struct.success = new ArrayList(_list542.size); - for (int _i543 = 0; _i543 < _list542.size; ++_i543) + org.apache.thrift.protocol.TList _list590 = iprot.readListBegin(); + struct.success = new ArrayList(_list590.size); + for (int _i591 = 0; _i591 < _list590.size; ++_i591) { - String _elem544; // optional - _elem544 = iprot.readString(); - struct.success.add(_elem544); + String _elem592; // required + _elem592 = iprot.readString(); + struct.success.add(_elem592); } iprot.readListEnd(); } @@ -57791,9 +58805,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, get_partition_name oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size())); - for (String _iter545 : struct.success) + for (String _iter593 : struct.success) { - oprot.writeString(_iter545); + oprot.writeString(_iter593); } oprot.writeListEnd(); } @@ -57832,9 +58846,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, get_partition_names if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (String _iter546 : struct.success) + for (String _iter594 : struct.success) { - oprot.writeString(_iter546); + oprot.writeString(_iter594); } } } @@ -57849,13 +58863,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, get_partition_names_ BitSet incoming = iprot.readBitSet(2); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list547 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); - struct.success = new ArrayList(_list547.size); - for (int _i548 = 0; _i548 < _list547.size; ++_i548) + org.apache.thrift.protocol.TList _list595 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); + struct.success = new ArrayList(_list595.size); + for (int _i596 = 0; _i596 < _list595.size; ++_i596) { - String _elem549; // optional - _elem549 = iprot.readString(); - struct.success.add(_elem549); + String _elem597; // required + _elem597 = iprot.readString(); + struct.success.add(_elem597); } } struct.setSuccessIsSet(true); @@ -58446,13 +59460,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, get_partitions_ps_a case 3: // PART_VALS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list550 = iprot.readListBegin(); - struct.part_vals = new ArrayList(_list550.size); - for (int _i551 = 0; _i551 < _list550.size; ++_i551) + org.apache.thrift.protocol.TList _list598 = iprot.readListBegin(); + struct.part_vals = new ArrayList(_list598.size); + for (int _i599 = 0; _i599 < _list598.size; ++_i599) { - String _elem552; // optional - _elem552 = iprot.readString(); - struct.part_vals.add(_elem552); + String _elem600; // required + _elem600 = iprot.readString(); + struct.part_vals.add(_elem600); } iprot.readListEnd(); } @@ -58496,9 +59510,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, get_partitions_ps_ oprot.writeFieldBegin(PART_VALS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.part_vals.size())); - for (String _iter553 : struct.part_vals) + for (String _iter601 : struct.part_vals) { - oprot.writeString(_iter553); + oprot.writeString(_iter601); } oprot.writeListEnd(); } @@ -58547,9 +59561,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, get_partitions_ps_a if (struct.isSetPart_vals()) { { oprot.writeI32(struct.part_vals.size()); - for (String _iter554 : struct.part_vals) + for (String _iter602 : struct.part_vals) { - oprot.writeString(_iter554); + oprot.writeString(_iter602); } } } @@ -58572,13 +59586,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, get_partitions_ps_ar } if (incoming.get(2)) { { - org.apache.thrift.protocol.TList _list555 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); - struct.part_vals = new ArrayList(_list555.size); - for (int _i556 = 0; _i556 < _list555.size; ++_i556) + org.apache.thrift.protocol.TList _list603 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); + struct.part_vals = new ArrayList(_list603.size); + for (int _i604 = 0; _i604 < _list603.size; ++_i604) { - String _elem557; // optional - _elem557 = iprot.readString(); - struct.part_vals.add(_elem557); + String _elem605; // required + _elem605 = iprot.readString(); + struct.part_vals.add(_elem605); } } struct.setPart_valsIsSet(true); @@ -59069,14 +60083,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, get_partitions_ps_r case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list558 = iprot.readListBegin(); - struct.success = new ArrayList(_list558.size); - for (int _i559 = 0; _i559 < _list558.size; ++_i559) + org.apache.thrift.protocol.TList _list606 = iprot.readListBegin(); + struct.success = new ArrayList(_list606.size); + for (int _i607 = 0; _i607 < _list606.size; ++_i607) { - Partition _elem560; // optional - _elem560 = new Partition(); - _elem560.read(iprot); - struct.success.add(_elem560); + Partition _elem608; // required + _elem608 = new Partition(); + _elem608.read(iprot); + struct.success.add(_elem608); } iprot.readListEnd(); } @@ -59120,9 +60134,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, get_partitions_ps_ oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size())); - for (Partition _iter561 : struct.success) + for (Partition _iter609 : struct.success) { - _iter561.write(oprot); + _iter609.write(oprot); } oprot.writeListEnd(); } @@ -59169,9 +60183,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, get_partitions_ps_r if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (Partition _iter562 : struct.success) + for (Partition _iter610 : struct.success) { - _iter562.write(oprot); + _iter610.write(oprot); } } } @@ -59189,14 +60203,14 @@ public void read(org.apache.thrift.protocol.TProtocol prot, get_partitions_ps_re BitSet incoming = iprot.readBitSet(3); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list563 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); - struct.success = new ArrayList(_list563.size); - for (int _i564 = 0; _i564 < _list563.size; ++_i564) + org.apache.thrift.protocol.TList _list611 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); + struct.success = new ArrayList(_list611.size); + for (int _i612 = 0; _i612 < _list611.size; ++_i612) { - Partition _elem565; // optional - _elem565 = new Partition(); - _elem565.read(iprot); - struct.success.add(_elem565); + Partition _elem613; // required + _elem613 = new Partition(); + _elem613.read(iprot); + struct.success.add(_elem613); } } struct.setSuccessIsSet(true); @@ -59974,13 +60988,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, get_partitions_ps_w case 3: // PART_VALS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list566 = iprot.readListBegin(); - struct.part_vals = new ArrayList(_list566.size); - for (int _i567 = 0; _i567 < _list566.size; ++_i567) + org.apache.thrift.protocol.TList _list614 = iprot.readListBegin(); + struct.part_vals = new ArrayList(_list614.size); + for (int _i615 = 0; _i615 < _list614.size; ++_i615) { - String _elem568; // optional - _elem568 = iprot.readString(); - struct.part_vals.add(_elem568); + String _elem616; // required + _elem616 = iprot.readString(); + struct.part_vals.add(_elem616); } iprot.readListEnd(); } @@ -60008,13 +61022,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, get_partitions_ps_w case 6: // GROUP_NAMES if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list569 = iprot.readListBegin(); - struct.group_names = new ArrayList(_list569.size); - for (int _i570 = 0; _i570 < _list569.size; ++_i570) + org.apache.thrift.protocol.TList _list617 = iprot.readListBegin(); + struct.group_names = new ArrayList(_list617.size); + for (int _i618 = 0; _i618 < _list617.size; ++_i618) { - String _elem571; // optional - _elem571 = iprot.readString(); - struct.group_names.add(_elem571); + String _elem619; // required + _elem619 = iprot.readString(); + struct.group_names.add(_elem619); } iprot.readListEnd(); } @@ -60050,9 +61064,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, get_partitions_ps_ oprot.writeFieldBegin(PART_VALS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.part_vals.size())); - for (String _iter572 : struct.part_vals) + for (String _iter620 : struct.part_vals) { - oprot.writeString(_iter572); + oprot.writeString(_iter620); } oprot.writeListEnd(); } @@ -60070,9 +61084,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, get_partitions_ps_ oprot.writeFieldBegin(GROUP_NAMES_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.group_names.size())); - for (String _iter573 : struct.group_names) + for (String _iter621 : struct.group_names) { - oprot.writeString(_iter573); + oprot.writeString(_iter621); } oprot.writeListEnd(); } @@ -60124,9 +61138,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, get_partitions_ps_w if (struct.isSetPart_vals()) { { oprot.writeI32(struct.part_vals.size()); - for (String _iter574 : struct.part_vals) + for (String _iter622 : struct.part_vals) { - oprot.writeString(_iter574); + oprot.writeString(_iter622); } } } @@ -60139,9 +61153,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, get_partitions_ps_w if (struct.isSetGroup_names()) { { oprot.writeI32(struct.group_names.size()); - for (String _iter575 : struct.group_names) + for (String _iter623 : struct.group_names) { - oprot.writeString(_iter575); + oprot.writeString(_iter623); } } } @@ -60161,13 +61175,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, get_partitions_ps_wi } if (incoming.get(2)) { { - org.apache.thrift.protocol.TList _list576 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); - struct.part_vals = new ArrayList(_list576.size); - for (int _i577 = 0; _i577 < _list576.size; ++_i577) + org.apache.thrift.protocol.TList _list624 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); + struct.part_vals = new ArrayList(_list624.size); + for (int _i625 = 0; _i625 < _list624.size; ++_i625) { - String _elem578; // optional - _elem578 = iprot.readString(); - struct.part_vals.add(_elem578); + String _elem626; // required + _elem626 = iprot.readString(); + struct.part_vals.add(_elem626); } } struct.setPart_valsIsSet(true); @@ -60182,13 +61196,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, get_partitions_ps_wi } if (incoming.get(5)) { { - org.apache.thrift.protocol.TList _list579 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); - struct.group_names = new ArrayList(_list579.size); - for (int _i580 = 0; _i580 < _list579.size; ++_i580) + org.apache.thrift.protocol.TList _list627 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); + struct.group_names = new ArrayList(_list627.size); + for (int _i628 = 0; _i628 < _list627.size; ++_i628) { - String _elem581; // optional - _elem581 = iprot.readString(); - struct.group_names.add(_elem581); + String _elem629; // required + _elem629 = iprot.readString(); + struct.group_names.add(_elem629); } } struct.setGroup_namesIsSet(true); @@ -60675,14 +61689,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, get_partitions_ps_w case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list582 = iprot.readListBegin(); - struct.success = new ArrayList(_list582.size); - for (int _i583 = 0; _i583 < _list582.size; ++_i583) + org.apache.thrift.protocol.TList _list630 = iprot.readListBegin(); + struct.success = new ArrayList(_list630.size); + for (int _i631 = 0; _i631 < _list630.size; ++_i631) { - Partition _elem584; // optional - _elem584 = new Partition(); - _elem584.read(iprot); - struct.success.add(_elem584); + Partition _elem632; // required + _elem632 = new Partition(); + _elem632.read(iprot); + struct.success.add(_elem632); } iprot.readListEnd(); } @@ -60726,9 +61740,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, get_partitions_ps_ oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size())); - for (Partition _iter585 : struct.success) + for (Partition _iter633 : struct.success) { - _iter585.write(oprot); + _iter633.write(oprot); } oprot.writeListEnd(); } @@ -60775,9 +61789,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, get_partitions_ps_w if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (Partition _iter586 : struct.success) + for (Partition _iter634 : struct.success) { - _iter586.write(oprot); + _iter634.write(oprot); } } } @@ -60795,14 +61809,14 @@ public void read(org.apache.thrift.protocol.TProtocol prot, get_partitions_ps_wi BitSet incoming = iprot.readBitSet(3); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list587 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); - struct.success = new ArrayList(_list587.size); - for (int _i588 = 0; _i588 < _list587.size; ++_i588) + org.apache.thrift.protocol.TList _list635 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); + struct.success = new ArrayList(_list635.size); + for (int _i636 = 0; _i636 < _list635.size; ++_i636) { - Partition _elem589; // optional - _elem589 = new Partition(); - _elem589.read(iprot); - struct.success.add(_elem589); + Partition _elem637; // required + _elem637 = new Partition(); + _elem637.read(iprot); + struct.success.add(_elem637); } } struct.setSuccessIsSet(true); @@ -61398,13 +62412,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, get_partition_names case 3: // PART_VALS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list590 = iprot.readListBegin(); - struct.part_vals = new ArrayList(_list590.size); - for (int _i591 = 0; _i591 < _list590.size; ++_i591) + org.apache.thrift.protocol.TList _list638 = iprot.readListBegin(); + struct.part_vals = new ArrayList(_list638.size); + for (int _i639 = 0; _i639 < _list638.size; ++_i639) { - String _elem592; // optional - _elem592 = iprot.readString(); - struct.part_vals.add(_elem592); + String _elem640; // required + _elem640 = iprot.readString(); + struct.part_vals.add(_elem640); } iprot.readListEnd(); } @@ -61448,9 +62462,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, get_partition_name oprot.writeFieldBegin(PART_VALS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.part_vals.size())); - for (String _iter593 : struct.part_vals) + for (String _iter641 : struct.part_vals) { - oprot.writeString(_iter593); + oprot.writeString(_iter641); } oprot.writeListEnd(); } @@ -61499,9 +62513,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, get_partition_names if (struct.isSetPart_vals()) { { oprot.writeI32(struct.part_vals.size()); - for (String _iter594 : struct.part_vals) + for (String _iter642 : struct.part_vals) { - oprot.writeString(_iter594); + oprot.writeString(_iter642); } } } @@ -61524,13 +62538,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, get_partition_names_ } if (incoming.get(2)) { { - org.apache.thrift.protocol.TList _list595 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); - struct.part_vals = new ArrayList(_list595.size); - for (int _i596 = 0; _i596 < _list595.size; ++_i596) + org.apache.thrift.protocol.TList _list643 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); + struct.part_vals = new ArrayList(_list643.size); + for (int _i644 = 0; _i644 < _list643.size; ++_i644) { - String _elem597; // optional - _elem597 = iprot.readString(); - struct.part_vals.add(_elem597); + String _elem645; // required + _elem645 = iprot.readString(); + struct.part_vals.add(_elem645); } } struct.setPart_valsIsSet(true); @@ -62021,13 +63035,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, get_partition_names case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list598 = iprot.readListBegin(); - struct.success = new ArrayList(_list598.size); - for (int _i599 = 0; _i599 < _list598.size; ++_i599) + org.apache.thrift.protocol.TList _list646 = iprot.readListBegin(); + struct.success = new ArrayList(_list646.size); + for (int _i647 = 0; _i647 < _list646.size; ++_i647) { - String _elem600; // optional - _elem600 = iprot.readString(); - struct.success.add(_elem600); + String _elem648; // required + _elem648 = iprot.readString(); + struct.success.add(_elem648); } iprot.readListEnd(); } @@ -62071,9 +63085,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, get_partition_name oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size())); - for (String _iter601 : struct.success) + for (String _iter649 : struct.success) { - oprot.writeString(_iter601); + oprot.writeString(_iter649); } oprot.writeListEnd(); } @@ -62120,9 +63134,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, get_partition_names if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (String _iter602 : struct.success) + for (String _iter650 : struct.success) { - oprot.writeString(_iter602); + oprot.writeString(_iter650); } } } @@ -62140,13 +63154,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, get_partition_names_ BitSet incoming = iprot.readBitSet(3); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list603 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); - struct.success = new ArrayList(_list603.size); - for (int _i604 = 0; _i604 < _list603.size; ++_i604) + org.apache.thrift.protocol.TList _list651 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); + struct.success = new ArrayList(_list651.size); + for (int _i652 = 0; _i652 < _list651.size; ++_i652) { - String _elem605; // optional - _elem605 = iprot.readString(); - struct.success.add(_elem605); + String _elem653; // required + _elem653 = iprot.readString(); + struct.success.add(_elem653); } } struct.setSuccessIsSet(true); @@ -63313,14 +64327,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, get_partitions_by_f case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list606 = iprot.readListBegin(); - struct.success = new ArrayList(_list606.size); - for (int _i607 = 0; _i607 < _list606.size; ++_i607) + org.apache.thrift.protocol.TList _list654 = iprot.readListBegin(); + struct.success = new ArrayList(_list654.size); + for (int _i655 = 0; _i655 < _list654.size; ++_i655) { - Partition _elem608; // optional - _elem608 = new Partition(); - _elem608.read(iprot); - struct.success.add(_elem608); + Partition _elem656; // required + _elem656 = new Partition(); + _elem656.read(iprot); + struct.success.add(_elem656); } iprot.readListEnd(); } @@ -63364,9 +64378,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, get_partitions_by_ oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size())); - for (Partition _iter609 : struct.success) + for (Partition _iter657 : struct.success) { - _iter609.write(oprot); + _iter657.write(oprot); } oprot.writeListEnd(); } @@ -63413,9 +64427,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, get_partitions_by_f if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (Partition _iter610 : struct.success) + for (Partition _iter658 : struct.success) { - _iter610.write(oprot); + _iter658.write(oprot); } } } @@ -63433,14 +64447,14 @@ public void read(org.apache.thrift.protocol.TProtocol prot, get_partitions_by_fi BitSet incoming = iprot.readBitSet(3); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list611 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); - struct.success = new ArrayList(_list611.size); - for (int _i612 = 0; _i612 < _list611.size; ++_i612) + org.apache.thrift.protocol.TList _list659 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); + struct.success = new ArrayList(_list659.size); + for (int _i660 = 0; _i660 < _list659.size; ++_i660) { - Partition _elem613; // optional - _elem613 = new Partition(); - _elem613.read(iprot); - struct.success.add(_elem613); + Partition _elem661; // required + _elem661 = new Partition(); + _elem661.read(iprot); + struct.success.add(_elem661); } } struct.setSuccessIsSet(true); @@ -64891,13 +65905,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, get_partitions_by_n case 3: // NAMES if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list614 = iprot.readListBegin(); - struct.names = new ArrayList(_list614.size); - for (int _i615 = 0; _i615 < _list614.size; ++_i615) + org.apache.thrift.protocol.TList _list662 = iprot.readListBegin(); + struct.names = new ArrayList(_list662.size); + for (int _i663 = 0; _i663 < _list662.size; ++_i663) { - String _elem616; // optional - _elem616 = iprot.readString(); - struct.names.add(_elem616); + String _elem664; // required + _elem664 = iprot.readString(); + struct.names.add(_elem664); } iprot.readListEnd(); } @@ -64933,9 +65947,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, get_partitions_by_ oprot.writeFieldBegin(NAMES_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.names.size())); - for (String _iter617 : struct.names) + for (String _iter665 : struct.names) { - oprot.writeString(_iter617); + oprot.writeString(_iter665); } oprot.writeListEnd(); } @@ -64978,9 +65992,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, get_partitions_by_n if (struct.isSetNames()) { { oprot.writeI32(struct.names.size()); - for (String _iter618 : struct.names) + for (String _iter666 : struct.names) { - oprot.writeString(_iter618); + oprot.writeString(_iter666); } } } @@ -65000,13 +66014,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, get_partitions_by_na } if (incoming.get(2)) { { - org.apache.thrift.protocol.TList _list619 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); - struct.names = new ArrayList(_list619.size); - for (int _i620 = 0; _i620 < _list619.size; ++_i620) + org.apache.thrift.protocol.TList _list667 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); + struct.names = new ArrayList(_list667.size); + for (int _i668 = 0; _i668 < _list667.size; ++_i668) { - String _elem621; // optional - _elem621 = iprot.readString(); - struct.names.add(_elem621); + String _elem669; // required + _elem669 = iprot.readString(); + struct.names.add(_elem669); } } struct.setNamesIsSet(true); @@ -65493,14 +66507,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, get_partitions_by_n case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list622 = iprot.readListBegin(); - struct.success = new ArrayList(_list622.size); - for (int _i623 = 0; _i623 < _list622.size; ++_i623) + org.apache.thrift.protocol.TList _list670 = iprot.readListBegin(); + struct.success = new ArrayList(_list670.size); + for (int _i671 = 0; _i671 < _list670.size; ++_i671) { - Partition _elem624; // optional - _elem624 = new Partition(); - _elem624.read(iprot); - struct.success.add(_elem624); + Partition _elem672; // required + _elem672 = new Partition(); + _elem672.read(iprot); + struct.success.add(_elem672); } iprot.readListEnd(); } @@ -65544,9 +66558,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, get_partitions_by_ oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size())); - for (Partition _iter625 : struct.success) + for (Partition _iter673 : struct.success) { - _iter625.write(oprot); + _iter673.write(oprot); } oprot.writeListEnd(); } @@ -65593,9 +66607,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, get_partitions_by_n if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (Partition _iter626 : struct.success) + for (Partition _iter674 : struct.success) { - _iter626.write(oprot); + _iter674.write(oprot); } } } @@ -65613,14 +66627,14 @@ public void read(org.apache.thrift.protocol.TProtocol prot, get_partitions_by_na BitSet incoming = iprot.readBitSet(3); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list627 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); - struct.success = new ArrayList(_list627.size); - for (int _i628 = 0; _i628 < _list627.size; ++_i628) + org.apache.thrift.protocol.TList _list675 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); + struct.success = new ArrayList(_list675.size); + for (int _i676 = 0; _i676 < _list675.size; ++_i676) { - Partition _elem629; // optional - _elem629 = new Partition(); - _elem629.read(iprot); - struct.success.add(_elem629); + Partition _elem677; // required + _elem677 = new Partition(); + _elem677.read(iprot); + struct.success.add(_elem677); } } struct.setSuccessIsSet(true); @@ -67170,14 +68184,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, alter_partitions_ar case 3: // NEW_PARTS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list630 = iprot.readListBegin(); - struct.new_parts = new ArrayList(_list630.size); - for (int _i631 = 0; _i631 < _list630.size; ++_i631) + org.apache.thrift.protocol.TList _list678 = iprot.readListBegin(); + struct.new_parts = new ArrayList(_list678.size); + for (int _i679 = 0; _i679 < _list678.size; ++_i679) { - Partition _elem632; // optional - _elem632 = new Partition(); - _elem632.read(iprot); - struct.new_parts.add(_elem632); + Partition _elem680; // required + _elem680 = new Partition(); + _elem680.read(iprot); + struct.new_parts.add(_elem680); } iprot.readListEnd(); } @@ -67213,9 +68227,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, alter_partitions_a oprot.writeFieldBegin(NEW_PARTS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.new_parts.size())); - for (Partition _iter633 : struct.new_parts) + for (Partition _iter681 : struct.new_parts) { - _iter633.write(oprot); + _iter681.write(oprot); } oprot.writeListEnd(); } @@ -67258,9 +68272,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, alter_partitions_ar if (struct.isSetNew_parts()) { { oprot.writeI32(struct.new_parts.size()); - for (Partition _iter634 : struct.new_parts) + for (Partition _iter682 : struct.new_parts) { - _iter634.write(oprot); + _iter682.write(oprot); } } } @@ -67280,14 +68294,14 @@ public void read(org.apache.thrift.protocol.TProtocol prot, alter_partitions_arg } if (incoming.get(2)) { { - org.apache.thrift.protocol.TList _list635 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); - struct.new_parts = new ArrayList(_list635.size); - for (int _i636 = 0; _i636 < _list635.size; ++_i636) + org.apache.thrift.protocol.TList _list683 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); + struct.new_parts = new ArrayList(_list683.size); + for (int _i684 = 0; _i684 < _list683.size; ++_i684) { - Partition _elem637; // optional - _elem637 = new Partition(); - _elem637.read(iprot); - struct.new_parts.add(_elem637); + Partition _elem685; // required + _elem685 = new Partition(); + _elem685.read(iprot); + struct.new_parts.add(_elem685); } } struct.setNew_partsIsSet(true); @@ -69486,13 +70500,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, rename_partition_ar case 3: // PART_VALS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list638 = iprot.readListBegin(); - struct.part_vals = new ArrayList(_list638.size); - for (int _i639 = 0; _i639 < _list638.size; ++_i639) + org.apache.thrift.protocol.TList _list686 = iprot.readListBegin(); + struct.part_vals = new ArrayList(_list686.size); + for (int _i687 = 0; _i687 < _list686.size; ++_i687) { - String _elem640; // optional - _elem640 = iprot.readString(); - struct.part_vals.add(_elem640); + String _elem688; // required + _elem688 = iprot.readString(); + struct.part_vals.add(_elem688); } iprot.readListEnd(); } @@ -69537,9 +70551,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, rename_partition_a oprot.writeFieldBegin(PART_VALS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.part_vals.size())); - for (String _iter641 : struct.part_vals) + for (String _iter689 : struct.part_vals) { - oprot.writeString(_iter641); + oprot.writeString(_iter689); } oprot.writeListEnd(); } @@ -69590,9 +70604,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, rename_partition_ar if (struct.isSetPart_vals()) { { oprot.writeI32(struct.part_vals.size()); - for (String _iter642 : struct.part_vals) + for (String _iter690 : struct.part_vals) { - oprot.writeString(_iter642); + oprot.writeString(_iter690); } } } @@ -69615,13 +70629,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, rename_partition_arg } if (incoming.get(2)) { { - org.apache.thrift.protocol.TList _list643 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); - struct.part_vals = new ArrayList(_list643.size); - for (int _i644 = 0; _i644 < _list643.size; ++_i644) + org.apache.thrift.protocol.TList _list691 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); + struct.part_vals = new ArrayList(_list691.size); + for (int _i692 = 0; _i692 < _list691.size; ++_i692) { - String _elem645; // optional - _elem645 = iprot.readString(); - struct.part_vals.add(_elem645); + String _elem693; // required + _elem693 = iprot.readString(); + struct.part_vals.add(_elem693); } } struct.setPart_valsIsSet(true); @@ -70498,13 +71512,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, partition_name_has_ case 1: // PART_VALS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list646 = iprot.readListBegin(); - struct.part_vals = new ArrayList(_list646.size); - for (int _i647 = 0; _i647 < _list646.size; ++_i647) + org.apache.thrift.protocol.TList _list694 = iprot.readListBegin(); + struct.part_vals = new ArrayList(_list694.size); + for (int _i695 = 0; _i695 < _list694.size; ++_i695) { - String _elem648; // optional - _elem648 = iprot.readString(); - struct.part_vals.add(_elem648); + String _elem696; // required + _elem696 = iprot.readString(); + struct.part_vals.add(_elem696); } iprot.readListEnd(); } @@ -70538,9 +71552,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, partition_name_has oprot.writeFieldBegin(PART_VALS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.part_vals.size())); - for (String _iter649 : struct.part_vals) + for (String _iter697 : struct.part_vals) { - oprot.writeString(_iter649); + oprot.writeString(_iter697); } oprot.writeListEnd(); } @@ -70577,9 +71591,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, partition_name_has_ if (struct.isSetPart_vals()) { { oprot.writeI32(struct.part_vals.size()); - for (String _iter650 : struct.part_vals) + for (String _iter698 : struct.part_vals) { - oprot.writeString(_iter650); + oprot.writeString(_iter698); } } } @@ -70594,13 +71608,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, partition_name_has_v BitSet incoming = iprot.readBitSet(2); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list651 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); - struct.part_vals = new ArrayList(_list651.size); - for (int _i652 = 0; _i652 < _list651.size; ++_i652) + org.apache.thrift.protocol.TList _list699 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); + struct.part_vals = new ArrayList(_list699.size); + for (int _i700 = 0; _i700 < _list699.size; ++_i700) { - String _elem653; // optional - _elem653 = iprot.readString(); - struct.part_vals.add(_elem653); + String _elem701; // required + _elem701 = iprot.readString(); + struct.part_vals.add(_elem701); } } struct.setPart_valsIsSet(true); @@ -72758,13 +73772,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, partition_name_to_v case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list654 = iprot.readListBegin(); - struct.success = new ArrayList(_list654.size); - for (int _i655 = 0; _i655 < _list654.size; ++_i655) + org.apache.thrift.protocol.TList _list702 = iprot.readListBegin(); + struct.success = new ArrayList(_list702.size); + for (int _i703 = 0; _i703 < _list702.size; ++_i703) { - String _elem656; // optional - _elem656 = iprot.readString(); - struct.success.add(_elem656); + String _elem704; // required + _elem704 = iprot.readString(); + struct.success.add(_elem704); } iprot.readListEnd(); } @@ -72799,9 +73813,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, partition_name_to_ oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size())); - for (String _iter657 : struct.success) + for (String _iter705 : struct.success) { - oprot.writeString(_iter657); + oprot.writeString(_iter705); } oprot.writeListEnd(); } @@ -72840,9 +73854,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, partition_name_to_v if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (String _iter658 : struct.success) + for (String _iter706 : struct.success) { - oprot.writeString(_iter658); + oprot.writeString(_iter706); } } } @@ -72857,13 +73871,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, partition_name_to_va BitSet incoming = iprot.readBitSet(2); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list659 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); - struct.success = new ArrayList(_list659.size); - for (int _i660 = 0; _i660 < _list659.size; ++_i660) + org.apache.thrift.protocol.TList _list707 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); + struct.success = new ArrayList(_list707.size); + for (int _i708 = 0; _i708 < _list707.size; ++_i708) { - String _elem661; // optional - _elem661 = iprot.readString(); - struct.success.add(_elem661); + String _elem709; // required + _elem709 = iprot.readString(); + struct.success.add(_elem709); } } struct.setSuccessIsSet(true); @@ -73637,15 +74651,15 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, partition_name_to_s case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { { - org.apache.thrift.protocol.TMap _map662 = iprot.readMapBegin(); - struct.success = new HashMap(2*_map662.size); - for (int _i663 = 0; _i663 < _map662.size; ++_i663) + org.apache.thrift.protocol.TMap _map710 = iprot.readMapBegin(); + struct.success = new HashMap(2*_map710.size); + for (int _i711 = 0; _i711 < _map710.size; ++_i711) { - String _key664; // required - String _val665; // required - _key664 = iprot.readString(); - _val665 = iprot.readString(); - struct.success.put(_key664, _val665); + String _key712; // required + String _val713; // required + _key712 = iprot.readString(); + _val713 = iprot.readString(); + struct.success.put(_key712, _val713); } iprot.readMapEnd(); } @@ -73680,10 +74694,10 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, partition_name_to_ oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.success.size())); - for (Map.Entry _iter666 : struct.success.entrySet()) + for (Map.Entry _iter714 : struct.success.entrySet()) { - oprot.writeString(_iter666.getKey()); - oprot.writeString(_iter666.getValue()); + oprot.writeString(_iter714.getKey()); + oprot.writeString(_iter714.getValue()); } oprot.writeMapEnd(); } @@ -73722,10 +74736,10 @@ public void write(org.apache.thrift.protocol.TProtocol prot, partition_name_to_s if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (Map.Entry _iter667 : struct.success.entrySet()) + for (Map.Entry _iter715 : struct.success.entrySet()) { - oprot.writeString(_iter667.getKey()); - oprot.writeString(_iter667.getValue()); + oprot.writeString(_iter715.getKey()); + oprot.writeString(_iter715.getValue()); } } } @@ -73740,15 +74754,15 @@ public void read(org.apache.thrift.protocol.TProtocol prot, partition_name_to_sp BitSet incoming = iprot.readBitSet(2); if (incoming.get(0)) { { - org.apache.thrift.protocol.TMap _map668 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32()); - struct.success = new HashMap(2*_map668.size); - for (int _i669 = 0; _i669 < _map668.size; ++_i669) + org.apache.thrift.protocol.TMap _map716 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32()); + struct.success = new HashMap(2*_map716.size); + for (int _i717 = 0; _i717 < _map716.size; ++_i717) { - String _key670; // required - String _val671; // required - _key670 = iprot.readString(); - _val671 = iprot.readString(); - struct.success.put(_key670, _val671); + String _key718; // required + String _val719; // required + _key718 = iprot.readString(); + _val719 = iprot.readString(); + struct.success.put(_key718, _val719); } } struct.setSuccessIsSet(true); @@ -74354,15 +75368,15 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, markPartitionForEve case 3: // PART_VALS if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { { - org.apache.thrift.protocol.TMap _map672 = iprot.readMapBegin(); - struct.part_vals = new HashMap(2*_map672.size); - for (int _i673 = 0; _i673 < _map672.size; ++_i673) + org.apache.thrift.protocol.TMap _map720 = iprot.readMapBegin(); + struct.part_vals = new HashMap(2*_map720.size); + for (int _i721 = 0; _i721 < _map720.size; ++_i721) { - String _key674; // required - String _val675; // required - _key674 = iprot.readString(); - _val675 = iprot.readString(); - struct.part_vals.put(_key674, _val675); + String _key722; // required + String _val723; // required + _key722 = iprot.readString(); + _val723 = iprot.readString(); + struct.part_vals.put(_key722, _val723); } iprot.readMapEnd(); } @@ -74406,10 +75420,10 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, markPartitionForEv oprot.writeFieldBegin(PART_VALS_FIELD_DESC); { oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.part_vals.size())); - for (Map.Entry _iter676 : struct.part_vals.entrySet()) + for (Map.Entry _iter724 : struct.part_vals.entrySet()) { - oprot.writeString(_iter676.getKey()); - oprot.writeString(_iter676.getValue()); + oprot.writeString(_iter724.getKey()); + oprot.writeString(_iter724.getValue()); } oprot.writeMapEnd(); } @@ -74460,10 +75474,10 @@ public void write(org.apache.thrift.protocol.TProtocol prot, markPartitionForEve if (struct.isSetPart_vals()) { { oprot.writeI32(struct.part_vals.size()); - for (Map.Entry _iter677 : struct.part_vals.entrySet()) + for (Map.Entry _iter725 : struct.part_vals.entrySet()) { - oprot.writeString(_iter677.getKey()); - oprot.writeString(_iter677.getValue()); + oprot.writeString(_iter725.getKey()); + oprot.writeString(_iter725.getValue()); } } } @@ -74486,15 +75500,15 @@ public void read(org.apache.thrift.protocol.TProtocol prot, markPartitionForEven } if (incoming.get(2)) { { - org.apache.thrift.protocol.TMap _map678 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32()); - struct.part_vals = new HashMap(2*_map678.size); - for (int _i679 = 0; _i679 < _map678.size; ++_i679) + org.apache.thrift.protocol.TMap _map726 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32()); + struct.part_vals = new HashMap(2*_map726.size); + for (int _i727 = 0; _i727 < _map726.size; ++_i727) { - String _key680; // required - String _val681; // required - _key680 = iprot.readString(); - _val681 = iprot.readString(); - struct.part_vals.put(_key680, _val681); + String _key728; // required + String _val729; // required + _key728 = iprot.readString(); + _val729 = iprot.readString(); + struct.part_vals.put(_key728, _val729); } } struct.setPart_valsIsSet(true); @@ -75989,15 +77003,15 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, isPartitionMarkedFo case 3: // PART_VALS if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { { - org.apache.thrift.protocol.TMap _map682 = iprot.readMapBegin(); - struct.part_vals = new HashMap(2*_map682.size); - for (int _i683 = 0; _i683 < _map682.size; ++_i683) + org.apache.thrift.protocol.TMap _map730 = iprot.readMapBegin(); + struct.part_vals = new HashMap(2*_map730.size); + for (int _i731 = 0; _i731 < _map730.size; ++_i731) { - String _key684; // required - String _val685; // required - _key684 = iprot.readString(); - _val685 = iprot.readString(); - struct.part_vals.put(_key684, _val685); + String _key732; // required + String _val733; // required + _key732 = iprot.readString(); + _val733 = iprot.readString(); + struct.part_vals.put(_key732, _val733); } iprot.readMapEnd(); } @@ -76041,10 +77055,10 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, isPartitionMarkedF oprot.writeFieldBegin(PART_VALS_FIELD_DESC); { oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.part_vals.size())); - for (Map.Entry _iter686 : struct.part_vals.entrySet()) + for (Map.Entry _iter734 : struct.part_vals.entrySet()) { - oprot.writeString(_iter686.getKey()); - oprot.writeString(_iter686.getValue()); + oprot.writeString(_iter734.getKey()); + oprot.writeString(_iter734.getValue()); } oprot.writeMapEnd(); } @@ -76095,10 +77109,10 @@ public void write(org.apache.thrift.protocol.TProtocol prot, isPartitionMarkedFo if (struct.isSetPart_vals()) { { oprot.writeI32(struct.part_vals.size()); - for (Map.Entry _iter687 : struct.part_vals.entrySet()) + for (Map.Entry _iter735 : struct.part_vals.entrySet()) { - oprot.writeString(_iter687.getKey()); - oprot.writeString(_iter687.getValue()); + oprot.writeString(_iter735.getKey()); + oprot.writeString(_iter735.getValue()); } } } @@ -76121,15 +77135,15 @@ public void read(org.apache.thrift.protocol.TProtocol prot, isPartitionMarkedFor } if (incoming.get(2)) { { - org.apache.thrift.protocol.TMap _map688 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32()); - struct.part_vals = new HashMap(2*_map688.size); - for (int _i689 = 0; _i689 < _map688.size; ++_i689) + org.apache.thrift.protocol.TMap _map736 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32()); + struct.part_vals = new HashMap(2*_map736.size); + for (int _i737 = 0; _i737 < _map736.size; ++_i737) { - String _key690; // required - String _val691; // required - _key690 = iprot.readString(); - _val691 = iprot.readString(); - struct.part_vals.put(_key690, _val691); + String _key738; // required + String _val739; // required + _key738 = iprot.readString(); + _val739 = iprot.readString(); + struct.part_vals.put(_key738, _val739); } } struct.setPart_valsIsSet(true); @@ -82853,14 +83867,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, get_indexes_result case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list692 = iprot.readListBegin(); - struct.success = new ArrayList(_list692.size); - for (int _i693 = 0; _i693 < _list692.size; ++_i693) + org.apache.thrift.protocol.TList _list740 = iprot.readListBegin(); + struct.success = new ArrayList(_list740.size); + for (int _i741 = 0; _i741 < _list740.size; ++_i741) { - Index _elem694; // optional - _elem694 = new Index(); - _elem694.read(iprot); - struct.success.add(_elem694); + Index _elem742; // required + _elem742 = new Index(); + _elem742.read(iprot); + struct.success.add(_elem742); } iprot.readListEnd(); } @@ -82904,9 +83918,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, get_indexes_result oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size())); - for (Index _iter695 : struct.success) + for (Index _iter743 : struct.success) { - _iter695.write(oprot); + _iter743.write(oprot); } oprot.writeListEnd(); } @@ -82953,9 +83967,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, get_indexes_result if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (Index _iter696 : struct.success) + for (Index _iter744 : struct.success) { - _iter696.write(oprot); + _iter744.write(oprot); } } } @@ -82973,14 +83987,14 @@ public void read(org.apache.thrift.protocol.TProtocol prot, get_indexes_result s BitSet incoming = iprot.readBitSet(3); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list697 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); - struct.success = new ArrayList(_list697.size); - for (int _i698 = 0; _i698 < _list697.size; ++_i698) + org.apache.thrift.protocol.TList _list745 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); + struct.success = new ArrayList(_list745.size); + for (int _i746 = 0; _i746 < _list745.size; ++_i746) { - Index _elem699; // optional - _elem699 = new Index(); - _elem699.read(iprot); - struct.success.add(_elem699); + Index _elem747; // required + _elem747 = new Index(); + _elem747.read(iprot); + struct.success.add(_elem747); } } struct.setSuccessIsSet(true); @@ -83962,13 +84976,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, get_index_names_res case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list700 = iprot.readListBegin(); - struct.success = new ArrayList(_list700.size); - for (int _i701 = 0; _i701 < _list700.size; ++_i701) + org.apache.thrift.protocol.TList _list748 = iprot.readListBegin(); + struct.success = new ArrayList(_list748.size); + for (int _i749 = 0; _i749 < _list748.size; ++_i749) { - String _elem702; // optional - _elem702 = iprot.readString(); - struct.success.add(_elem702); + String _elem750; // required + _elem750 = iprot.readString(); + struct.success.add(_elem750); } iprot.readListEnd(); } @@ -84003,9 +85017,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, get_index_names_re oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size())); - for (String _iter703 : struct.success) + for (String _iter751 : struct.success) { - oprot.writeString(_iter703); + oprot.writeString(_iter751); } oprot.writeListEnd(); } @@ -84044,9 +85058,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, get_index_names_res if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (String _iter704 : struct.success) + for (String _iter752 : struct.success) { - oprot.writeString(_iter704); + oprot.writeString(_iter752); } } } @@ -84061,13 +85075,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, get_index_names_resu BitSet incoming = iprot.readBitSet(2); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list705 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); - struct.success = new ArrayList(_list705.size); - for (int _i706 = 0; _i706 < _list705.size; ++_i706) + org.apache.thrift.protocol.TList _list753 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); + struct.success = new ArrayList(_list753.size); + for (int _i754 = 0; _i754 < _list753.size; ++_i754) { - String _elem707; // optional - _elem707 = iprot.readString(); - struct.success.add(_elem707); + String _elem755; // required + _elem755 = iprot.readString(); + struct.success.add(_elem755); } } struct.setSuccessIsSet(true); @@ -97722,13 +98736,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, get_functions_resul case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list708 = iprot.readListBegin(); - struct.success = new ArrayList(_list708.size); - for (int _i709 = 0; _i709 < _list708.size; ++_i709) + org.apache.thrift.protocol.TList _list756 = iprot.readListBegin(); + struct.success = new ArrayList(_list756.size); + for (int _i757 = 0; _i757 < _list756.size; ++_i757) { - String _elem710; // optional - _elem710 = iprot.readString(); - struct.success.add(_elem710); + String _elem758; // required + _elem758 = iprot.readString(); + struct.success.add(_elem758); } iprot.readListEnd(); } @@ -97763,9 +98777,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, get_functions_resu oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size())); - for (String _iter711 : struct.success) + for (String _iter759 : struct.success) { - oprot.writeString(_iter711); + oprot.writeString(_iter759); } oprot.writeListEnd(); } @@ -97804,9 +98818,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, get_functions_resul if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (String _iter712 : struct.success) + for (String _iter760 : struct.success) { - oprot.writeString(_iter712); + oprot.writeString(_iter760); } } } @@ -97821,13 +98835,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, get_functions_result BitSet incoming = iprot.readBitSet(2); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list713 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); - struct.success = new ArrayList(_list713.size); - for (int _i714 = 0; _i714 < _list713.size; ++_i714) + org.apache.thrift.protocol.TList _list761 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); + struct.success = new ArrayList(_list761.size); + for (int _i762 = 0; _i762 < _list761.size; ++_i762) { - String _elem715; // optional - _elem715 = iprot.readString(); - struct.success.add(_elem715); + String _elem763; // required + _elem763 = iprot.readString(); + struct.success.add(_elem763); } } struct.setSuccessIsSet(true); @@ -101170,13 +102184,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, get_role_names_resu case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list716 = iprot.readListBegin(); - struct.success = new ArrayList(_list716.size); - for (int _i717 = 0; _i717 < _list716.size; ++_i717) + org.apache.thrift.protocol.TList _list764 = iprot.readListBegin(); + struct.success = new ArrayList(_list764.size); + for (int _i765 = 0; _i765 < _list764.size; ++_i765) { - String _elem718; // optional - _elem718 = iprot.readString(); - struct.success.add(_elem718); + String _elem766; // required + _elem766 = iprot.readString(); + struct.success.add(_elem766); } iprot.readListEnd(); } @@ -101211,9 +102225,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, get_role_names_res oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size())); - for (String _iter719 : struct.success) + for (String _iter767 : struct.success) { - oprot.writeString(_iter719); + oprot.writeString(_iter767); } oprot.writeListEnd(); } @@ -101252,9 +102266,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, get_role_names_resu if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (String _iter720 : struct.success) + for (String _iter768 : struct.success) { - oprot.writeString(_iter720); + oprot.writeString(_iter768); } } } @@ -101269,13 +102283,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, get_role_names_resul BitSet incoming = iprot.readBitSet(2); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list721 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); - struct.success = new ArrayList(_list721.size); - for (int _i722 = 0; _i722 < _list721.size; ++_i722) + org.apache.thrift.protocol.TList _list769 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); + struct.success = new ArrayList(_list769.size); + for (int _i770 = 0; _i770 < _list769.size; ++_i770) { - String _elem723; // optional - _elem723 = iprot.readString(); - struct.success.add(_elem723); + String _elem771; // required + _elem771 = iprot.readString(); + struct.success.add(_elem771); } } struct.setSuccessIsSet(true); @@ -104566,14 +105580,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, list_roles_result s case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list724 = iprot.readListBegin(); - struct.success = new ArrayList(_list724.size); - for (int _i725 = 0; _i725 < _list724.size; ++_i725) + org.apache.thrift.protocol.TList _list772 = iprot.readListBegin(); + struct.success = new ArrayList(_list772.size); + for (int _i773 = 0; _i773 < _list772.size; ++_i773) { - Role _elem726; // optional - _elem726 = new Role(); - _elem726.read(iprot); - struct.success.add(_elem726); + Role _elem774; // required + _elem774 = new Role(); + _elem774.read(iprot); + struct.success.add(_elem774); } iprot.readListEnd(); } @@ -104608,9 +105622,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, list_roles_result oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size())); - for (Role _iter727 : struct.success) + for (Role _iter775 : struct.success) { - _iter727.write(oprot); + _iter775.write(oprot); } oprot.writeListEnd(); } @@ -104649,9 +105663,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, list_roles_result s if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (Role _iter728 : struct.success) + for (Role _iter776 : struct.success) { - _iter728.write(oprot); + _iter776.write(oprot); } } } @@ -104666,14 +105680,14 @@ public void read(org.apache.thrift.protocol.TProtocol prot, list_roles_result st BitSet incoming = iprot.readBitSet(2); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list729 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); - struct.success = new ArrayList(_list729.size); - for (int _i730 = 0; _i730 < _list729.size; ++_i730) + org.apache.thrift.protocol.TList _list777 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); + struct.success = new ArrayList(_list777.size); + for (int _i778 = 0; _i778 < _list777.size; ++_i778) { - Role _elem731; // optional - _elem731 = new Role(); - _elem731.read(iprot); - struct.success.add(_elem731); + Role _elem779; // required + _elem779 = new Role(); + _elem779.read(iprot); + struct.success.add(_elem779); } } struct.setSuccessIsSet(true); @@ -105185,13 +106199,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, get_privilege_set_a case 3: // GROUP_NAMES if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list732 = iprot.readListBegin(); - struct.group_names = new ArrayList(_list732.size); - for (int _i733 = 0; _i733 < _list732.size; ++_i733) + org.apache.thrift.protocol.TList _list780 = iprot.readListBegin(); + struct.group_names = new ArrayList(_list780.size); + for (int _i781 = 0; _i781 < _list780.size; ++_i781) { - String _elem734; // optional - _elem734 = iprot.readString(); - struct.group_names.add(_elem734); + String _elem782; // required + _elem782 = iprot.readString(); + struct.group_names.add(_elem782); } iprot.readListEnd(); } @@ -105227,9 +106241,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, get_privilege_set_ oprot.writeFieldBegin(GROUP_NAMES_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.group_names.size())); - for (String _iter735 : struct.group_names) + for (String _iter783 : struct.group_names) { - oprot.writeString(_iter735); + oprot.writeString(_iter783); } oprot.writeListEnd(); } @@ -105272,9 +106286,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, get_privilege_set_a if (struct.isSetGroup_names()) { { oprot.writeI32(struct.group_names.size()); - for (String _iter736 : struct.group_names) + for (String _iter784 : struct.group_names) { - oprot.writeString(_iter736); + oprot.writeString(_iter784); } } } @@ -105295,13 +106309,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, get_privilege_set_ar } if (incoming.get(2)) { { - org.apache.thrift.protocol.TList _list737 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); - struct.group_names = new ArrayList(_list737.size); - for (int _i738 = 0; _i738 < _list737.size; ++_i738) + org.apache.thrift.protocol.TList _list785 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); + struct.group_names = new ArrayList(_list785.size); + for (int _i786 = 0; _i786 < _list785.size; ++_i786) { - String _elem739; // optional - _elem739 = iprot.readString(); - struct.group_names.add(_elem739); + String _elem787; // required + _elem787 = iprot.readString(); + struct.group_names.add(_elem787); } } struct.setGroup_namesIsSet(true); @@ -106759,14 +107773,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, list_privileges_res case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list740 = iprot.readListBegin(); - struct.success = new ArrayList(_list740.size); - for (int _i741 = 0; _i741 < _list740.size; ++_i741) + org.apache.thrift.protocol.TList _list788 = iprot.readListBegin(); + struct.success = new ArrayList(_list788.size); + for (int _i789 = 0; _i789 < _list788.size; ++_i789) { - HiveObjectPrivilege _elem742; // optional - _elem742 = new HiveObjectPrivilege(); - _elem742.read(iprot); - struct.success.add(_elem742); + HiveObjectPrivilege _elem790; // required + _elem790 = new HiveObjectPrivilege(); + _elem790.read(iprot); + struct.success.add(_elem790); } iprot.readListEnd(); } @@ -106801,9 +107815,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, list_privileges_re oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size())); - for (HiveObjectPrivilege _iter743 : struct.success) + for (HiveObjectPrivilege _iter791 : struct.success) { - _iter743.write(oprot); + _iter791.write(oprot); } oprot.writeListEnd(); } @@ -106842,9 +107856,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, list_privileges_res if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (HiveObjectPrivilege _iter744 : struct.success) + for (HiveObjectPrivilege _iter792 : struct.success) { - _iter744.write(oprot); + _iter792.write(oprot); } } } @@ -106859,14 +107873,14 @@ public void read(org.apache.thrift.protocol.TProtocol prot, list_privileges_resu BitSet incoming = iprot.readBitSet(2); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list745 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); - struct.success = new ArrayList(_list745.size); - for (int _i746 = 0; _i746 < _list745.size; ++_i746) + org.apache.thrift.protocol.TList _list793 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); + struct.success = new ArrayList(_list793.size); + for (int _i794 = 0; _i794 < _list793.size; ++_i794) { - HiveObjectPrivilege _elem747; // optional - _elem747 = new HiveObjectPrivilege(); - _elem747.read(iprot); - struct.success.add(_elem747); + HiveObjectPrivilege _elem795; // required + _elem795 = new HiveObjectPrivilege(); + _elem795.read(iprot); + struct.success.add(_elem795); } } struct.setSuccessIsSet(true); @@ -108939,13 +109953,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, set_ugi_args struct case 2: // GROUP_NAMES if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list748 = iprot.readListBegin(); - struct.group_names = new ArrayList(_list748.size); - for (int _i749 = 0; _i749 < _list748.size; ++_i749) + org.apache.thrift.protocol.TList _list796 = iprot.readListBegin(); + struct.group_names = new ArrayList(_list796.size); + for (int _i797 = 0; _i797 < _list796.size; ++_i797) { - String _elem750; // optional - _elem750 = iprot.readString(); - struct.group_names.add(_elem750); + String _elem798; // required + _elem798 = iprot.readString(); + struct.group_names.add(_elem798); } iprot.readListEnd(); } @@ -108976,9 +109990,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, set_ugi_args struc oprot.writeFieldBegin(GROUP_NAMES_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.group_names.size())); - for (String _iter751 : struct.group_names) + for (String _iter799 : struct.group_names) { - oprot.writeString(_iter751); + oprot.writeString(_iter799); } oprot.writeListEnd(); } @@ -109015,9 +110029,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, set_ugi_args struct if (struct.isSetGroup_names()) { { oprot.writeI32(struct.group_names.size()); - for (String _iter752 : struct.group_names) + for (String _iter800 : struct.group_names) { - oprot.writeString(_iter752); + oprot.writeString(_iter800); } } } @@ -109033,13 +110047,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, set_ugi_args struct) } if (incoming.get(1)) { { - org.apache.thrift.protocol.TList _list753 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); - struct.group_names = new ArrayList(_list753.size); - for (int _i754 = 0; _i754 < _list753.size; ++_i754) + org.apache.thrift.protocol.TList _list801 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); + struct.group_names = new ArrayList(_list801.size); + for (int _i802 = 0; _i802 < _list801.size; ++_i802) { - String _elem755; // optional - _elem755 = iprot.readString(); - struct.group_names.add(_elem755); + String _elem803; // required + _elem803 = iprot.readString(); + struct.group_names.add(_elem803); } } struct.setGroup_namesIsSet(true); @@ -109445,13 +110459,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, set_ugi_result stru case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list756 = iprot.readListBegin(); - struct.success = new ArrayList(_list756.size); - for (int _i757 = 0; _i757 < _list756.size; ++_i757) + org.apache.thrift.protocol.TList _list804 = iprot.readListBegin(); + struct.success = new ArrayList(_list804.size); + for (int _i805 = 0; _i805 < _list804.size; ++_i805) { - String _elem758; // optional - _elem758 = iprot.readString(); - struct.success.add(_elem758); + String _elem806; // required + _elem806 = iprot.readString(); + struct.success.add(_elem806); } iprot.readListEnd(); } @@ -109486,9 +110500,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, set_ugi_result str oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size())); - for (String _iter759 : struct.success) + for (String _iter807 : struct.success) { - oprot.writeString(_iter759); + oprot.writeString(_iter807); } oprot.writeListEnd(); } @@ -109527,9 +110541,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, set_ugi_result stru if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (String _iter760 : struct.success) + for (String _iter808 : struct.success) { - oprot.writeString(_iter760); + oprot.writeString(_iter808); } } } @@ -109544,13 +110558,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, set_ugi_result struc BitSet incoming = iprot.readBitSet(2); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list761 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); - struct.success = new ArrayList(_list761.size); - for (int _i762 = 0; _i762 < _list761.size; ++_i762) + org.apache.thrift.protocol.TList _list809 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); + struct.success = new ArrayList(_list809.size); + for (int _i810 = 0; _i810 < _list809.size; ++_i810) { - String _elem763; // optional - _elem763 = iprot.readString(); - struct.success.add(_elem763); + String _elem811; // required + _elem811 = iprot.readString(); + struct.success.add(_elem811); } } struct.setSuccessIsSet(true); @@ -112031,4 +113045,9307 @@ public void read(org.apache.thrift.protocol.TProtocol prot, cancel_delegation_to } + public static class get_open_txns_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("get_open_txns_args"); + + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new get_open_txns_argsStandardSchemeFactory()); + schemes.put(TupleScheme.class, new get_open_txns_argsTupleSchemeFactory()); + } + + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { +; + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(get_open_txns_args.class, metaDataMap); + } + + public get_open_txns_args() { + } + + /** + * Performs a deep copy on other. + */ + public get_open_txns_args(get_open_txns_args other) { + } + + public get_open_txns_args deepCopy() { + return new get_open_txns_args(this); + } + + @Override + public void clear() { + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof get_open_txns_args) + return this.equals((get_open_txns_args)that); + return false; + } + + public boolean equals(get_open_txns_args that) { + if (that == null) + return false; + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + return builder.toHashCode(); + } + + public int compareTo(get_open_txns_args other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + get_open_txns_args typedOther = (get_open_txns_args)other; + + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("get_open_txns_args("); + boolean first = true; + + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class get_open_txns_argsStandardSchemeFactory implements SchemeFactory { + public get_open_txns_argsStandardScheme getScheme() { + return new get_open_txns_argsStandardScheme(); + } + } + + private static class get_open_txns_argsStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, get_open_txns_args struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, get_open_txns_args struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class get_open_txns_argsTupleSchemeFactory implements SchemeFactory { + public get_open_txns_argsTupleScheme getScheme() { + return new get_open_txns_argsTupleScheme(); + } + } + + private static class get_open_txns_argsTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, get_open_txns_args struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, get_open_txns_args struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + } + } + + } + + public static class get_open_txns_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("get_open_txns_result"); + + private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new get_open_txns_resultStandardSchemeFactory()); + schemes.put(TupleScheme.class, new get_open_txns_resultTupleSchemeFactory()); + } + + private GetOpenTxnsResponse success; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + SUCCESS((short)0, "success"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 0: // SUCCESS + return SUCCESS; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, GetOpenTxnsResponse.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(get_open_txns_result.class, metaDataMap); + } + + public get_open_txns_result() { + } + + public get_open_txns_result( + GetOpenTxnsResponse success) + { + this(); + this.success = success; + } + + /** + * Performs a deep copy on other. + */ + public get_open_txns_result(get_open_txns_result other) { + if (other.isSetSuccess()) { + this.success = new GetOpenTxnsResponse(other.success); + } + } + + public get_open_txns_result deepCopy() { + return new get_open_txns_result(this); + } + + @Override + public void clear() { + this.success = null; + } + + public GetOpenTxnsResponse getSuccess() { + return this.success; + } + + public void setSuccess(GetOpenTxnsResponse success) { + this.success = success; + } + + public void unsetSuccess() { + this.success = null; + } + + /** Returns true if field success is set (has been assigned a value) and false otherwise */ + public boolean isSetSuccess() { + return this.success != null; + } + + public void setSuccessIsSet(boolean value) { + if (!value) { + this.success = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case SUCCESS: + if (value == null) { + unsetSuccess(); + } else { + setSuccess((GetOpenTxnsResponse)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case SUCCESS: + return getSuccess(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case SUCCESS: + return isSetSuccess(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof get_open_txns_result) + return this.equals((get_open_txns_result)that); + return false; + } + + public boolean equals(get_open_txns_result that) { + if (that == null) + return false; + + boolean this_present_success = true && this.isSetSuccess(); + boolean that_present_success = true && that.isSetSuccess(); + if (this_present_success || that_present_success) { + if (!(this_present_success && that_present_success)) + return false; + if (!this.success.equals(that.success)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_success = true && (isSetSuccess()); + builder.append(present_success); + if (present_success) + builder.append(success); + + return builder.toHashCode(); + } + + public int compareTo(get_open_txns_result other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + get_open_txns_result typedOther = (get_open_txns_result)other; + + lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(typedOther.isSetSuccess()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetSuccess()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, typedOther.success); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("get_open_txns_result("); + boolean first = true; + + sb.append("success:"); + if (this.success == null) { + sb.append("null"); + } else { + sb.append(this.success); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + if (success != null) { + success.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class get_open_txns_resultStandardSchemeFactory implements SchemeFactory { + public get_open_txns_resultStandardScheme getScheme() { + return new get_open_txns_resultStandardScheme(); + } + } + + private static class get_open_txns_resultStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, get_open_txns_result struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 0: // SUCCESS + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.success = new GetOpenTxnsResponse(); + struct.success.read(iprot); + struct.setSuccessIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, get_open_txns_result struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.success != null) { + oprot.writeFieldBegin(SUCCESS_FIELD_DESC); + struct.success.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class get_open_txns_resultTupleSchemeFactory implements SchemeFactory { + public get_open_txns_resultTupleScheme getScheme() { + return new get_open_txns_resultTupleScheme(); + } + } + + private static class get_open_txns_resultTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, get_open_txns_result struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetSuccess()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetSuccess()) { + struct.success.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, get_open_txns_result struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.success = new GetOpenTxnsResponse(); + struct.success.read(iprot); + struct.setSuccessIsSet(true); + } + } + } + + } + + public static class get_open_txns_info_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("get_open_txns_info_args"); + + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new get_open_txns_info_argsStandardSchemeFactory()); + schemes.put(TupleScheme.class, new get_open_txns_info_argsTupleSchemeFactory()); + } + + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { +; + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(get_open_txns_info_args.class, metaDataMap); + } + + public get_open_txns_info_args() { + } + + /** + * Performs a deep copy on other. + */ + public get_open_txns_info_args(get_open_txns_info_args other) { + } + + public get_open_txns_info_args deepCopy() { + return new get_open_txns_info_args(this); + } + + @Override + public void clear() { + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof get_open_txns_info_args) + return this.equals((get_open_txns_info_args)that); + return false; + } + + public boolean equals(get_open_txns_info_args that) { + if (that == null) + return false; + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + return builder.toHashCode(); + } + + public int compareTo(get_open_txns_info_args other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + get_open_txns_info_args typedOther = (get_open_txns_info_args)other; + + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("get_open_txns_info_args("); + boolean first = true; + + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class get_open_txns_info_argsStandardSchemeFactory implements SchemeFactory { + public get_open_txns_info_argsStandardScheme getScheme() { + return new get_open_txns_info_argsStandardScheme(); + } + } + + private static class get_open_txns_info_argsStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, get_open_txns_info_args struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, get_open_txns_info_args struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class get_open_txns_info_argsTupleSchemeFactory implements SchemeFactory { + public get_open_txns_info_argsTupleScheme getScheme() { + return new get_open_txns_info_argsTupleScheme(); + } + } + + private static class get_open_txns_info_argsTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, get_open_txns_info_args struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, get_open_txns_info_args struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + } + } + + } + + public static class get_open_txns_info_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("get_open_txns_info_result"); + + private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new get_open_txns_info_resultStandardSchemeFactory()); + schemes.put(TupleScheme.class, new get_open_txns_info_resultTupleSchemeFactory()); + } + + private GetOpenTxnsInfoResponse success; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + SUCCESS((short)0, "success"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 0: // SUCCESS + return SUCCESS; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, GetOpenTxnsInfoResponse.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(get_open_txns_info_result.class, metaDataMap); + } + + public get_open_txns_info_result() { + } + + public get_open_txns_info_result( + GetOpenTxnsInfoResponse success) + { + this(); + this.success = success; + } + + /** + * Performs a deep copy on other. + */ + public get_open_txns_info_result(get_open_txns_info_result other) { + if (other.isSetSuccess()) { + this.success = new GetOpenTxnsInfoResponse(other.success); + } + } + + public get_open_txns_info_result deepCopy() { + return new get_open_txns_info_result(this); + } + + @Override + public void clear() { + this.success = null; + } + + public GetOpenTxnsInfoResponse getSuccess() { + return this.success; + } + + public void setSuccess(GetOpenTxnsInfoResponse success) { + this.success = success; + } + + public void unsetSuccess() { + this.success = null; + } + + /** Returns true if field success is set (has been assigned a value) and false otherwise */ + public boolean isSetSuccess() { + return this.success != null; + } + + public void setSuccessIsSet(boolean value) { + if (!value) { + this.success = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case SUCCESS: + if (value == null) { + unsetSuccess(); + } else { + setSuccess((GetOpenTxnsInfoResponse)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case SUCCESS: + return getSuccess(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case SUCCESS: + return isSetSuccess(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof get_open_txns_info_result) + return this.equals((get_open_txns_info_result)that); + return false; + } + + public boolean equals(get_open_txns_info_result that) { + if (that == null) + return false; + + boolean this_present_success = true && this.isSetSuccess(); + boolean that_present_success = true && that.isSetSuccess(); + if (this_present_success || that_present_success) { + if (!(this_present_success && that_present_success)) + return false; + if (!this.success.equals(that.success)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_success = true && (isSetSuccess()); + builder.append(present_success); + if (present_success) + builder.append(success); + + return builder.toHashCode(); + } + + public int compareTo(get_open_txns_info_result other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + get_open_txns_info_result typedOther = (get_open_txns_info_result)other; + + lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(typedOther.isSetSuccess()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetSuccess()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, typedOther.success); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("get_open_txns_info_result("); + boolean first = true; + + sb.append("success:"); + if (this.success == null) { + sb.append("null"); + } else { + sb.append(this.success); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + if (success != null) { + success.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class get_open_txns_info_resultStandardSchemeFactory implements SchemeFactory { + public get_open_txns_info_resultStandardScheme getScheme() { + return new get_open_txns_info_resultStandardScheme(); + } + } + + private static class get_open_txns_info_resultStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, get_open_txns_info_result struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 0: // SUCCESS + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.success = new GetOpenTxnsInfoResponse(); + struct.success.read(iprot); + struct.setSuccessIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, get_open_txns_info_result struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.success != null) { + oprot.writeFieldBegin(SUCCESS_FIELD_DESC); + struct.success.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class get_open_txns_info_resultTupleSchemeFactory implements SchemeFactory { + public get_open_txns_info_resultTupleScheme getScheme() { + return new get_open_txns_info_resultTupleScheme(); + } + } + + private static class get_open_txns_info_resultTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, get_open_txns_info_result struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetSuccess()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetSuccess()) { + struct.success.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, get_open_txns_info_result struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.success = new GetOpenTxnsInfoResponse(); + struct.success.read(iprot); + struct.setSuccessIsSet(true); + } + } + } + + } + + public static class open_txns_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("open_txns_args"); + + private static final org.apache.thrift.protocol.TField RQST_FIELD_DESC = new org.apache.thrift.protocol.TField("rqst", org.apache.thrift.protocol.TType.STRUCT, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new open_txns_argsStandardSchemeFactory()); + schemes.put(TupleScheme.class, new open_txns_argsTupleSchemeFactory()); + } + + private OpenTxnRequest rqst; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + RQST((short)1, "rqst"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // RQST + return RQST; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.RQST, new org.apache.thrift.meta_data.FieldMetaData("rqst", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, OpenTxnRequest.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(open_txns_args.class, metaDataMap); + } + + public open_txns_args() { + } + + public open_txns_args( + OpenTxnRequest rqst) + { + this(); + this.rqst = rqst; + } + + /** + * Performs a deep copy on other. + */ + public open_txns_args(open_txns_args other) { + if (other.isSetRqst()) { + this.rqst = new OpenTxnRequest(other.rqst); + } + } + + public open_txns_args deepCopy() { + return new open_txns_args(this); + } + + @Override + public void clear() { + this.rqst = null; + } + + public OpenTxnRequest getRqst() { + return this.rqst; + } + + public void setRqst(OpenTxnRequest rqst) { + this.rqst = rqst; + } + + public void unsetRqst() { + this.rqst = null; + } + + /** Returns true if field rqst is set (has been assigned a value) and false otherwise */ + public boolean isSetRqst() { + return this.rqst != null; + } + + public void setRqstIsSet(boolean value) { + if (!value) { + this.rqst = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case RQST: + if (value == null) { + unsetRqst(); + } else { + setRqst((OpenTxnRequest)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case RQST: + return getRqst(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case RQST: + return isSetRqst(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof open_txns_args) + return this.equals((open_txns_args)that); + return false; + } + + public boolean equals(open_txns_args that) { + if (that == null) + return false; + + boolean this_present_rqst = true && this.isSetRqst(); + boolean that_present_rqst = true && that.isSetRqst(); + if (this_present_rqst || that_present_rqst) { + if (!(this_present_rqst && that_present_rqst)) + return false; + if (!this.rqst.equals(that.rqst)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_rqst = true && (isSetRqst()); + builder.append(present_rqst); + if (present_rqst) + builder.append(rqst); + + return builder.toHashCode(); + } + + public int compareTo(open_txns_args other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + open_txns_args typedOther = (open_txns_args)other; + + lastComparison = Boolean.valueOf(isSetRqst()).compareTo(typedOther.isSetRqst()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetRqst()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.rqst, typedOther.rqst); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("open_txns_args("); + boolean first = true; + + sb.append("rqst:"); + if (this.rqst == null) { + sb.append("null"); + } else { + sb.append(this.rqst); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + if (rqst != null) { + rqst.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class open_txns_argsStandardSchemeFactory implements SchemeFactory { + public open_txns_argsStandardScheme getScheme() { + return new open_txns_argsStandardScheme(); + } + } + + private static class open_txns_argsStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, open_txns_args struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // RQST + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.rqst = new OpenTxnRequest(); + struct.rqst.read(iprot); + struct.setRqstIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, open_txns_args struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.rqst != null) { + oprot.writeFieldBegin(RQST_FIELD_DESC); + struct.rqst.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class open_txns_argsTupleSchemeFactory implements SchemeFactory { + public open_txns_argsTupleScheme getScheme() { + return new open_txns_argsTupleScheme(); + } + } + + private static class open_txns_argsTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, open_txns_args struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetRqst()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetRqst()) { + struct.rqst.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, open_txns_args struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.rqst = new OpenTxnRequest(); + struct.rqst.read(iprot); + struct.setRqstIsSet(true); + } + } + } + + } + + public static class open_txns_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("open_txns_result"); + + private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new open_txns_resultStandardSchemeFactory()); + schemes.put(TupleScheme.class, new open_txns_resultTupleSchemeFactory()); + } + + private OpenTxnsResponse success; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + SUCCESS((short)0, "success"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 0: // SUCCESS + return SUCCESS; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, OpenTxnsResponse.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(open_txns_result.class, metaDataMap); + } + + public open_txns_result() { + } + + public open_txns_result( + OpenTxnsResponse success) + { + this(); + this.success = success; + } + + /** + * Performs a deep copy on other. + */ + public open_txns_result(open_txns_result other) { + if (other.isSetSuccess()) { + this.success = new OpenTxnsResponse(other.success); + } + } + + public open_txns_result deepCopy() { + return new open_txns_result(this); + } + + @Override + public void clear() { + this.success = null; + } + + public OpenTxnsResponse getSuccess() { + return this.success; + } + + public void setSuccess(OpenTxnsResponse success) { + this.success = success; + } + + public void unsetSuccess() { + this.success = null; + } + + /** Returns true if field success is set (has been assigned a value) and false otherwise */ + public boolean isSetSuccess() { + return this.success != null; + } + + public void setSuccessIsSet(boolean value) { + if (!value) { + this.success = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case SUCCESS: + if (value == null) { + unsetSuccess(); + } else { + setSuccess((OpenTxnsResponse)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case SUCCESS: + return getSuccess(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case SUCCESS: + return isSetSuccess(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof open_txns_result) + return this.equals((open_txns_result)that); + return false; + } + + public boolean equals(open_txns_result that) { + if (that == null) + return false; + + boolean this_present_success = true && this.isSetSuccess(); + boolean that_present_success = true && that.isSetSuccess(); + if (this_present_success || that_present_success) { + if (!(this_present_success && that_present_success)) + return false; + if (!this.success.equals(that.success)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_success = true && (isSetSuccess()); + builder.append(present_success); + if (present_success) + builder.append(success); + + return builder.toHashCode(); + } + + public int compareTo(open_txns_result other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + open_txns_result typedOther = (open_txns_result)other; + + lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(typedOther.isSetSuccess()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetSuccess()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, typedOther.success); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("open_txns_result("); + boolean first = true; + + sb.append("success:"); + if (this.success == null) { + sb.append("null"); + } else { + sb.append(this.success); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + if (success != null) { + success.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class open_txns_resultStandardSchemeFactory implements SchemeFactory { + public open_txns_resultStandardScheme getScheme() { + return new open_txns_resultStandardScheme(); + } + } + + private static class open_txns_resultStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, open_txns_result struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 0: // SUCCESS + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.success = new OpenTxnsResponse(); + struct.success.read(iprot); + struct.setSuccessIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, open_txns_result struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.success != null) { + oprot.writeFieldBegin(SUCCESS_FIELD_DESC); + struct.success.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class open_txns_resultTupleSchemeFactory implements SchemeFactory { + public open_txns_resultTupleScheme getScheme() { + return new open_txns_resultTupleScheme(); + } + } + + private static class open_txns_resultTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, open_txns_result struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetSuccess()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetSuccess()) { + struct.success.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, open_txns_result struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.success = new OpenTxnsResponse(); + struct.success.read(iprot); + struct.setSuccessIsSet(true); + } + } + } + + } + + public static class abort_txn_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("abort_txn_args"); + + private static final org.apache.thrift.protocol.TField RQST_FIELD_DESC = new org.apache.thrift.protocol.TField("rqst", org.apache.thrift.protocol.TType.STRUCT, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new abort_txn_argsStandardSchemeFactory()); + schemes.put(TupleScheme.class, new abort_txn_argsTupleSchemeFactory()); + } + + private AbortTxnRequest rqst; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + RQST((short)1, "rqst"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // RQST + return RQST; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.RQST, new org.apache.thrift.meta_data.FieldMetaData("rqst", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, AbortTxnRequest.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(abort_txn_args.class, metaDataMap); + } + + public abort_txn_args() { + } + + public abort_txn_args( + AbortTxnRequest rqst) + { + this(); + this.rqst = rqst; + } + + /** + * Performs a deep copy on other. + */ + public abort_txn_args(abort_txn_args other) { + if (other.isSetRqst()) { + this.rqst = new AbortTxnRequest(other.rqst); + } + } + + public abort_txn_args deepCopy() { + return new abort_txn_args(this); + } + + @Override + public void clear() { + this.rqst = null; + } + + public AbortTxnRequest getRqst() { + return this.rqst; + } + + public void setRqst(AbortTxnRequest rqst) { + this.rqst = rqst; + } + + public void unsetRqst() { + this.rqst = null; + } + + /** Returns true if field rqst is set (has been assigned a value) and false otherwise */ + public boolean isSetRqst() { + return this.rqst != null; + } + + public void setRqstIsSet(boolean value) { + if (!value) { + this.rqst = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case RQST: + if (value == null) { + unsetRqst(); + } else { + setRqst((AbortTxnRequest)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case RQST: + return getRqst(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case RQST: + return isSetRqst(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof abort_txn_args) + return this.equals((abort_txn_args)that); + return false; + } + + public boolean equals(abort_txn_args that) { + if (that == null) + return false; + + boolean this_present_rqst = true && this.isSetRqst(); + boolean that_present_rqst = true && that.isSetRqst(); + if (this_present_rqst || that_present_rqst) { + if (!(this_present_rqst && that_present_rqst)) + return false; + if (!this.rqst.equals(that.rqst)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_rqst = true && (isSetRqst()); + builder.append(present_rqst); + if (present_rqst) + builder.append(rqst); + + return builder.toHashCode(); + } + + public int compareTo(abort_txn_args other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + abort_txn_args typedOther = (abort_txn_args)other; + + lastComparison = Boolean.valueOf(isSetRqst()).compareTo(typedOther.isSetRqst()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetRqst()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.rqst, typedOther.rqst); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("abort_txn_args("); + boolean first = true; + + sb.append("rqst:"); + if (this.rqst == null) { + sb.append("null"); + } else { + sb.append(this.rqst); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + if (rqst != null) { + rqst.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class abort_txn_argsStandardSchemeFactory implements SchemeFactory { + public abort_txn_argsStandardScheme getScheme() { + return new abort_txn_argsStandardScheme(); + } + } + + private static class abort_txn_argsStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, abort_txn_args struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // RQST + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.rqst = new AbortTxnRequest(); + struct.rqst.read(iprot); + struct.setRqstIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, abort_txn_args struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.rqst != null) { + oprot.writeFieldBegin(RQST_FIELD_DESC); + struct.rqst.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class abort_txn_argsTupleSchemeFactory implements SchemeFactory { + public abort_txn_argsTupleScheme getScheme() { + return new abort_txn_argsTupleScheme(); + } + } + + private static class abort_txn_argsTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, abort_txn_args struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetRqst()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetRqst()) { + struct.rqst.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, abort_txn_args struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.rqst = new AbortTxnRequest(); + struct.rqst.read(iprot); + struct.setRqstIsSet(true); + } + } + } + + } + + public static class abort_txn_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("abort_txn_result"); + + private static final org.apache.thrift.protocol.TField O1_FIELD_DESC = new org.apache.thrift.protocol.TField("o1", org.apache.thrift.protocol.TType.STRUCT, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new abort_txn_resultStandardSchemeFactory()); + schemes.put(TupleScheme.class, new abort_txn_resultTupleSchemeFactory()); + } + + private NoSuchTxnException o1; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + O1((short)1, "o1"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // O1 + return O1; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.O1, new org.apache.thrift.meta_data.FieldMetaData("o1", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(abort_txn_result.class, metaDataMap); + } + + public abort_txn_result() { + } + + public abort_txn_result( + NoSuchTxnException o1) + { + this(); + this.o1 = o1; + } + + /** + * Performs a deep copy on other. + */ + public abort_txn_result(abort_txn_result other) { + if (other.isSetO1()) { + this.o1 = new NoSuchTxnException(other.o1); + } + } + + public abort_txn_result deepCopy() { + return new abort_txn_result(this); + } + + @Override + public void clear() { + this.o1 = null; + } + + public NoSuchTxnException getO1() { + return this.o1; + } + + public void setO1(NoSuchTxnException o1) { + this.o1 = o1; + } + + public void unsetO1() { + this.o1 = null; + } + + /** Returns true if field o1 is set (has been assigned a value) and false otherwise */ + public boolean isSetO1() { + return this.o1 != null; + } + + public void setO1IsSet(boolean value) { + if (!value) { + this.o1 = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case O1: + if (value == null) { + unsetO1(); + } else { + setO1((NoSuchTxnException)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case O1: + return getO1(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case O1: + return isSetO1(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof abort_txn_result) + return this.equals((abort_txn_result)that); + return false; + } + + public boolean equals(abort_txn_result that) { + if (that == null) + return false; + + boolean this_present_o1 = true && this.isSetO1(); + boolean that_present_o1 = true && that.isSetO1(); + if (this_present_o1 || that_present_o1) { + if (!(this_present_o1 && that_present_o1)) + return false; + if (!this.o1.equals(that.o1)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_o1 = true && (isSetO1()); + builder.append(present_o1); + if (present_o1) + builder.append(o1); + + return builder.toHashCode(); + } + + public int compareTo(abort_txn_result other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + abort_txn_result typedOther = (abort_txn_result)other; + + lastComparison = Boolean.valueOf(isSetO1()).compareTo(typedOther.isSetO1()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetO1()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.o1, typedOther.o1); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("abort_txn_result("); + boolean first = true; + + sb.append("o1:"); + if (this.o1 == null) { + sb.append("null"); + } else { + sb.append(this.o1); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class abort_txn_resultStandardSchemeFactory implements SchemeFactory { + public abort_txn_resultStandardScheme getScheme() { + return new abort_txn_resultStandardScheme(); + } + } + + private static class abort_txn_resultStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, abort_txn_result struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // O1 + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.o1 = new NoSuchTxnException(); + struct.o1.read(iprot); + struct.setO1IsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, abort_txn_result struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.o1 != null) { + oprot.writeFieldBegin(O1_FIELD_DESC); + struct.o1.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class abort_txn_resultTupleSchemeFactory implements SchemeFactory { + public abort_txn_resultTupleScheme getScheme() { + return new abort_txn_resultTupleScheme(); + } + } + + private static class abort_txn_resultTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, abort_txn_result struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetO1()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetO1()) { + struct.o1.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, abort_txn_result struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.o1 = new NoSuchTxnException(); + struct.o1.read(iprot); + struct.setO1IsSet(true); + } + } + } + + } + + public static class commit_txn_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("commit_txn_args"); + + private static final org.apache.thrift.protocol.TField RQST_FIELD_DESC = new org.apache.thrift.protocol.TField("rqst", org.apache.thrift.protocol.TType.STRUCT, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new commit_txn_argsStandardSchemeFactory()); + schemes.put(TupleScheme.class, new commit_txn_argsTupleSchemeFactory()); + } + + private CommitTxnRequest rqst; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + RQST((short)1, "rqst"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // RQST + return RQST; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.RQST, new org.apache.thrift.meta_data.FieldMetaData("rqst", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, CommitTxnRequest.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(commit_txn_args.class, metaDataMap); + } + + public commit_txn_args() { + } + + public commit_txn_args( + CommitTxnRequest rqst) + { + this(); + this.rqst = rqst; + } + + /** + * Performs a deep copy on other. + */ + public commit_txn_args(commit_txn_args other) { + if (other.isSetRqst()) { + this.rqst = new CommitTxnRequest(other.rqst); + } + } + + public commit_txn_args deepCopy() { + return new commit_txn_args(this); + } + + @Override + public void clear() { + this.rqst = null; + } + + public CommitTxnRequest getRqst() { + return this.rqst; + } + + public void setRqst(CommitTxnRequest rqst) { + this.rqst = rqst; + } + + public void unsetRqst() { + this.rqst = null; + } + + /** Returns true if field rqst is set (has been assigned a value) and false otherwise */ + public boolean isSetRqst() { + return this.rqst != null; + } + + public void setRqstIsSet(boolean value) { + if (!value) { + this.rqst = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case RQST: + if (value == null) { + unsetRqst(); + } else { + setRqst((CommitTxnRequest)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case RQST: + return getRqst(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case RQST: + return isSetRqst(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof commit_txn_args) + return this.equals((commit_txn_args)that); + return false; + } + + public boolean equals(commit_txn_args that) { + if (that == null) + return false; + + boolean this_present_rqst = true && this.isSetRqst(); + boolean that_present_rqst = true && that.isSetRqst(); + if (this_present_rqst || that_present_rqst) { + if (!(this_present_rqst && that_present_rqst)) + return false; + if (!this.rqst.equals(that.rqst)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_rqst = true && (isSetRqst()); + builder.append(present_rqst); + if (present_rqst) + builder.append(rqst); + + return builder.toHashCode(); + } + + public int compareTo(commit_txn_args other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + commit_txn_args typedOther = (commit_txn_args)other; + + lastComparison = Boolean.valueOf(isSetRqst()).compareTo(typedOther.isSetRqst()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetRqst()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.rqst, typedOther.rqst); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("commit_txn_args("); + boolean first = true; + + sb.append("rqst:"); + if (this.rqst == null) { + sb.append("null"); + } else { + sb.append(this.rqst); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + if (rqst != null) { + rqst.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class commit_txn_argsStandardSchemeFactory implements SchemeFactory { + public commit_txn_argsStandardScheme getScheme() { + return new commit_txn_argsStandardScheme(); + } + } + + private static class commit_txn_argsStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, commit_txn_args struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // RQST + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.rqst = new CommitTxnRequest(); + struct.rqst.read(iprot); + struct.setRqstIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, commit_txn_args struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.rqst != null) { + oprot.writeFieldBegin(RQST_FIELD_DESC); + struct.rqst.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class commit_txn_argsTupleSchemeFactory implements SchemeFactory { + public commit_txn_argsTupleScheme getScheme() { + return new commit_txn_argsTupleScheme(); + } + } + + private static class commit_txn_argsTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, commit_txn_args struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetRqst()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetRqst()) { + struct.rqst.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, commit_txn_args struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.rqst = new CommitTxnRequest(); + struct.rqst.read(iprot); + struct.setRqstIsSet(true); + } + } + } + + } + + public static class commit_txn_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("commit_txn_result"); + + private static final org.apache.thrift.protocol.TField O1_FIELD_DESC = new org.apache.thrift.protocol.TField("o1", org.apache.thrift.protocol.TType.STRUCT, (short)1); + private static final org.apache.thrift.protocol.TField O2_FIELD_DESC = new org.apache.thrift.protocol.TField("o2", org.apache.thrift.protocol.TType.STRUCT, (short)2); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new commit_txn_resultStandardSchemeFactory()); + schemes.put(TupleScheme.class, new commit_txn_resultTupleSchemeFactory()); + } + + private NoSuchTxnException o1; // required + private TxnAbortedException o2; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + O1((short)1, "o1"), + O2((short)2, "o2"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // O1 + return O1; + case 2: // O2 + return O2; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.O1, new org.apache.thrift.meta_data.FieldMetaData("o1", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT))); + tmpMap.put(_Fields.O2, new org.apache.thrift.meta_data.FieldMetaData("o2", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(commit_txn_result.class, metaDataMap); + } + + public commit_txn_result() { + } + + public commit_txn_result( + NoSuchTxnException o1, + TxnAbortedException o2) + { + this(); + this.o1 = o1; + this.o2 = o2; + } + + /** + * Performs a deep copy on other. + */ + public commit_txn_result(commit_txn_result other) { + if (other.isSetO1()) { + this.o1 = new NoSuchTxnException(other.o1); + } + if (other.isSetO2()) { + this.o2 = new TxnAbortedException(other.o2); + } + } + + public commit_txn_result deepCopy() { + return new commit_txn_result(this); + } + + @Override + public void clear() { + this.o1 = null; + this.o2 = null; + } + + public NoSuchTxnException getO1() { + return this.o1; + } + + public void setO1(NoSuchTxnException o1) { + this.o1 = o1; + } + + public void unsetO1() { + this.o1 = null; + } + + /** Returns true if field o1 is set (has been assigned a value) and false otherwise */ + public boolean isSetO1() { + return this.o1 != null; + } + + public void setO1IsSet(boolean value) { + if (!value) { + this.o1 = null; + } + } + + public TxnAbortedException getO2() { + return this.o2; + } + + public void setO2(TxnAbortedException o2) { + this.o2 = o2; + } + + public void unsetO2() { + this.o2 = null; + } + + /** Returns true if field o2 is set (has been assigned a value) and false otherwise */ + public boolean isSetO2() { + return this.o2 != null; + } + + public void setO2IsSet(boolean value) { + if (!value) { + this.o2 = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case O1: + if (value == null) { + unsetO1(); + } else { + setO1((NoSuchTxnException)value); + } + break; + + case O2: + if (value == null) { + unsetO2(); + } else { + setO2((TxnAbortedException)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case O1: + return getO1(); + + case O2: + return getO2(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case O1: + return isSetO1(); + case O2: + return isSetO2(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof commit_txn_result) + return this.equals((commit_txn_result)that); + return false; + } + + public boolean equals(commit_txn_result that) { + if (that == null) + return false; + + boolean this_present_o1 = true && this.isSetO1(); + boolean that_present_o1 = true && that.isSetO1(); + if (this_present_o1 || that_present_o1) { + if (!(this_present_o1 && that_present_o1)) + return false; + if (!this.o1.equals(that.o1)) + return false; + } + + boolean this_present_o2 = true && this.isSetO2(); + boolean that_present_o2 = true && that.isSetO2(); + if (this_present_o2 || that_present_o2) { + if (!(this_present_o2 && that_present_o2)) + return false; + if (!this.o2.equals(that.o2)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_o1 = true && (isSetO1()); + builder.append(present_o1); + if (present_o1) + builder.append(o1); + + boolean present_o2 = true && (isSetO2()); + builder.append(present_o2); + if (present_o2) + builder.append(o2); + + return builder.toHashCode(); + } + + public int compareTo(commit_txn_result other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + commit_txn_result typedOther = (commit_txn_result)other; + + lastComparison = Boolean.valueOf(isSetO1()).compareTo(typedOther.isSetO1()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetO1()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.o1, typedOther.o1); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetO2()).compareTo(typedOther.isSetO2()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetO2()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.o2, typedOther.o2); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("commit_txn_result("); + boolean first = true; + + sb.append("o1:"); + if (this.o1 == null) { + sb.append("null"); + } else { + sb.append(this.o1); + } + first = false; + if (!first) sb.append(", "); + sb.append("o2:"); + if (this.o2 == null) { + sb.append("null"); + } else { + sb.append(this.o2); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class commit_txn_resultStandardSchemeFactory implements SchemeFactory { + public commit_txn_resultStandardScheme getScheme() { + return new commit_txn_resultStandardScheme(); + } + } + + private static class commit_txn_resultStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, commit_txn_result struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // O1 + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.o1 = new NoSuchTxnException(); + struct.o1.read(iprot); + struct.setO1IsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // O2 + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.o2 = new TxnAbortedException(); + struct.o2.read(iprot); + struct.setO2IsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, commit_txn_result struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.o1 != null) { + oprot.writeFieldBegin(O1_FIELD_DESC); + struct.o1.write(oprot); + oprot.writeFieldEnd(); + } + if (struct.o2 != null) { + oprot.writeFieldBegin(O2_FIELD_DESC); + struct.o2.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class commit_txn_resultTupleSchemeFactory implements SchemeFactory { + public commit_txn_resultTupleScheme getScheme() { + return new commit_txn_resultTupleScheme(); + } + } + + private static class commit_txn_resultTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, commit_txn_result struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetO1()) { + optionals.set(0); + } + if (struct.isSetO2()) { + optionals.set(1); + } + oprot.writeBitSet(optionals, 2); + if (struct.isSetO1()) { + struct.o1.write(oprot); + } + if (struct.isSetO2()) { + struct.o2.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, commit_txn_result struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(2); + if (incoming.get(0)) { + struct.o1 = new NoSuchTxnException(); + struct.o1.read(iprot); + struct.setO1IsSet(true); + } + if (incoming.get(1)) { + struct.o2 = new TxnAbortedException(); + struct.o2.read(iprot); + struct.setO2IsSet(true); + } + } + } + + } + + public static class lock_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("lock_args"); + + private static final org.apache.thrift.protocol.TField RQST_FIELD_DESC = new org.apache.thrift.protocol.TField("rqst", org.apache.thrift.protocol.TType.STRUCT, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new lock_argsStandardSchemeFactory()); + schemes.put(TupleScheme.class, new lock_argsTupleSchemeFactory()); + } + + private LockRequest rqst; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + RQST((short)1, "rqst"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // RQST + return RQST; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.RQST, new org.apache.thrift.meta_data.FieldMetaData("rqst", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, LockRequest.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(lock_args.class, metaDataMap); + } + + public lock_args() { + } + + public lock_args( + LockRequest rqst) + { + this(); + this.rqst = rqst; + } + + /** + * Performs a deep copy on other. + */ + public lock_args(lock_args other) { + if (other.isSetRqst()) { + this.rqst = new LockRequest(other.rqst); + } + } + + public lock_args deepCopy() { + return new lock_args(this); + } + + @Override + public void clear() { + this.rqst = null; + } + + public LockRequest getRqst() { + return this.rqst; + } + + public void setRqst(LockRequest rqst) { + this.rqst = rqst; + } + + public void unsetRqst() { + this.rqst = null; + } + + /** Returns true if field rqst is set (has been assigned a value) and false otherwise */ + public boolean isSetRqst() { + return this.rqst != null; + } + + public void setRqstIsSet(boolean value) { + if (!value) { + this.rqst = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case RQST: + if (value == null) { + unsetRqst(); + } else { + setRqst((LockRequest)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case RQST: + return getRqst(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case RQST: + return isSetRqst(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof lock_args) + return this.equals((lock_args)that); + return false; + } + + public boolean equals(lock_args that) { + if (that == null) + return false; + + boolean this_present_rqst = true && this.isSetRqst(); + boolean that_present_rqst = true && that.isSetRqst(); + if (this_present_rqst || that_present_rqst) { + if (!(this_present_rqst && that_present_rqst)) + return false; + if (!this.rqst.equals(that.rqst)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_rqst = true && (isSetRqst()); + builder.append(present_rqst); + if (present_rqst) + builder.append(rqst); + + return builder.toHashCode(); + } + + public int compareTo(lock_args other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + lock_args typedOther = (lock_args)other; + + lastComparison = Boolean.valueOf(isSetRqst()).compareTo(typedOther.isSetRqst()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetRqst()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.rqst, typedOther.rqst); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("lock_args("); + boolean first = true; + + sb.append("rqst:"); + if (this.rqst == null) { + sb.append("null"); + } else { + sb.append(this.rqst); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + if (rqst != null) { + rqst.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class lock_argsStandardSchemeFactory implements SchemeFactory { + public lock_argsStandardScheme getScheme() { + return new lock_argsStandardScheme(); + } + } + + private static class lock_argsStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, lock_args struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // RQST + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.rqst = new LockRequest(); + struct.rqst.read(iprot); + struct.setRqstIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, lock_args struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.rqst != null) { + oprot.writeFieldBegin(RQST_FIELD_DESC); + struct.rqst.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class lock_argsTupleSchemeFactory implements SchemeFactory { + public lock_argsTupleScheme getScheme() { + return new lock_argsTupleScheme(); + } + } + + private static class lock_argsTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, lock_args struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetRqst()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetRqst()) { + struct.rqst.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, lock_args struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.rqst = new LockRequest(); + struct.rqst.read(iprot); + struct.setRqstIsSet(true); + } + } + } + + } + + public static class lock_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("lock_result"); + + private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0); + private static final org.apache.thrift.protocol.TField O1_FIELD_DESC = new org.apache.thrift.protocol.TField("o1", org.apache.thrift.protocol.TType.STRUCT, (short)1); + private static final org.apache.thrift.protocol.TField O2_FIELD_DESC = new org.apache.thrift.protocol.TField("o2", org.apache.thrift.protocol.TType.STRUCT, (short)2); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new lock_resultStandardSchemeFactory()); + schemes.put(TupleScheme.class, new lock_resultTupleSchemeFactory()); + } + + private LockResponse success; // required + private NoSuchTxnException o1; // required + private TxnAbortedException o2; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + SUCCESS((short)0, "success"), + O1((short)1, "o1"), + O2((short)2, "o2"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 0: // SUCCESS + return SUCCESS; + case 1: // O1 + return O1; + case 2: // O2 + return O2; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, LockResponse.class))); + tmpMap.put(_Fields.O1, new org.apache.thrift.meta_data.FieldMetaData("o1", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT))); + tmpMap.put(_Fields.O2, new org.apache.thrift.meta_data.FieldMetaData("o2", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(lock_result.class, metaDataMap); + } + + public lock_result() { + } + + public lock_result( + LockResponse success, + NoSuchTxnException o1, + TxnAbortedException o2) + { + this(); + this.success = success; + this.o1 = o1; + this.o2 = o2; + } + + /** + * Performs a deep copy on other. + */ + public lock_result(lock_result other) { + if (other.isSetSuccess()) { + this.success = new LockResponse(other.success); + } + if (other.isSetO1()) { + this.o1 = new NoSuchTxnException(other.o1); + } + if (other.isSetO2()) { + this.o2 = new TxnAbortedException(other.o2); + } + } + + public lock_result deepCopy() { + return new lock_result(this); + } + + @Override + public void clear() { + this.success = null; + this.o1 = null; + this.o2 = null; + } + + public LockResponse getSuccess() { + return this.success; + } + + public void setSuccess(LockResponse success) { + this.success = success; + } + + public void unsetSuccess() { + this.success = null; + } + + /** Returns true if field success is set (has been assigned a value) and false otherwise */ + public boolean isSetSuccess() { + return this.success != null; + } + + public void setSuccessIsSet(boolean value) { + if (!value) { + this.success = null; + } + } + + public NoSuchTxnException getO1() { + return this.o1; + } + + public void setO1(NoSuchTxnException o1) { + this.o1 = o1; + } + + public void unsetO1() { + this.o1 = null; + } + + /** Returns true if field o1 is set (has been assigned a value) and false otherwise */ + public boolean isSetO1() { + return this.o1 != null; + } + + public void setO1IsSet(boolean value) { + if (!value) { + this.o1 = null; + } + } + + public TxnAbortedException getO2() { + return this.o2; + } + + public void setO2(TxnAbortedException o2) { + this.o2 = o2; + } + + public void unsetO2() { + this.o2 = null; + } + + /** Returns true if field o2 is set (has been assigned a value) and false otherwise */ + public boolean isSetO2() { + return this.o2 != null; + } + + public void setO2IsSet(boolean value) { + if (!value) { + this.o2 = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case SUCCESS: + if (value == null) { + unsetSuccess(); + } else { + setSuccess((LockResponse)value); + } + break; + + case O1: + if (value == null) { + unsetO1(); + } else { + setO1((NoSuchTxnException)value); + } + break; + + case O2: + if (value == null) { + unsetO2(); + } else { + setO2((TxnAbortedException)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case SUCCESS: + return getSuccess(); + + case O1: + return getO1(); + + case O2: + return getO2(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case SUCCESS: + return isSetSuccess(); + case O1: + return isSetO1(); + case O2: + return isSetO2(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof lock_result) + return this.equals((lock_result)that); + return false; + } + + public boolean equals(lock_result that) { + if (that == null) + return false; + + boolean this_present_success = true && this.isSetSuccess(); + boolean that_present_success = true && that.isSetSuccess(); + if (this_present_success || that_present_success) { + if (!(this_present_success && that_present_success)) + return false; + if (!this.success.equals(that.success)) + return false; + } + + boolean this_present_o1 = true && this.isSetO1(); + boolean that_present_o1 = true && that.isSetO1(); + if (this_present_o1 || that_present_o1) { + if (!(this_present_o1 && that_present_o1)) + return false; + if (!this.o1.equals(that.o1)) + return false; + } + + boolean this_present_o2 = true && this.isSetO2(); + boolean that_present_o2 = true && that.isSetO2(); + if (this_present_o2 || that_present_o2) { + if (!(this_present_o2 && that_present_o2)) + return false; + if (!this.o2.equals(that.o2)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_success = true && (isSetSuccess()); + builder.append(present_success); + if (present_success) + builder.append(success); + + boolean present_o1 = true && (isSetO1()); + builder.append(present_o1); + if (present_o1) + builder.append(o1); + + boolean present_o2 = true && (isSetO2()); + builder.append(present_o2); + if (present_o2) + builder.append(o2); + + return builder.toHashCode(); + } + + public int compareTo(lock_result other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + lock_result typedOther = (lock_result)other; + + lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(typedOther.isSetSuccess()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetSuccess()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, typedOther.success); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetO1()).compareTo(typedOther.isSetO1()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetO1()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.o1, typedOther.o1); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetO2()).compareTo(typedOther.isSetO2()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetO2()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.o2, typedOther.o2); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("lock_result("); + boolean first = true; + + sb.append("success:"); + if (this.success == null) { + sb.append("null"); + } else { + sb.append(this.success); + } + first = false; + if (!first) sb.append(", "); + sb.append("o1:"); + if (this.o1 == null) { + sb.append("null"); + } else { + sb.append(this.o1); + } + first = false; + if (!first) sb.append(", "); + sb.append("o2:"); + if (this.o2 == null) { + sb.append("null"); + } else { + sb.append(this.o2); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + if (success != null) { + success.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class lock_resultStandardSchemeFactory implements SchemeFactory { + public lock_resultStandardScheme getScheme() { + return new lock_resultStandardScheme(); + } + } + + private static class lock_resultStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, lock_result struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 0: // SUCCESS + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.success = new LockResponse(); + struct.success.read(iprot); + struct.setSuccessIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 1: // O1 + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.o1 = new NoSuchTxnException(); + struct.o1.read(iprot); + struct.setO1IsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // O2 + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.o2 = new TxnAbortedException(); + struct.o2.read(iprot); + struct.setO2IsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, lock_result struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.success != null) { + oprot.writeFieldBegin(SUCCESS_FIELD_DESC); + struct.success.write(oprot); + oprot.writeFieldEnd(); + } + if (struct.o1 != null) { + oprot.writeFieldBegin(O1_FIELD_DESC); + struct.o1.write(oprot); + oprot.writeFieldEnd(); + } + if (struct.o2 != null) { + oprot.writeFieldBegin(O2_FIELD_DESC); + struct.o2.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class lock_resultTupleSchemeFactory implements SchemeFactory { + public lock_resultTupleScheme getScheme() { + return new lock_resultTupleScheme(); + } + } + + private static class lock_resultTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, lock_result struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetSuccess()) { + optionals.set(0); + } + if (struct.isSetO1()) { + optionals.set(1); + } + if (struct.isSetO2()) { + optionals.set(2); + } + oprot.writeBitSet(optionals, 3); + if (struct.isSetSuccess()) { + struct.success.write(oprot); + } + if (struct.isSetO1()) { + struct.o1.write(oprot); + } + if (struct.isSetO2()) { + struct.o2.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, lock_result struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(3); + if (incoming.get(0)) { + struct.success = new LockResponse(); + struct.success.read(iprot); + struct.setSuccessIsSet(true); + } + if (incoming.get(1)) { + struct.o1 = new NoSuchTxnException(); + struct.o1.read(iprot); + struct.setO1IsSet(true); + } + if (incoming.get(2)) { + struct.o2 = new TxnAbortedException(); + struct.o2.read(iprot); + struct.setO2IsSet(true); + } + } + } + + } + + public static class check_lock_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("check_lock_args"); + + private static final org.apache.thrift.protocol.TField RQST_FIELD_DESC = new org.apache.thrift.protocol.TField("rqst", org.apache.thrift.protocol.TType.STRUCT, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new check_lock_argsStandardSchemeFactory()); + schemes.put(TupleScheme.class, new check_lock_argsTupleSchemeFactory()); + } + + private CheckLockRequest rqst; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + RQST((short)1, "rqst"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // RQST + return RQST; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.RQST, new org.apache.thrift.meta_data.FieldMetaData("rqst", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, CheckLockRequest.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(check_lock_args.class, metaDataMap); + } + + public check_lock_args() { + } + + public check_lock_args( + CheckLockRequest rqst) + { + this(); + this.rqst = rqst; + } + + /** + * Performs a deep copy on other. + */ + public check_lock_args(check_lock_args other) { + if (other.isSetRqst()) { + this.rqst = new CheckLockRequest(other.rqst); + } + } + + public check_lock_args deepCopy() { + return new check_lock_args(this); + } + + @Override + public void clear() { + this.rqst = null; + } + + public CheckLockRequest getRqst() { + return this.rqst; + } + + public void setRqst(CheckLockRequest rqst) { + this.rqst = rqst; + } + + public void unsetRqst() { + this.rqst = null; + } + + /** Returns true if field rqst is set (has been assigned a value) and false otherwise */ + public boolean isSetRqst() { + return this.rqst != null; + } + + public void setRqstIsSet(boolean value) { + if (!value) { + this.rqst = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case RQST: + if (value == null) { + unsetRqst(); + } else { + setRqst((CheckLockRequest)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case RQST: + return getRqst(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case RQST: + return isSetRqst(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof check_lock_args) + return this.equals((check_lock_args)that); + return false; + } + + public boolean equals(check_lock_args that) { + if (that == null) + return false; + + boolean this_present_rqst = true && this.isSetRqst(); + boolean that_present_rqst = true && that.isSetRqst(); + if (this_present_rqst || that_present_rqst) { + if (!(this_present_rqst && that_present_rqst)) + return false; + if (!this.rqst.equals(that.rqst)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_rqst = true && (isSetRqst()); + builder.append(present_rqst); + if (present_rqst) + builder.append(rqst); + + return builder.toHashCode(); + } + + public int compareTo(check_lock_args other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + check_lock_args typedOther = (check_lock_args)other; + + lastComparison = Boolean.valueOf(isSetRqst()).compareTo(typedOther.isSetRqst()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetRqst()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.rqst, typedOther.rqst); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("check_lock_args("); + boolean first = true; + + sb.append("rqst:"); + if (this.rqst == null) { + sb.append("null"); + } else { + sb.append(this.rqst); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + if (rqst != null) { + rqst.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class check_lock_argsStandardSchemeFactory implements SchemeFactory { + public check_lock_argsStandardScheme getScheme() { + return new check_lock_argsStandardScheme(); + } + } + + private static class check_lock_argsStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, check_lock_args struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // RQST + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.rqst = new CheckLockRequest(); + struct.rqst.read(iprot); + struct.setRqstIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, check_lock_args struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.rqst != null) { + oprot.writeFieldBegin(RQST_FIELD_DESC); + struct.rqst.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class check_lock_argsTupleSchemeFactory implements SchemeFactory { + public check_lock_argsTupleScheme getScheme() { + return new check_lock_argsTupleScheme(); + } + } + + private static class check_lock_argsTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, check_lock_args struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetRqst()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetRqst()) { + struct.rqst.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, check_lock_args struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.rqst = new CheckLockRequest(); + struct.rqst.read(iprot); + struct.setRqstIsSet(true); + } + } + } + + } + + public static class check_lock_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("check_lock_result"); + + private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0); + private static final org.apache.thrift.protocol.TField O1_FIELD_DESC = new org.apache.thrift.protocol.TField("o1", org.apache.thrift.protocol.TType.STRUCT, (short)1); + private static final org.apache.thrift.protocol.TField O2_FIELD_DESC = new org.apache.thrift.protocol.TField("o2", org.apache.thrift.protocol.TType.STRUCT, (short)2); + private static final org.apache.thrift.protocol.TField O3_FIELD_DESC = new org.apache.thrift.protocol.TField("o3", org.apache.thrift.protocol.TType.STRUCT, (short)3); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new check_lock_resultStandardSchemeFactory()); + schemes.put(TupleScheme.class, new check_lock_resultTupleSchemeFactory()); + } + + private LockResponse success; // required + private NoSuchTxnException o1; // required + private TxnAbortedException o2; // required + private NoSuchLockException o3; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + SUCCESS((short)0, "success"), + O1((short)1, "o1"), + O2((short)2, "o2"), + O3((short)3, "o3"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 0: // SUCCESS + return SUCCESS; + case 1: // O1 + return O1; + case 2: // O2 + return O2; + case 3: // O3 + return O3; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, LockResponse.class))); + tmpMap.put(_Fields.O1, new org.apache.thrift.meta_data.FieldMetaData("o1", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT))); + tmpMap.put(_Fields.O2, new org.apache.thrift.meta_data.FieldMetaData("o2", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT))); + tmpMap.put(_Fields.O3, new org.apache.thrift.meta_data.FieldMetaData("o3", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(check_lock_result.class, metaDataMap); + } + + public check_lock_result() { + } + + public check_lock_result( + LockResponse success, + NoSuchTxnException o1, + TxnAbortedException o2, + NoSuchLockException o3) + { + this(); + this.success = success; + this.o1 = o1; + this.o2 = o2; + this.o3 = o3; + } + + /** + * Performs a deep copy on other. + */ + public check_lock_result(check_lock_result other) { + if (other.isSetSuccess()) { + this.success = new LockResponse(other.success); + } + if (other.isSetO1()) { + this.o1 = new NoSuchTxnException(other.o1); + } + if (other.isSetO2()) { + this.o2 = new TxnAbortedException(other.o2); + } + if (other.isSetO3()) { + this.o3 = new NoSuchLockException(other.o3); + } + } + + public check_lock_result deepCopy() { + return new check_lock_result(this); + } + + @Override + public void clear() { + this.success = null; + this.o1 = null; + this.o2 = null; + this.o3 = null; + } + + public LockResponse getSuccess() { + return this.success; + } + + public void setSuccess(LockResponse success) { + this.success = success; + } + + public void unsetSuccess() { + this.success = null; + } + + /** Returns true if field success is set (has been assigned a value) and false otherwise */ + public boolean isSetSuccess() { + return this.success != null; + } + + public void setSuccessIsSet(boolean value) { + if (!value) { + this.success = null; + } + } + + public NoSuchTxnException getO1() { + return this.o1; + } + + public void setO1(NoSuchTxnException o1) { + this.o1 = o1; + } + + public void unsetO1() { + this.o1 = null; + } + + /** Returns true if field o1 is set (has been assigned a value) and false otherwise */ + public boolean isSetO1() { + return this.o1 != null; + } + + public void setO1IsSet(boolean value) { + if (!value) { + this.o1 = null; + } + } + + public TxnAbortedException getO2() { + return this.o2; + } + + public void setO2(TxnAbortedException o2) { + this.o2 = o2; + } + + public void unsetO2() { + this.o2 = null; + } + + /** Returns true if field o2 is set (has been assigned a value) and false otherwise */ + public boolean isSetO2() { + return this.o2 != null; + } + + public void setO2IsSet(boolean value) { + if (!value) { + this.o2 = null; + } + } + + public NoSuchLockException getO3() { + return this.o3; + } + + public void setO3(NoSuchLockException o3) { + this.o3 = o3; + } + + public void unsetO3() { + this.o3 = null; + } + + /** Returns true if field o3 is set (has been assigned a value) and false otherwise */ + public boolean isSetO3() { + return this.o3 != null; + } + + public void setO3IsSet(boolean value) { + if (!value) { + this.o3 = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case SUCCESS: + if (value == null) { + unsetSuccess(); + } else { + setSuccess((LockResponse)value); + } + break; + + case O1: + if (value == null) { + unsetO1(); + } else { + setO1((NoSuchTxnException)value); + } + break; + + case O2: + if (value == null) { + unsetO2(); + } else { + setO2((TxnAbortedException)value); + } + break; + + case O3: + if (value == null) { + unsetO3(); + } else { + setO3((NoSuchLockException)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case SUCCESS: + return getSuccess(); + + case O1: + return getO1(); + + case O2: + return getO2(); + + case O3: + return getO3(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case SUCCESS: + return isSetSuccess(); + case O1: + return isSetO1(); + case O2: + return isSetO2(); + case O3: + return isSetO3(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof check_lock_result) + return this.equals((check_lock_result)that); + return false; + } + + public boolean equals(check_lock_result that) { + if (that == null) + return false; + + boolean this_present_success = true && this.isSetSuccess(); + boolean that_present_success = true && that.isSetSuccess(); + if (this_present_success || that_present_success) { + if (!(this_present_success && that_present_success)) + return false; + if (!this.success.equals(that.success)) + return false; + } + + boolean this_present_o1 = true && this.isSetO1(); + boolean that_present_o1 = true && that.isSetO1(); + if (this_present_o1 || that_present_o1) { + if (!(this_present_o1 && that_present_o1)) + return false; + if (!this.o1.equals(that.o1)) + return false; + } + + boolean this_present_o2 = true && this.isSetO2(); + boolean that_present_o2 = true && that.isSetO2(); + if (this_present_o2 || that_present_o2) { + if (!(this_present_o2 && that_present_o2)) + return false; + if (!this.o2.equals(that.o2)) + return false; + } + + boolean this_present_o3 = true && this.isSetO3(); + boolean that_present_o3 = true && that.isSetO3(); + if (this_present_o3 || that_present_o3) { + if (!(this_present_o3 && that_present_o3)) + return false; + if (!this.o3.equals(that.o3)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_success = true && (isSetSuccess()); + builder.append(present_success); + if (present_success) + builder.append(success); + + boolean present_o1 = true && (isSetO1()); + builder.append(present_o1); + if (present_o1) + builder.append(o1); + + boolean present_o2 = true && (isSetO2()); + builder.append(present_o2); + if (present_o2) + builder.append(o2); + + boolean present_o3 = true && (isSetO3()); + builder.append(present_o3); + if (present_o3) + builder.append(o3); + + return builder.toHashCode(); + } + + public int compareTo(check_lock_result other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + check_lock_result typedOther = (check_lock_result)other; + + lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(typedOther.isSetSuccess()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetSuccess()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, typedOther.success); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetO1()).compareTo(typedOther.isSetO1()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetO1()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.o1, typedOther.o1); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetO2()).compareTo(typedOther.isSetO2()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetO2()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.o2, typedOther.o2); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetO3()).compareTo(typedOther.isSetO3()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetO3()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.o3, typedOther.o3); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("check_lock_result("); + boolean first = true; + + sb.append("success:"); + if (this.success == null) { + sb.append("null"); + } else { + sb.append(this.success); + } + first = false; + if (!first) sb.append(", "); + sb.append("o1:"); + if (this.o1 == null) { + sb.append("null"); + } else { + sb.append(this.o1); + } + first = false; + if (!first) sb.append(", "); + sb.append("o2:"); + if (this.o2 == null) { + sb.append("null"); + } else { + sb.append(this.o2); + } + first = false; + if (!first) sb.append(", "); + sb.append("o3:"); + if (this.o3 == null) { + sb.append("null"); + } else { + sb.append(this.o3); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + if (success != null) { + success.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class check_lock_resultStandardSchemeFactory implements SchemeFactory { + public check_lock_resultStandardScheme getScheme() { + return new check_lock_resultStandardScheme(); + } + } + + private static class check_lock_resultStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, check_lock_result struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 0: // SUCCESS + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.success = new LockResponse(); + struct.success.read(iprot); + struct.setSuccessIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 1: // O1 + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.o1 = new NoSuchTxnException(); + struct.o1.read(iprot); + struct.setO1IsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // O2 + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.o2 = new TxnAbortedException(); + struct.o2.read(iprot); + struct.setO2IsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 3: // O3 + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.o3 = new NoSuchLockException(); + struct.o3.read(iprot); + struct.setO3IsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, check_lock_result struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.success != null) { + oprot.writeFieldBegin(SUCCESS_FIELD_DESC); + struct.success.write(oprot); + oprot.writeFieldEnd(); + } + if (struct.o1 != null) { + oprot.writeFieldBegin(O1_FIELD_DESC); + struct.o1.write(oprot); + oprot.writeFieldEnd(); + } + if (struct.o2 != null) { + oprot.writeFieldBegin(O2_FIELD_DESC); + struct.o2.write(oprot); + oprot.writeFieldEnd(); + } + if (struct.o3 != null) { + oprot.writeFieldBegin(O3_FIELD_DESC); + struct.o3.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class check_lock_resultTupleSchemeFactory implements SchemeFactory { + public check_lock_resultTupleScheme getScheme() { + return new check_lock_resultTupleScheme(); + } + } + + private static class check_lock_resultTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, check_lock_result struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetSuccess()) { + optionals.set(0); + } + if (struct.isSetO1()) { + optionals.set(1); + } + if (struct.isSetO2()) { + optionals.set(2); + } + if (struct.isSetO3()) { + optionals.set(3); + } + oprot.writeBitSet(optionals, 4); + if (struct.isSetSuccess()) { + struct.success.write(oprot); + } + if (struct.isSetO1()) { + struct.o1.write(oprot); + } + if (struct.isSetO2()) { + struct.o2.write(oprot); + } + if (struct.isSetO3()) { + struct.o3.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, check_lock_result struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(4); + if (incoming.get(0)) { + struct.success = new LockResponse(); + struct.success.read(iprot); + struct.setSuccessIsSet(true); + } + if (incoming.get(1)) { + struct.o1 = new NoSuchTxnException(); + struct.o1.read(iprot); + struct.setO1IsSet(true); + } + if (incoming.get(2)) { + struct.o2 = new TxnAbortedException(); + struct.o2.read(iprot); + struct.setO2IsSet(true); + } + if (incoming.get(3)) { + struct.o3 = new NoSuchLockException(); + struct.o3.read(iprot); + struct.setO3IsSet(true); + } + } + } + + } + + public static class unlock_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("unlock_args"); + + private static final org.apache.thrift.protocol.TField RQST_FIELD_DESC = new org.apache.thrift.protocol.TField("rqst", org.apache.thrift.protocol.TType.STRUCT, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new unlock_argsStandardSchemeFactory()); + schemes.put(TupleScheme.class, new unlock_argsTupleSchemeFactory()); + } + + private UnlockRequest rqst; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + RQST((short)1, "rqst"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // RQST + return RQST; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.RQST, new org.apache.thrift.meta_data.FieldMetaData("rqst", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, UnlockRequest.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(unlock_args.class, metaDataMap); + } + + public unlock_args() { + } + + public unlock_args( + UnlockRequest rqst) + { + this(); + this.rqst = rqst; + } + + /** + * Performs a deep copy on other. + */ + public unlock_args(unlock_args other) { + if (other.isSetRqst()) { + this.rqst = new UnlockRequest(other.rqst); + } + } + + public unlock_args deepCopy() { + return new unlock_args(this); + } + + @Override + public void clear() { + this.rqst = null; + } + + public UnlockRequest getRqst() { + return this.rqst; + } + + public void setRqst(UnlockRequest rqst) { + this.rqst = rqst; + } + + public void unsetRqst() { + this.rqst = null; + } + + /** Returns true if field rqst is set (has been assigned a value) and false otherwise */ + public boolean isSetRqst() { + return this.rqst != null; + } + + public void setRqstIsSet(boolean value) { + if (!value) { + this.rqst = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case RQST: + if (value == null) { + unsetRqst(); + } else { + setRqst((UnlockRequest)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case RQST: + return getRqst(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case RQST: + return isSetRqst(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof unlock_args) + return this.equals((unlock_args)that); + return false; + } + + public boolean equals(unlock_args that) { + if (that == null) + return false; + + boolean this_present_rqst = true && this.isSetRqst(); + boolean that_present_rqst = true && that.isSetRqst(); + if (this_present_rqst || that_present_rqst) { + if (!(this_present_rqst && that_present_rqst)) + return false; + if (!this.rqst.equals(that.rqst)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_rqst = true && (isSetRqst()); + builder.append(present_rqst); + if (present_rqst) + builder.append(rqst); + + return builder.toHashCode(); + } + + public int compareTo(unlock_args other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + unlock_args typedOther = (unlock_args)other; + + lastComparison = Boolean.valueOf(isSetRqst()).compareTo(typedOther.isSetRqst()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetRqst()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.rqst, typedOther.rqst); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("unlock_args("); + boolean first = true; + + sb.append("rqst:"); + if (this.rqst == null) { + sb.append("null"); + } else { + sb.append(this.rqst); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + if (rqst != null) { + rqst.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class unlock_argsStandardSchemeFactory implements SchemeFactory { + public unlock_argsStandardScheme getScheme() { + return new unlock_argsStandardScheme(); + } + } + + private static class unlock_argsStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, unlock_args struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // RQST + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.rqst = new UnlockRequest(); + struct.rqst.read(iprot); + struct.setRqstIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, unlock_args struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.rqst != null) { + oprot.writeFieldBegin(RQST_FIELD_DESC); + struct.rqst.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class unlock_argsTupleSchemeFactory implements SchemeFactory { + public unlock_argsTupleScheme getScheme() { + return new unlock_argsTupleScheme(); + } + } + + private static class unlock_argsTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, unlock_args struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetRqst()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetRqst()) { + struct.rqst.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, unlock_args struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.rqst = new UnlockRequest(); + struct.rqst.read(iprot); + struct.setRqstIsSet(true); + } + } + } + + } + + public static class unlock_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("unlock_result"); + + private static final org.apache.thrift.protocol.TField O1_FIELD_DESC = new org.apache.thrift.protocol.TField("o1", org.apache.thrift.protocol.TType.STRUCT, (short)1); + private static final org.apache.thrift.protocol.TField O2_FIELD_DESC = new org.apache.thrift.protocol.TField("o2", org.apache.thrift.protocol.TType.STRUCT, (short)2); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new unlock_resultStandardSchemeFactory()); + schemes.put(TupleScheme.class, new unlock_resultTupleSchemeFactory()); + } + + private NoSuchLockException o1; // required + private TxnOpenException o2; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + O1((short)1, "o1"), + O2((short)2, "o2"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // O1 + return O1; + case 2: // O2 + return O2; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.O1, new org.apache.thrift.meta_data.FieldMetaData("o1", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT))); + tmpMap.put(_Fields.O2, new org.apache.thrift.meta_data.FieldMetaData("o2", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(unlock_result.class, metaDataMap); + } + + public unlock_result() { + } + + public unlock_result( + NoSuchLockException o1, + TxnOpenException o2) + { + this(); + this.o1 = o1; + this.o2 = o2; + } + + /** + * Performs a deep copy on other. + */ + public unlock_result(unlock_result other) { + if (other.isSetO1()) { + this.o1 = new NoSuchLockException(other.o1); + } + if (other.isSetO2()) { + this.o2 = new TxnOpenException(other.o2); + } + } + + public unlock_result deepCopy() { + return new unlock_result(this); + } + + @Override + public void clear() { + this.o1 = null; + this.o2 = null; + } + + public NoSuchLockException getO1() { + return this.o1; + } + + public void setO1(NoSuchLockException o1) { + this.o1 = o1; + } + + public void unsetO1() { + this.o1 = null; + } + + /** Returns true if field o1 is set (has been assigned a value) and false otherwise */ + public boolean isSetO1() { + return this.o1 != null; + } + + public void setO1IsSet(boolean value) { + if (!value) { + this.o1 = null; + } + } + + public TxnOpenException getO2() { + return this.o2; + } + + public void setO2(TxnOpenException o2) { + this.o2 = o2; + } + + public void unsetO2() { + this.o2 = null; + } + + /** Returns true if field o2 is set (has been assigned a value) and false otherwise */ + public boolean isSetO2() { + return this.o2 != null; + } + + public void setO2IsSet(boolean value) { + if (!value) { + this.o2 = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case O1: + if (value == null) { + unsetO1(); + } else { + setO1((NoSuchLockException)value); + } + break; + + case O2: + if (value == null) { + unsetO2(); + } else { + setO2((TxnOpenException)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case O1: + return getO1(); + + case O2: + return getO2(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case O1: + return isSetO1(); + case O2: + return isSetO2(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof unlock_result) + return this.equals((unlock_result)that); + return false; + } + + public boolean equals(unlock_result that) { + if (that == null) + return false; + + boolean this_present_o1 = true && this.isSetO1(); + boolean that_present_o1 = true && that.isSetO1(); + if (this_present_o1 || that_present_o1) { + if (!(this_present_o1 && that_present_o1)) + return false; + if (!this.o1.equals(that.o1)) + return false; + } + + boolean this_present_o2 = true && this.isSetO2(); + boolean that_present_o2 = true && that.isSetO2(); + if (this_present_o2 || that_present_o2) { + if (!(this_present_o2 && that_present_o2)) + return false; + if (!this.o2.equals(that.o2)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_o1 = true && (isSetO1()); + builder.append(present_o1); + if (present_o1) + builder.append(o1); + + boolean present_o2 = true && (isSetO2()); + builder.append(present_o2); + if (present_o2) + builder.append(o2); + + return builder.toHashCode(); + } + + public int compareTo(unlock_result other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + unlock_result typedOther = (unlock_result)other; + + lastComparison = Boolean.valueOf(isSetO1()).compareTo(typedOther.isSetO1()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetO1()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.o1, typedOther.o1); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetO2()).compareTo(typedOther.isSetO2()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetO2()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.o2, typedOther.o2); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("unlock_result("); + boolean first = true; + + sb.append("o1:"); + if (this.o1 == null) { + sb.append("null"); + } else { + sb.append(this.o1); + } + first = false; + if (!first) sb.append(", "); + sb.append("o2:"); + if (this.o2 == null) { + sb.append("null"); + } else { + sb.append(this.o2); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class unlock_resultStandardSchemeFactory implements SchemeFactory { + public unlock_resultStandardScheme getScheme() { + return new unlock_resultStandardScheme(); + } + } + + private static class unlock_resultStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, unlock_result struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // O1 + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.o1 = new NoSuchLockException(); + struct.o1.read(iprot); + struct.setO1IsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // O2 + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.o2 = new TxnOpenException(); + struct.o2.read(iprot); + struct.setO2IsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, unlock_result struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.o1 != null) { + oprot.writeFieldBegin(O1_FIELD_DESC); + struct.o1.write(oprot); + oprot.writeFieldEnd(); + } + if (struct.o2 != null) { + oprot.writeFieldBegin(O2_FIELD_DESC); + struct.o2.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class unlock_resultTupleSchemeFactory implements SchemeFactory { + public unlock_resultTupleScheme getScheme() { + return new unlock_resultTupleScheme(); + } + } + + private static class unlock_resultTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, unlock_result struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetO1()) { + optionals.set(0); + } + if (struct.isSetO2()) { + optionals.set(1); + } + oprot.writeBitSet(optionals, 2); + if (struct.isSetO1()) { + struct.o1.write(oprot); + } + if (struct.isSetO2()) { + struct.o2.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, unlock_result struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(2); + if (incoming.get(0)) { + struct.o1 = new NoSuchLockException(); + struct.o1.read(iprot); + struct.setO1IsSet(true); + } + if (incoming.get(1)) { + struct.o2 = new TxnOpenException(); + struct.o2.read(iprot); + struct.setO2IsSet(true); + } + } + } + + } + + public static class show_locks_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("show_locks_args"); + + private static final org.apache.thrift.protocol.TField RQST_FIELD_DESC = new org.apache.thrift.protocol.TField("rqst", org.apache.thrift.protocol.TType.STRUCT, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new show_locks_argsStandardSchemeFactory()); + schemes.put(TupleScheme.class, new show_locks_argsTupleSchemeFactory()); + } + + private ShowLocksRequest rqst; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + RQST((short)1, "rqst"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // RQST + return RQST; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.RQST, new org.apache.thrift.meta_data.FieldMetaData("rqst", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ShowLocksRequest.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(show_locks_args.class, metaDataMap); + } + + public show_locks_args() { + } + + public show_locks_args( + ShowLocksRequest rqst) + { + this(); + this.rqst = rqst; + } + + /** + * Performs a deep copy on other. + */ + public show_locks_args(show_locks_args other) { + if (other.isSetRqst()) { + this.rqst = new ShowLocksRequest(other.rqst); + } + } + + public show_locks_args deepCopy() { + return new show_locks_args(this); + } + + @Override + public void clear() { + this.rqst = null; + } + + public ShowLocksRequest getRqst() { + return this.rqst; + } + + public void setRqst(ShowLocksRequest rqst) { + this.rqst = rqst; + } + + public void unsetRqst() { + this.rqst = null; + } + + /** Returns true if field rqst is set (has been assigned a value) and false otherwise */ + public boolean isSetRqst() { + return this.rqst != null; + } + + public void setRqstIsSet(boolean value) { + if (!value) { + this.rqst = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case RQST: + if (value == null) { + unsetRqst(); + } else { + setRqst((ShowLocksRequest)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case RQST: + return getRqst(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case RQST: + return isSetRqst(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof show_locks_args) + return this.equals((show_locks_args)that); + return false; + } + + public boolean equals(show_locks_args that) { + if (that == null) + return false; + + boolean this_present_rqst = true && this.isSetRqst(); + boolean that_present_rqst = true && that.isSetRqst(); + if (this_present_rqst || that_present_rqst) { + if (!(this_present_rqst && that_present_rqst)) + return false; + if (!this.rqst.equals(that.rqst)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_rqst = true && (isSetRqst()); + builder.append(present_rqst); + if (present_rqst) + builder.append(rqst); + + return builder.toHashCode(); + } + + public int compareTo(show_locks_args other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + show_locks_args typedOther = (show_locks_args)other; + + lastComparison = Boolean.valueOf(isSetRqst()).compareTo(typedOther.isSetRqst()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetRqst()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.rqst, typedOther.rqst); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("show_locks_args("); + boolean first = true; + + sb.append("rqst:"); + if (this.rqst == null) { + sb.append("null"); + } else { + sb.append(this.rqst); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + if (rqst != null) { + rqst.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class show_locks_argsStandardSchemeFactory implements SchemeFactory { + public show_locks_argsStandardScheme getScheme() { + return new show_locks_argsStandardScheme(); + } + } + + private static class show_locks_argsStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, show_locks_args struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // RQST + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.rqst = new ShowLocksRequest(); + struct.rqst.read(iprot); + struct.setRqstIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, show_locks_args struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.rqst != null) { + oprot.writeFieldBegin(RQST_FIELD_DESC); + struct.rqst.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class show_locks_argsTupleSchemeFactory implements SchemeFactory { + public show_locks_argsTupleScheme getScheme() { + return new show_locks_argsTupleScheme(); + } + } + + private static class show_locks_argsTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, show_locks_args struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetRqst()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetRqst()) { + struct.rqst.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, show_locks_args struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.rqst = new ShowLocksRequest(); + struct.rqst.read(iprot); + struct.setRqstIsSet(true); + } + } + } + + } + + public static class show_locks_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("show_locks_result"); + + private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new show_locks_resultStandardSchemeFactory()); + schemes.put(TupleScheme.class, new show_locks_resultTupleSchemeFactory()); + } + + private ShowLocksResponse success; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + SUCCESS((short)0, "success"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 0: // SUCCESS + return SUCCESS; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ShowLocksResponse.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(show_locks_result.class, metaDataMap); + } + + public show_locks_result() { + } + + public show_locks_result( + ShowLocksResponse success) + { + this(); + this.success = success; + } + + /** + * Performs a deep copy on other. + */ + public show_locks_result(show_locks_result other) { + if (other.isSetSuccess()) { + this.success = new ShowLocksResponse(other.success); + } + } + + public show_locks_result deepCopy() { + return new show_locks_result(this); + } + + @Override + public void clear() { + this.success = null; + } + + public ShowLocksResponse getSuccess() { + return this.success; + } + + public void setSuccess(ShowLocksResponse success) { + this.success = success; + } + + public void unsetSuccess() { + this.success = null; + } + + /** Returns true if field success is set (has been assigned a value) and false otherwise */ + public boolean isSetSuccess() { + return this.success != null; + } + + public void setSuccessIsSet(boolean value) { + if (!value) { + this.success = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case SUCCESS: + if (value == null) { + unsetSuccess(); + } else { + setSuccess((ShowLocksResponse)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case SUCCESS: + return getSuccess(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case SUCCESS: + return isSetSuccess(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof show_locks_result) + return this.equals((show_locks_result)that); + return false; + } + + public boolean equals(show_locks_result that) { + if (that == null) + return false; + + boolean this_present_success = true && this.isSetSuccess(); + boolean that_present_success = true && that.isSetSuccess(); + if (this_present_success || that_present_success) { + if (!(this_present_success && that_present_success)) + return false; + if (!this.success.equals(that.success)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_success = true && (isSetSuccess()); + builder.append(present_success); + if (present_success) + builder.append(success); + + return builder.toHashCode(); + } + + public int compareTo(show_locks_result other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + show_locks_result typedOther = (show_locks_result)other; + + lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(typedOther.isSetSuccess()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetSuccess()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, typedOther.success); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("show_locks_result("); + boolean first = true; + + sb.append("success:"); + if (this.success == null) { + sb.append("null"); + } else { + sb.append(this.success); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + if (success != null) { + success.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class show_locks_resultStandardSchemeFactory implements SchemeFactory { + public show_locks_resultStandardScheme getScheme() { + return new show_locks_resultStandardScheme(); + } + } + + private static class show_locks_resultStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, show_locks_result struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 0: // SUCCESS + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.success = new ShowLocksResponse(); + struct.success.read(iprot); + struct.setSuccessIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, show_locks_result struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.success != null) { + oprot.writeFieldBegin(SUCCESS_FIELD_DESC); + struct.success.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class show_locks_resultTupleSchemeFactory implements SchemeFactory { + public show_locks_resultTupleScheme getScheme() { + return new show_locks_resultTupleScheme(); + } + } + + private static class show_locks_resultTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, show_locks_result struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetSuccess()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetSuccess()) { + struct.success.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, show_locks_result struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.success = new ShowLocksResponse(); + struct.success.read(iprot); + struct.setSuccessIsSet(true); + } + } + } + + } + + public static class heartbeat_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("heartbeat_args"); + + private static final org.apache.thrift.protocol.TField IDS_FIELD_DESC = new org.apache.thrift.protocol.TField("ids", org.apache.thrift.protocol.TType.STRUCT, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new heartbeat_argsStandardSchemeFactory()); + schemes.put(TupleScheme.class, new heartbeat_argsTupleSchemeFactory()); + } + + private HeartbeatRequest ids; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + IDS((short)1, "ids"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // IDS + return IDS; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.IDS, new org.apache.thrift.meta_data.FieldMetaData("ids", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, HeartbeatRequest.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(heartbeat_args.class, metaDataMap); + } + + public heartbeat_args() { + } + + public heartbeat_args( + HeartbeatRequest ids) + { + this(); + this.ids = ids; + } + + /** + * Performs a deep copy on other. + */ + public heartbeat_args(heartbeat_args other) { + if (other.isSetIds()) { + this.ids = new HeartbeatRequest(other.ids); + } + } + + public heartbeat_args deepCopy() { + return new heartbeat_args(this); + } + + @Override + public void clear() { + this.ids = null; + } + + public HeartbeatRequest getIds() { + return this.ids; + } + + public void setIds(HeartbeatRequest ids) { + this.ids = ids; + } + + public void unsetIds() { + this.ids = null; + } + + /** Returns true if field ids is set (has been assigned a value) and false otherwise */ + public boolean isSetIds() { + return this.ids != null; + } + + public void setIdsIsSet(boolean value) { + if (!value) { + this.ids = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case IDS: + if (value == null) { + unsetIds(); + } else { + setIds((HeartbeatRequest)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case IDS: + return getIds(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case IDS: + return isSetIds(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof heartbeat_args) + return this.equals((heartbeat_args)that); + return false; + } + + public boolean equals(heartbeat_args that) { + if (that == null) + return false; + + boolean this_present_ids = true && this.isSetIds(); + boolean that_present_ids = true && that.isSetIds(); + if (this_present_ids || that_present_ids) { + if (!(this_present_ids && that_present_ids)) + return false; + if (!this.ids.equals(that.ids)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_ids = true && (isSetIds()); + builder.append(present_ids); + if (present_ids) + builder.append(ids); + + return builder.toHashCode(); + } + + public int compareTo(heartbeat_args other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + heartbeat_args typedOther = (heartbeat_args)other; + + lastComparison = Boolean.valueOf(isSetIds()).compareTo(typedOther.isSetIds()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetIds()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.ids, typedOther.ids); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("heartbeat_args("); + boolean first = true; + + sb.append("ids:"); + if (this.ids == null) { + sb.append("null"); + } else { + sb.append(this.ids); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + if (ids != null) { + ids.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class heartbeat_argsStandardSchemeFactory implements SchemeFactory { + public heartbeat_argsStandardScheme getScheme() { + return new heartbeat_argsStandardScheme(); + } + } + + private static class heartbeat_argsStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, heartbeat_args struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // IDS + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.ids = new HeartbeatRequest(); + struct.ids.read(iprot); + struct.setIdsIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, heartbeat_args struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.ids != null) { + oprot.writeFieldBegin(IDS_FIELD_DESC); + struct.ids.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class heartbeat_argsTupleSchemeFactory implements SchemeFactory { + public heartbeat_argsTupleScheme getScheme() { + return new heartbeat_argsTupleScheme(); + } + } + + private static class heartbeat_argsTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, heartbeat_args struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetIds()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetIds()) { + struct.ids.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, heartbeat_args struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.ids = new HeartbeatRequest(); + struct.ids.read(iprot); + struct.setIdsIsSet(true); + } + } + } + + } + + public static class heartbeat_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("heartbeat_result"); + + private static final org.apache.thrift.protocol.TField O1_FIELD_DESC = new org.apache.thrift.protocol.TField("o1", org.apache.thrift.protocol.TType.STRUCT, (short)1); + private static final org.apache.thrift.protocol.TField O2_FIELD_DESC = new org.apache.thrift.protocol.TField("o2", org.apache.thrift.protocol.TType.STRUCT, (short)2); + private static final org.apache.thrift.protocol.TField O3_FIELD_DESC = new org.apache.thrift.protocol.TField("o3", org.apache.thrift.protocol.TType.STRUCT, (short)3); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new heartbeat_resultStandardSchemeFactory()); + schemes.put(TupleScheme.class, new heartbeat_resultTupleSchemeFactory()); + } + + private NoSuchLockException o1; // required + private NoSuchTxnException o2; // required + private TxnAbortedException o3; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + O1((short)1, "o1"), + O2((short)2, "o2"), + O3((short)3, "o3"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // O1 + return O1; + case 2: // O2 + return O2; + case 3: // O3 + return O3; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.O1, new org.apache.thrift.meta_data.FieldMetaData("o1", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT))); + tmpMap.put(_Fields.O2, new org.apache.thrift.meta_data.FieldMetaData("o2", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT))); + tmpMap.put(_Fields.O3, new org.apache.thrift.meta_data.FieldMetaData("o3", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(heartbeat_result.class, metaDataMap); + } + + public heartbeat_result() { + } + + public heartbeat_result( + NoSuchLockException o1, + NoSuchTxnException o2, + TxnAbortedException o3) + { + this(); + this.o1 = o1; + this.o2 = o2; + this.o3 = o3; + } + + /** + * Performs a deep copy on other. + */ + public heartbeat_result(heartbeat_result other) { + if (other.isSetO1()) { + this.o1 = new NoSuchLockException(other.o1); + } + if (other.isSetO2()) { + this.o2 = new NoSuchTxnException(other.o2); + } + if (other.isSetO3()) { + this.o3 = new TxnAbortedException(other.o3); + } + } + + public heartbeat_result deepCopy() { + return new heartbeat_result(this); + } + + @Override + public void clear() { + this.o1 = null; + this.o2 = null; + this.o3 = null; + } + + public NoSuchLockException getO1() { + return this.o1; + } + + public void setO1(NoSuchLockException o1) { + this.o1 = o1; + } + + public void unsetO1() { + this.o1 = null; + } + + /** Returns true if field o1 is set (has been assigned a value) and false otherwise */ + public boolean isSetO1() { + return this.o1 != null; + } + + public void setO1IsSet(boolean value) { + if (!value) { + this.o1 = null; + } + } + + public NoSuchTxnException getO2() { + return this.o2; + } + + public void setO2(NoSuchTxnException o2) { + this.o2 = o2; + } + + public void unsetO2() { + this.o2 = null; + } + + /** Returns true if field o2 is set (has been assigned a value) and false otherwise */ + public boolean isSetO2() { + return this.o2 != null; + } + + public void setO2IsSet(boolean value) { + if (!value) { + this.o2 = null; + } + } + + public TxnAbortedException getO3() { + return this.o3; + } + + public void setO3(TxnAbortedException o3) { + this.o3 = o3; + } + + public void unsetO3() { + this.o3 = null; + } + + /** Returns true if field o3 is set (has been assigned a value) and false otherwise */ + public boolean isSetO3() { + return this.o3 != null; + } + + public void setO3IsSet(boolean value) { + if (!value) { + this.o3 = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case O1: + if (value == null) { + unsetO1(); + } else { + setO1((NoSuchLockException)value); + } + break; + + case O2: + if (value == null) { + unsetO2(); + } else { + setO2((NoSuchTxnException)value); + } + break; + + case O3: + if (value == null) { + unsetO3(); + } else { + setO3((TxnAbortedException)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case O1: + return getO1(); + + case O2: + return getO2(); + + case O3: + return getO3(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case O1: + return isSetO1(); + case O2: + return isSetO2(); + case O3: + return isSetO3(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof heartbeat_result) + return this.equals((heartbeat_result)that); + return false; + } + + public boolean equals(heartbeat_result that) { + if (that == null) + return false; + + boolean this_present_o1 = true && this.isSetO1(); + boolean that_present_o1 = true && that.isSetO1(); + if (this_present_o1 || that_present_o1) { + if (!(this_present_o1 && that_present_o1)) + return false; + if (!this.o1.equals(that.o1)) + return false; + } + + boolean this_present_o2 = true && this.isSetO2(); + boolean that_present_o2 = true && that.isSetO2(); + if (this_present_o2 || that_present_o2) { + if (!(this_present_o2 && that_present_o2)) + return false; + if (!this.o2.equals(that.o2)) + return false; + } + + boolean this_present_o3 = true && this.isSetO3(); + boolean that_present_o3 = true && that.isSetO3(); + if (this_present_o3 || that_present_o3) { + if (!(this_present_o3 && that_present_o3)) + return false; + if (!this.o3.equals(that.o3)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_o1 = true && (isSetO1()); + builder.append(present_o1); + if (present_o1) + builder.append(o1); + + boolean present_o2 = true && (isSetO2()); + builder.append(present_o2); + if (present_o2) + builder.append(o2); + + boolean present_o3 = true && (isSetO3()); + builder.append(present_o3); + if (present_o3) + builder.append(o3); + + return builder.toHashCode(); + } + + public int compareTo(heartbeat_result other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + heartbeat_result typedOther = (heartbeat_result)other; + + lastComparison = Boolean.valueOf(isSetO1()).compareTo(typedOther.isSetO1()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetO1()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.o1, typedOther.o1); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetO2()).compareTo(typedOther.isSetO2()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetO2()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.o2, typedOther.o2); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetO3()).compareTo(typedOther.isSetO3()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetO3()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.o3, typedOther.o3); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("heartbeat_result("); + boolean first = true; + + sb.append("o1:"); + if (this.o1 == null) { + sb.append("null"); + } else { + sb.append(this.o1); + } + first = false; + if (!first) sb.append(", "); + sb.append("o2:"); + if (this.o2 == null) { + sb.append("null"); + } else { + sb.append(this.o2); + } + first = false; + if (!first) sb.append(", "); + sb.append("o3:"); + if (this.o3 == null) { + sb.append("null"); + } else { + sb.append(this.o3); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class heartbeat_resultStandardSchemeFactory implements SchemeFactory { + public heartbeat_resultStandardScheme getScheme() { + return new heartbeat_resultStandardScheme(); + } + } + + private static class heartbeat_resultStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, heartbeat_result struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // O1 + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.o1 = new NoSuchLockException(); + struct.o1.read(iprot); + struct.setO1IsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // O2 + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.o2 = new NoSuchTxnException(); + struct.o2.read(iprot); + struct.setO2IsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 3: // O3 + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.o3 = new TxnAbortedException(); + struct.o3.read(iprot); + struct.setO3IsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, heartbeat_result struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.o1 != null) { + oprot.writeFieldBegin(O1_FIELD_DESC); + struct.o1.write(oprot); + oprot.writeFieldEnd(); + } + if (struct.o2 != null) { + oprot.writeFieldBegin(O2_FIELD_DESC); + struct.o2.write(oprot); + oprot.writeFieldEnd(); + } + if (struct.o3 != null) { + oprot.writeFieldBegin(O3_FIELD_DESC); + struct.o3.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class heartbeat_resultTupleSchemeFactory implements SchemeFactory { + public heartbeat_resultTupleScheme getScheme() { + return new heartbeat_resultTupleScheme(); + } + } + + private static class heartbeat_resultTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, heartbeat_result struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetO1()) { + optionals.set(0); + } + if (struct.isSetO2()) { + optionals.set(1); + } + if (struct.isSetO3()) { + optionals.set(2); + } + oprot.writeBitSet(optionals, 3); + if (struct.isSetO1()) { + struct.o1.write(oprot); + } + if (struct.isSetO2()) { + struct.o2.write(oprot); + } + if (struct.isSetO3()) { + struct.o3.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, heartbeat_result struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(3); + if (incoming.get(0)) { + struct.o1 = new NoSuchLockException(); + struct.o1.read(iprot); + struct.setO1IsSet(true); + } + if (incoming.get(1)) { + struct.o2 = new NoSuchTxnException(); + struct.o2.read(iprot); + struct.setO2IsSet(true); + } + if (incoming.get(2)) { + struct.o3 = new TxnAbortedException(); + struct.o3.read(iprot); + struct.setO3IsSet(true); + } + } + } + + } + + public static class compact_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("compact_args"); + + private static final org.apache.thrift.protocol.TField RQST_FIELD_DESC = new org.apache.thrift.protocol.TField("rqst", org.apache.thrift.protocol.TType.STRUCT, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new compact_argsStandardSchemeFactory()); + schemes.put(TupleScheme.class, new compact_argsTupleSchemeFactory()); + } + + private CompactionRequest rqst; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + RQST((short)1, "rqst"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // RQST + return RQST; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.RQST, new org.apache.thrift.meta_data.FieldMetaData("rqst", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, CompactionRequest.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(compact_args.class, metaDataMap); + } + + public compact_args() { + } + + public compact_args( + CompactionRequest rqst) + { + this(); + this.rqst = rqst; + } + + /** + * Performs a deep copy on other. + */ + public compact_args(compact_args other) { + if (other.isSetRqst()) { + this.rqst = new CompactionRequest(other.rqst); + } + } + + public compact_args deepCopy() { + return new compact_args(this); + } + + @Override + public void clear() { + this.rqst = null; + } + + public CompactionRequest getRqst() { + return this.rqst; + } + + public void setRqst(CompactionRequest rqst) { + this.rqst = rqst; + } + + public void unsetRqst() { + this.rqst = null; + } + + /** Returns true if field rqst is set (has been assigned a value) and false otherwise */ + public boolean isSetRqst() { + return this.rqst != null; + } + + public void setRqstIsSet(boolean value) { + if (!value) { + this.rqst = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case RQST: + if (value == null) { + unsetRqst(); + } else { + setRqst((CompactionRequest)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case RQST: + return getRqst(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case RQST: + return isSetRqst(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof compact_args) + return this.equals((compact_args)that); + return false; + } + + public boolean equals(compact_args that) { + if (that == null) + return false; + + boolean this_present_rqst = true && this.isSetRqst(); + boolean that_present_rqst = true && that.isSetRqst(); + if (this_present_rqst || that_present_rqst) { + if (!(this_present_rqst && that_present_rqst)) + return false; + if (!this.rqst.equals(that.rqst)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_rqst = true && (isSetRqst()); + builder.append(present_rqst); + if (present_rqst) + builder.append(rqst); + + return builder.toHashCode(); + } + + public int compareTo(compact_args other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + compact_args typedOther = (compact_args)other; + + lastComparison = Boolean.valueOf(isSetRqst()).compareTo(typedOther.isSetRqst()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetRqst()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.rqst, typedOther.rqst); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("compact_args("); + boolean first = true; + + sb.append("rqst:"); + if (this.rqst == null) { + sb.append("null"); + } else { + sb.append(this.rqst); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + if (rqst != null) { + rqst.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class compact_argsStandardSchemeFactory implements SchemeFactory { + public compact_argsStandardScheme getScheme() { + return new compact_argsStandardScheme(); + } + } + + private static class compact_argsStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, compact_args struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // RQST + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.rqst = new CompactionRequest(); + struct.rqst.read(iprot); + struct.setRqstIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, compact_args struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.rqst != null) { + oprot.writeFieldBegin(RQST_FIELD_DESC); + struct.rqst.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class compact_argsTupleSchemeFactory implements SchemeFactory { + public compact_argsTupleScheme getScheme() { + return new compact_argsTupleScheme(); + } + } + + private static class compact_argsTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, compact_args struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetRqst()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetRqst()) { + struct.rqst.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, compact_args struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.rqst = new CompactionRequest(); + struct.rqst.read(iprot); + struct.setRqstIsSet(true); + } + } + } + + } + + public static class compact_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("compact_result"); + + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new compact_resultStandardSchemeFactory()); + schemes.put(TupleScheme.class, new compact_resultTupleSchemeFactory()); + } + + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { +; + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(compact_result.class, metaDataMap); + } + + public compact_result() { + } + + /** + * Performs a deep copy on other. + */ + public compact_result(compact_result other) { + } + + public compact_result deepCopy() { + return new compact_result(this); + } + + @Override + public void clear() { + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof compact_result) + return this.equals((compact_result)that); + return false; + } + + public boolean equals(compact_result that) { + if (that == null) + return false; + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + return builder.toHashCode(); + } + + public int compareTo(compact_result other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + compact_result typedOther = (compact_result)other; + + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("compact_result("); + boolean first = true; + + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class compact_resultStandardSchemeFactory implements SchemeFactory { + public compact_resultStandardScheme getScheme() { + return new compact_resultStandardScheme(); + } + } + + private static class compact_resultStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, compact_result struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, compact_result struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class compact_resultTupleSchemeFactory implements SchemeFactory { + public compact_resultTupleScheme getScheme() { + return new compact_resultTupleScheme(); + } + } + + private static class compact_resultTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, compact_result struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, compact_result struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + } + } + + } + + public static class show_compact_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("show_compact_args"); + + private static final org.apache.thrift.protocol.TField RQST_FIELD_DESC = new org.apache.thrift.protocol.TField("rqst", org.apache.thrift.protocol.TType.STRUCT, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new show_compact_argsStandardSchemeFactory()); + schemes.put(TupleScheme.class, new show_compact_argsTupleSchemeFactory()); + } + + private ShowCompactRequest rqst; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + RQST((short)1, "rqst"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // RQST + return RQST; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.RQST, new org.apache.thrift.meta_data.FieldMetaData("rqst", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ShowCompactRequest.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(show_compact_args.class, metaDataMap); + } + + public show_compact_args() { + } + + public show_compact_args( + ShowCompactRequest rqst) + { + this(); + this.rqst = rqst; + } + + /** + * Performs a deep copy on other. + */ + public show_compact_args(show_compact_args other) { + if (other.isSetRqst()) { + this.rqst = new ShowCompactRequest(other.rqst); + } + } + + public show_compact_args deepCopy() { + return new show_compact_args(this); + } + + @Override + public void clear() { + this.rqst = null; + } + + public ShowCompactRequest getRqst() { + return this.rqst; + } + + public void setRqst(ShowCompactRequest rqst) { + this.rqst = rqst; + } + + public void unsetRqst() { + this.rqst = null; + } + + /** Returns true if field rqst is set (has been assigned a value) and false otherwise */ + public boolean isSetRqst() { + return this.rqst != null; + } + + public void setRqstIsSet(boolean value) { + if (!value) { + this.rqst = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case RQST: + if (value == null) { + unsetRqst(); + } else { + setRqst((ShowCompactRequest)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case RQST: + return getRqst(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case RQST: + return isSetRqst(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof show_compact_args) + return this.equals((show_compact_args)that); + return false; + } + + public boolean equals(show_compact_args that) { + if (that == null) + return false; + + boolean this_present_rqst = true && this.isSetRqst(); + boolean that_present_rqst = true && that.isSetRqst(); + if (this_present_rqst || that_present_rqst) { + if (!(this_present_rqst && that_present_rqst)) + return false; + if (!this.rqst.equals(that.rqst)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_rqst = true && (isSetRqst()); + builder.append(present_rqst); + if (present_rqst) + builder.append(rqst); + + return builder.toHashCode(); + } + + public int compareTo(show_compact_args other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + show_compact_args typedOther = (show_compact_args)other; + + lastComparison = Boolean.valueOf(isSetRqst()).compareTo(typedOther.isSetRqst()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetRqst()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.rqst, typedOther.rqst); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("show_compact_args("); + boolean first = true; + + sb.append("rqst:"); + if (this.rqst == null) { + sb.append("null"); + } else { + sb.append(this.rqst); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + if (rqst != null) { + rqst.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class show_compact_argsStandardSchemeFactory implements SchemeFactory { + public show_compact_argsStandardScheme getScheme() { + return new show_compact_argsStandardScheme(); + } + } + + private static class show_compact_argsStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, show_compact_args struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // RQST + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.rqst = new ShowCompactRequest(); + struct.rqst.read(iprot); + struct.setRqstIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, show_compact_args struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.rqst != null) { + oprot.writeFieldBegin(RQST_FIELD_DESC); + struct.rqst.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class show_compact_argsTupleSchemeFactory implements SchemeFactory { + public show_compact_argsTupleScheme getScheme() { + return new show_compact_argsTupleScheme(); + } + } + + private static class show_compact_argsTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, show_compact_args struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetRqst()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetRqst()) { + struct.rqst.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, show_compact_args struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.rqst = new ShowCompactRequest(); + struct.rqst.read(iprot); + struct.setRqstIsSet(true); + } + } + } + + } + + public static class show_compact_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("show_compact_result"); + + private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new show_compact_resultStandardSchemeFactory()); + schemes.put(TupleScheme.class, new show_compact_resultTupleSchemeFactory()); + } + + private ShowCompactResponse success; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + SUCCESS((short)0, "success"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 0: // SUCCESS + return SUCCESS; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ShowCompactResponse.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(show_compact_result.class, metaDataMap); + } + + public show_compact_result() { + } + + public show_compact_result( + ShowCompactResponse success) + { + this(); + this.success = success; + } + + /** + * Performs a deep copy on other. + */ + public show_compact_result(show_compact_result other) { + if (other.isSetSuccess()) { + this.success = new ShowCompactResponse(other.success); + } + } + + public show_compact_result deepCopy() { + return new show_compact_result(this); + } + + @Override + public void clear() { + this.success = null; + } + + public ShowCompactResponse getSuccess() { + return this.success; + } + + public void setSuccess(ShowCompactResponse success) { + this.success = success; + } + + public void unsetSuccess() { + this.success = null; + } + + /** Returns true if field success is set (has been assigned a value) and false otherwise */ + public boolean isSetSuccess() { + return this.success != null; + } + + public void setSuccessIsSet(boolean value) { + if (!value) { + this.success = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case SUCCESS: + if (value == null) { + unsetSuccess(); + } else { + setSuccess((ShowCompactResponse)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case SUCCESS: + return getSuccess(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case SUCCESS: + return isSetSuccess(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof show_compact_result) + return this.equals((show_compact_result)that); + return false; + } + + public boolean equals(show_compact_result that) { + if (that == null) + return false; + + boolean this_present_success = true && this.isSetSuccess(); + boolean that_present_success = true && that.isSetSuccess(); + if (this_present_success || that_present_success) { + if (!(this_present_success && that_present_success)) + return false; + if (!this.success.equals(that.success)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_success = true && (isSetSuccess()); + builder.append(present_success); + if (present_success) + builder.append(success); + + return builder.toHashCode(); + } + + public int compareTo(show_compact_result other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + show_compact_result typedOther = (show_compact_result)other; + + lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(typedOther.isSetSuccess()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetSuccess()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, typedOther.success); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("show_compact_result("); + boolean first = true; + + sb.append("success:"); + if (this.success == null) { + sb.append("null"); + } else { + sb.append(this.success); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + if (success != null) { + success.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class show_compact_resultStandardSchemeFactory implements SchemeFactory { + public show_compact_resultStandardScheme getScheme() { + return new show_compact_resultStandardScheme(); + } + } + + private static class show_compact_resultStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, show_compact_result struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 0: // SUCCESS + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.success = new ShowCompactResponse(); + struct.success.read(iprot); + struct.setSuccessIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, show_compact_result struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.success != null) { + oprot.writeFieldBegin(SUCCESS_FIELD_DESC); + struct.success.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class show_compact_resultTupleSchemeFactory implements SchemeFactory { + public show_compact_resultTupleScheme getScheme() { + return new show_compact_resultTupleScheme(); + } + } + + private static class show_compact_resultTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, show_compact_result struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetSuccess()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetSuccess()) { + struct.success.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, show_compact_result struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.success = new ShowCompactResponse(); + struct.success.read(iprot); + struct.setSuccessIsSet(true); + } + } + } + + } + } diff --git metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TxnAbortedException.java metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TxnAbortedException.java new file mode 100644 index 0000000..ae39507 --- /dev/null +++ metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TxnAbortedException.java @@ -0,0 +1,391 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hadoop.hive.metastore.api; + +import org.apache.commons.lang.builder.HashCodeBuilder; +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class TxnAbortedException extends TException implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TxnAbortedException"); + + private static final org.apache.thrift.protocol.TField MESSAGE_FIELD_DESC = new org.apache.thrift.protocol.TField("message", org.apache.thrift.protocol.TType.STRING, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TxnAbortedExceptionStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TxnAbortedExceptionTupleSchemeFactory()); + } + + private String message; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + MESSAGE((short)1, "message"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // MESSAGE + return MESSAGE; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.MESSAGE, new org.apache.thrift.meta_data.FieldMetaData("message", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TxnAbortedException.class, metaDataMap); + } + + public TxnAbortedException() { + } + + public TxnAbortedException( + String message) + { + this(); + this.message = message; + } + + /** + * Performs a deep copy on other. + */ + public TxnAbortedException(TxnAbortedException other) { + if (other.isSetMessage()) { + this.message = other.message; + } + } + + public TxnAbortedException deepCopy() { + return new TxnAbortedException(this); + } + + @Override + public void clear() { + this.message = null; + } + + public String getMessage() { + return this.message; + } + + public void setMessage(String message) { + this.message = message; + } + + public void unsetMessage() { + this.message = null; + } + + /** Returns true if field message is set (has been assigned a value) and false otherwise */ + public boolean isSetMessage() { + return this.message != null; + } + + public void setMessageIsSet(boolean value) { + if (!value) { + this.message = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case MESSAGE: + if (value == null) { + unsetMessage(); + } else { + setMessage((String)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case MESSAGE: + return getMessage(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case MESSAGE: + return isSetMessage(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TxnAbortedException) + return this.equals((TxnAbortedException)that); + return false; + } + + public boolean equals(TxnAbortedException that) { + if (that == null) + return false; + + boolean this_present_message = true && this.isSetMessage(); + boolean that_present_message = true && that.isSetMessage(); + if (this_present_message || that_present_message) { + if (!(this_present_message && that_present_message)) + return false; + if (!this.message.equals(that.message)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_message = true && (isSetMessage()); + builder.append(present_message); + if (present_message) + builder.append(message); + + return builder.toHashCode(); + } + + public int compareTo(TxnAbortedException other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + TxnAbortedException typedOther = (TxnAbortedException)other; + + lastComparison = Boolean.valueOf(isSetMessage()).compareTo(typedOther.isSetMessage()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetMessage()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.message, typedOther.message); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TxnAbortedException("); + boolean first = true; + + sb.append("message:"); + if (this.message == null) { + sb.append("null"); + } else { + sb.append(this.message); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TxnAbortedExceptionStandardSchemeFactory implements SchemeFactory { + public TxnAbortedExceptionStandardScheme getScheme() { + return new TxnAbortedExceptionStandardScheme(); + } + } + + private static class TxnAbortedExceptionStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TxnAbortedException struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // MESSAGE + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.message = iprot.readString(); + struct.setMessageIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TxnAbortedException struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.message != null) { + oprot.writeFieldBegin(MESSAGE_FIELD_DESC); + oprot.writeString(struct.message); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TxnAbortedExceptionTupleSchemeFactory implements SchemeFactory { + public TxnAbortedExceptionTupleScheme getScheme() { + return new TxnAbortedExceptionTupleScheme(); + } + } + + private static class TxnAbortedExceptionTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TxnAbortedException struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetMessage()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetMessage()) { + oprot.writeString(struct.message); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TxnAbortedException struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.message = iprot.readString(); + struct.setMessageIsSet(true); + } + } + } + +} + diff --git metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TxnInfo.java metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TxnInfo.java new file mode 100644 index 0000000..18cbe53 --- /dev/null +++ metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TxnInfo.java @@ -0,0 +1,698 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hadoop.hive.metastore.api; + +import org.apache.commons.lang.builder.HashCodeBuilder; +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class TxnInfo implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TxnInfo"); + + private static final org.apache.thrift.protocol.TField ID_FIELD_DESC = new org.apache.thrift.protocol.TField("id", org.apache.thrift.protocol.TType.I64, (short)1); + private static final org.apache.thrift.protocol.TField STATE_FIELD_DESC = new org.apache.thrift.protocol.TField("state", org.apache.thrift.protocol.TType.I32, (short)2); + private static final org.apache.thrift.protocol.TField USER_FIELD_DESC = new org.apache.thrift.protocol.TField("user", org.apache.thrift.protocol.TType.STRING, (short)3); + private static final org.apache.thrift.protocol.TField HOSTNAME_FIELD_DESC = new org.apache.thrift.protocol.TField("hostname", org.apache.thrift.protocol.TType.STRING, (short)4); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TxnInfoStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TxnInfoTupleSchemeFactory()); + } + + private long id; // required + private TxnState state; // required + private String user; // required + private String hostname; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + ID((short)1, "id"), + /** + * + * @see TxnState + */ + STATE((short)2, "state"), + USER((short)3, "user"), + HOSTNAME((short)4, "hostname"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // ID + return ID; + case 2: // STATE + return STATE; + case 3: // USER + return USER; + case 4: // HOSTNAME + return HOSTNAME; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + private static final int __ID_ISSET_ID = 0; + private byte __isset_bitfield = 0; + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.ID, new org.apache.thrift.meta_data.FieldMetaData("id", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64))); + tmpMap.put(_Fields.STATE, new org.apache.thrift.meta_data.FieldMetaData("state", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, TxnState.class))); + tmpMap.put(_Fields.USER, new org.apache.thrift.meta_data.FieldMetaData("user", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + tmpMap.put(_Fields.HOSTNAME, new org.apache.thrift.meta_data.FieldMetaData("hostname", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TxnInfo.class, metaDataMap); + } + + public TxnInfo() { + } + + public TxnInfo( + long id, + TxnState state, + String user, + String hostname) + { + this(); + this.id = id; + setIdIsSet(true); + this.state = state; + this.user = user; + this.hostname = hostname; + } + + /** + * Performs a deep copy on other. + */ + public TxnInfo(TxnInfo other) { + __isset_bitfield = other.__isset_bitfield; + this.id = other.id; + if (other.isSetState()) { + this.state = other.state; + } + if (other.isSetUser()) { + this.user = other.user; + } + if (other.isSetHostname()) { + this.hostname = other.hostname; + } + } + + public TxnInfo deepCopy() { + return new TxnInfo(this); + } + + @Override + public void clear() { + setIdIsSet(false); + this.id = 0; + this.state = null; + this.user = null; + this.hostname = null; + } + + public long getId() { + return this.id; + } + + public void setId(long id) { + this.id = id; + setIdIsSet(true); + } + + public void unsetId() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __ID_ISSET_ID); + } + + /** Returns true if field id is set (has been assigned a value) and false otherwise */ + public boolean isSetId() { + return EncodingUtils.testBit(__isset_bitfield, __ID_ISSET_ID); + } + + public void setIdIsSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __ID_ISSET_ID, value); + } + + /** + * + * @see TxnState + */ + public TxnState getState() { + return this.state; + } + + /** + * + * @see TxnState + */ + public void setState(TxnState state) { + this.state = state; + } + + public void unsetState() { + this.state = null; + } + + /** Returns true if field state is set (has been assigned a value) and false otherwise */ + public boolean isSetState() { + return this.state != null; + } + + public void setStateIsSet(boolean value) { + if (!value) { + this.state = null; + } + } + + public String getUser() { + return this.user; + } + + public void setUser(String user) { + this.user = user; + } + + public void unsetUser() { + this.user = null; + } + + /** Returns true if field user is set (has been assigned a value) and false otherwise */ + public boolean isSetUser() { + return this.user != null; + } + + public void setUserIsSet(boolean value) { + if (!value) { + this.user = null; + } + } + + public String getHostname() { + return this.hostname; + } + + public void setHostname(String hostname) { + this.hostname = hostname; + } + + public void unsetHostname() { + this.hostname = null; + } + + /** Returns true if field hostname is set (has been assigned a value) and false otherwise */ + public boolean isSetHostname() { + return this.hostname != null; + } + + public void setHostnameIsSet(boolean value) { + if (!value) { + this.hostname = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case ID: + if (value == null) { + unsetId(); + } else { + setId((Long)value); + } + break; + + case STATE: + if (value == null) { + unsetState(); + } else { + setState((TxnState)value); + } + break; + + case USER: + if (value == null) { + unsetUser(); + } else { + setUser((String)value); + } + break; + + case HOSTNAME: + if (value == null) { + unsetHostname(); + } else { + setHostname((String)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case ID: + return Long.valueOf(getId()); + + case STATE: + return getState(); + + case USER: + return getUser(); + + case HOSTNAME: + return getHostname(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case ID: + return isSetId(); + case STATE: + return isSetState(); + case USER: + return isSetUser(); + case HOSTNAME: + return isSetHostname(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TxnInfo) + return this.equals((TxnInfo)that); + return false; + } + + public boolean equals(TxnInfo that) { + if (that == null) + return false; + + boolean this_present_id = true; + boolean that_present_id = true; + if (this_present_id || that_present_id) { + if (!(this_present_id && that_present_id)) + return false; + if (this.id != that.id) + return false; + } + + boolean this_present_state = true && this.isSetState(); + boolean that_present_state = true && that.isSetState(); + if (this_present_state || that_present_state) { + if (!(this_present_state && that_present_state)) + return false; + if (!this.state.equals(that.state)) + return false; + } + + boolean this_present_user = true && this.isSetUser(); + boolean that_present_user = true && that.isSetUser(); + if (this_present_user || that_present_user) { + if (!(this_present_user && that_present_user)) + return false; + if (!this.user.equals(that.user)) + return false; + } + + boolean this_present_hostname = true && this.isSetHostname(); + boolean that_present_hostname = true && that.isSetHostname(); + if (this_present_hostname || that_present_hostname) { + if (!(this_present_hostname && that_present_hostname)) + return false; + if (!this.hostname.equals(that.hostname)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_id = true; + builder.append(present_id); + if (present_id) + builder.append(id); + + boolean present_state = true && (isSetState()); + builder.append(present_state); + if (present_state) + builder.append(state.getValue()); + + boolean present_user = true && (isSetUser()); + builder.append(present_user); + if (present_user) + builder.append(user); + + boolean present_hostname = true && (isSetHostname()); + builder.append(present_hostname); + if (present_hostname) + builder.append(hostname); + + return builder.toHashCode(); + } + + public int compareTo(TxnInfo other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + TxnInfo typedOther = (TxnInfo)other; + + lastComparison = Boolean.valueOf(isSetId()).compareTo(typedOther.isSetId()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetId()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.id, typedOther.id); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetState()).compareTo(typedOther.isSetState()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetState()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.state, typedOther.state); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetUser()).compareTo(typedOther.isSetUser()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetUser()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.user, typedOther.user); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetHostname()).compareTo(typedOther.isSetHostname()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetHostname()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.hostname, typedOther.hostname); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TxnInfo("); + boolean first = true; + + sb.append("id:"); + sb.append(this.id); + first = false; + if (!first) sb.append(", "); + sb.append("state:"); + if (this.state == null) { + sb.append("null"); + } else { + sb.append(this.state); + } + first = false; + if (!first) sb.append(", "); + sb.append("user:"); + if (this.user == null) { + sb.append("null"); + } else { + sb.append(this.user); + } + first = false; + if (!first) sb.append(", "); + sb.append("hostname:"); + if (this.hostname == null) { + sb.append("null"); + } else { + sb.append(this.hostname); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetId()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'id' is unset! Struct:" + toString()); + } + + if (!isSetState()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'state' is unset! Struct:" + toString()); + } + + if (!isSetUser()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'user' is unset! Struct:" + toString()); + } + + if (!isSetHostname()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'hostname' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. + __isset_bitfield = 0; + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TxnInfoStandardSchemeFactory implements SchemeFactory { + public TxnInfoStandardScheme getScheme() { + return new TxnInfoStandardScheme(); + } + } + + private static class TxnInfoStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TxnInfo struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // ID + if (schemeField.type == org.apache.thrift.protocol.TType.I64) { + struct.id = iprot.readI64(); + struct.setIdIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // STATE + if (schemeField.type == org.apache.thrift.protocol.TType.I32) { + struct.state = TxnState.findByValue(iprot.readI32()); + struct.setStateIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 3: // USER + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.user = iprot.readString(); + struct.setUserIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 4: // HOSTNAME + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.hostname = iprot.readString(); + struct.setHostnameIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TxnInfo struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + oprot.writeFieldBegin(ID_FIELD_DESC); + oprot.writeI64(struct.id); + oprot.writeFieldEnd(); + if (struct.state != null) { + oprot.writeFieldBegin(STATE_FIELD_DESC); + oprot.writeI32(struct.state.getValue()); + oprot.writeFieldEnd(); + } + if (struct.user != null) { + oprot.writeFieldBegin(USER_FIELD_DESC); + oprot.writeString(struct.user); + oprot.writeFieldEnd(); + } + if (struct.hostname != null) { + oprot.writeFieldBegin(HOSTNAME_FIELD_DESC); + oprot.writeString(struct.hostname); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TxnInfoTupleSchemeFactory implements SchemeFactory { + public TxnInfoTupleScheme getScheme() { + return new TxnInfoTupleScheme(); + } + } + + private static class TxnInfoTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TxnInfo struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + oprot.writeI64(struct.id); + oprot.writeI32(struct.state.getValue()); + oprot.writeString(struct.user); + oprot.writeString(struct.hostname); + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TxnInfo struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.id = iprot.readI64(); + struct.setIdIsSet(true); + struct.state = TxnState.findByValue(iprot.readI32()); + struct.setStateIsSet(true); + struct.user = iprot.readString(); + struct.setUserIsSet(true); + struct.hostname = iprot.readString(); + struct.setHostnameIsSet(true); + } + } + +} + diff --git metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TxnOpenException.java metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TxnOpenException.java new file mode 100644 index 0000000..4f5d02d --- /dev/null +++ metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TxnOpenException.java @@ -0,0 +1,391 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hadoop.hive.metastore.api; + +import org.apache.commons.lang.builder.HashCodeBuilder; +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class TxnOpenException extends TException implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TxnOpenException"); + + private static final org.apache.thrift.protocol.TField MESSAGE_FIELD_DESC = new org.apache.thrift.protocol.TField("message", org.apache.thrift.protocol.TType.STRING, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TxnOpenExceptionStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TxnOpenExceptionTupleSchemeFactory()); + } + + private String message; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + MESSAGE((short)1, "message"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // MESSAGE + return MESSAGE; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.MESSAGE, new org.apache.thrift.meta_data.FieldMetaData("message", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TxnOpenException.class, metaDataMap); + } + + public TxnOpenException() { + } + + public TxnOpenException( + String message) + { + this(); + this.message = message; + } + + /** + * Performs a deep copy on other. + */ + public TxnOpenException(TxnOpenException other) { + if (other.isSetMessage()) { + this.message = other.message; + } + } + + public TxnOpenException deepCopy() { + return new TxnOpenException(this); + } + + @Override + public void clear() { + this.message = null; + } + + public String getMessage() { + return this.message; + } + + public void setMessage(String message) { + this.message = message; + } + + public void unsetMessage() { + this.message = null; + } + + /** Returns true if field message is set (has been assigned a value) and false otherwise */ + public boolean isSetMessage() { + return this.message != null; + } + + public void setMessageIsSet(boolean value) { + if (!value) { + this.message = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case MESSAGE: + if (value == null) { + unsetMessage(); + } else { + setMessage((String)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case MESSAGE: + return getMessage(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case MESSAGE: + return isSetMessage(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TxnOpenException) + return this.equals((TxnOpenException)that); + return false; + } + + public boolean equals(TxnOpenException that) { + if (that == null) + return false; + + boolean this_present_message = true && this.isSetMessage(); + boolean that_present_message = true && that.isSetMessage(); + if (this_present_message || that_present_message) { + if (!(this_present_message && that_present_message)) + return false; + if (!this.message.equals(that.message)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_message = true && (isSetMessage()); + builder.append(present_message); + if (present_message) + builder.append(message); + + return builder.toHashCode(); + } + + public int compareTo(TxnOpenException other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + TxnOpenException typedOther = (TxnOpenException)other; + + lastComparison = Boolean.valueOf(isSetMessage()).compareTo(typedOther.isSetMessage()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetMessage()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.message, typedOther.message); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TxnOpenException("); + boolean first = true; + + sb.append("message:"); + if (this.message == null) { + sb.append("null"); + } else { + sb.append(this.message); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TxnOpenExceptionStandardSchemeFactory implements SchemeFactory { + public TxnOpenExceptionStandardScheme getScheme() { + return new TxnOpenExceptionStandardScheme(); + } + } + + private static class TxnOpenExceptionStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TxnOpenException struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // MESSAGE + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.message = iprot.readString(); + struct.setMessageIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TxnOpenException struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.message != null) { + oprot.writeFieldBegin(MESSAGE_FIELD_DESC); + oprot.writeString(struct.message); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TxnOpenExceptionTupleSchemeFactory implements SchemeFactory { + public TxnOpenExceptionTupleScheme getScheme() { + return new TxnOpenExceptionTupleScheme(); + } + } + + private static class TxnOpenExceptionTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TxnOpenException struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetMessage()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetMessage()) { + oprot.writeString(struct.message); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TxnOpenException struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.message = iprot.readString(); + struct.setMessageIsSet(true); + } + } + } + +} + diff --git metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TxnState.java metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TxnState.java new file mode 100644 index 0000000..f4540a1 --- /dev/null +++ metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TxnState.java @@ -0,0 +1,48 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hadoop.hive.metastore.api; + + +import java.util.Map; +import java.util.HashMap; +import org.apache.thrift.TEnum; + +public enum TxnState implements org.apache.thrift.TEnum { + COMMITTED(1), + ABORTED(2), + OPEN(3); + + private final int value; + + private TxnState(int value) { + this.value = value; + } + + /** + * Get the integer value of this enum value, as defined in the Thrift IDL. + */ + public int getValue() { + return value; + } + + /** + * Find a the enum type by its integer value, as defined in the Thrift IDL. + * @return null if the value is not found. + */ + public static TxnState findByValue(int value) { + switch (value) { + case 1: + return COMMITTED; + case 2: + return ABORTED; + case 3: + return OPEN; + default: + return null; + } + } +} diff --git metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Type.java metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Type.java index 1882b57..bb81e3c 100644 --- metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Type.java +++ metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Type.java @@ -618,7 +618,7 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, Type struct) throws struct.fields = new ArrayList(_list0.size); for (int _i1 = 0; _i1 < _list0.size; ++_i1) { - FieldSchema _elem2; // optional + FieldSchema _elem2; // required _elem2 = new FieldSchema(); _elem2.read(iprot); struct.fields.add(_elem2); @@ -749,7 +749,7 @@ public void read(org.apache.thrift.protocol.TProtocol prot, Type struct) throws struct.fields = new ArrayList(_list5.size); for (int _i6 = 0; _i6 < _list5.size; ++_i6) { - FieldSchema _elem7; // optional + FieldSchema _elem7; // required _elem7 = new FieldSchema(); _elem7.read(iprot); struct.fields.add(_elem7); diff --git metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/UnlockRequest.java metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/UnlockRequest.java new file mode 100644 index 0000000..54b949d --- /dev/null +++ metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/UnlockRequest.java @@ -0,0 +1,383 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hadoop.hive.metastore.api; + +import org.apache.commons.lang.builder.HashCodeBuilder; +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class UnlockRequest implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("UnlockRequest"); + + private static final org.apache.thrift.protocol.TField LOCKID_FIELD_DESC = new org.apache.thrift.protocol.TField("lockid", org.apache.thrift.protocol.TType.I64, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new UnlockRequestStandardSchemeFactory()); + schemes.put(TupleScheme.class, new UnlockRequestTupleSchemeFactory()); + } + + private long lockid; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + LOCKID((short)1, "lockid"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // LOCKID + return LOCKID; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + private static final int __LOCKID_ISSET_ID = 0; + private byte __isset_bitfield = 0; + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.LOCKID, new org.apache.thrift.meta_data.FieldMetaData("lockid", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(UnlockRequest.class, metaDataMap); + } + + public UnlockRequest() { + } + + public UnlockRequest( + long lockid) + { + this(); + this.lockid = lockid; + setLockidIsSet(true); + } + + /** + * Performs a deep copy on other. + */ + public UnlockRequest(UnlockRequest other) { + __isset_bitfield = other.__isset_bitfield; + this.lockid = other.lockid; + } + + public UnlockRequest deepCopy() { + return new UnlockRequest(this); + } + + @Override + public void clear() { + setLockidIsSet(false); + this.lockid = 0; + } + + public long getLockid() { + return this.lockid; + } + + public void setLockid(long lockid) { + this.lockid = lockid; + setLockidIsSet(true); + } + + public void unsetLockid() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __LOCKID_ISSET_ID); + } + + /** Returns true if field lockid is set (has been assigned a value) and false otherwise */ + public boolean isSetLockid() { + return EncodingUtils.testBit(__isset_bitfield, __LOCKID_ISSET_ID); + } + + public void setLockidIsSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __LOCKID_ISSET_ID, value); + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case LOCKID: + if (value == null) { + unsetLockid(); + } else { + setLockid((Long)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case LOCKID: + return Long.valueOf(getLockid()); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case LOCKID: + return isSetLockid(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof UnlockRequest) + return this.equals((UnlockRequest)that); + return false; + } + + public boolean equals(UnlockRequest that) { + if (that == null) + return false; + + boolean this_present_lockid = true; + boolean that_present_lockid = true; + if (this_present_lockid || that_present_lockid) { + if (!(this_present_lockid && that_present_lockid)) + return false; + if (this.lockid != that.lockid) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_lockid = true; + builder.append(present_lockid); + if (present_lockid) + builder.append(lockid); + + return builder.toHashCode(); + } + + public int compareTo(UnlockRequest other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + UnlockRequest typedOther = (UnlockRequest)other; + + lastComparison = Boolean.valueOf(isSetLockid()).compareTo(typedOther.isSetLockid()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetLockid()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.lockid, typedOther.lockid); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("UnlockRequest("); + boolean first = true; + + sb.append("lockid:"); + sb.append(this.lockid); + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetLockid()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'lockid' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. + __isset_bitfield = 0; + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class UnlockRequestStandardSchemeFactory implements SchemeFactory { + public UnlockRequestStandardScheme getScheme() { + return new UnlockRequestStandardScheme(); + } + } + + private static class UnlockRequestStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, UnlockRequest struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // LOCKID + if (schemeField.type == org.apache.thrift.protocol.TType.I64) { + struct.lockid = iprot.readI64(); + struct.setLockidIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, UnlockRequest struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + oprot.writeFieldBegin(LOCKID_FIELD_DESC); + oprot.writeI64(struct.lockid); + oprot.writeFieldEnd(); + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class UnlockRequestTupleSchemeFactory implements SchemeFactory { + public UnlockRequestTupleScheme getScheme() { + return new UnlockRequestTupleScheme(); + } + } + + private static class UnlockRequestTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, UnlockRequest struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + oprot.writeI64(struct.lockid); + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, UnlockRequest struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.lockid = iprot.readI64(); + struct.setLockidIsSet(true); + } + } + +} + diff --git metastore/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php metastore/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php index be3329a..1dfbc42 100644 --- metastore/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php +++ metastore/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php @@ -108,6 +108,18 @@ interface ThriftHiveMetastoreIf extends \FacebookServiceIf { public function get_delegation_token($token_owner, $renewer_kerberos_principal_name); public function renew_delegation_token($token_str_form); public function cancel_delegation_token($token_str_form); + public function get_open_txns(); + public function get_open_txns_info(); + public function open_txns(\metastore\OpenTxnRequest $rqst); + public function abort_txn(\metastore\AbortTxnRequest $rqst); + public function commit_txn(\metastore\CommitTxnRequest $rqst); + public function lock(\metastore\LockRequest $rqst); + public function check_lock(\metastore\CheckLockRequest $rqst); + public function unlock(\metastore\UnlockRequest $rqst); + public function show_locks(\metastore\ShowLocksRequest $rqst); + public function heartbeat(\metastore\HeartbeatRequest $ids); + public function compact(\metastore\CompactionRequest $rqst); + public function show_compact(\metastore\ShowCompactRequest $rqst); } class ThriftHiveMetastoreClient extends \FacebookServiceClient implements \metastore\ThriftHiveMetastoreIf { @@ -5496,34 +5508,3155 @@ class ThriftHiveMetastoreClient extends \FacebookServiceClient implements \metas return; } + public function get_open_txns() + { + $this->send_get_open_txns(); + return $this->recv_get_open_txns(); + } + + public function send_get_open_txns() + { + $args = new \metastore\ThriftHiveMetastore_get_open_txns_args(); + $bin_accel = ($this->output_ instanceof TProtocol::$TBINARYPROTOCOLACCELERATED) && function_exists('thrift_protocol_write_binary'); + if ($bin_accel) + { + thrift_protocol_write_binary($this->output_, 'get_open_txns', TMessageType::CALL, $args, $this->seqid_, $this->output_->isStrictWrite()); + } + else + { + $this->output_->writeMessageBegin('get_open_txns', TMessageType::CALL, $this->seqid_); + $args->write($this->output_); + $this->output_->writeMessageEnd(); + $this->output_->getTransport()->flush(); + } + } + + public function recv_get_open_txns() + { + $bin_accel = ($this->input_ instanceof TProtocol::$TBINARYPROTOCOLACCELERATED) && function_exists('thrift_protocol_read_binary'); + if ($bin_accel) $result = thrift_protocol_read_binary($this->input_, '\metastore\ThriftHiveMetastore_get_open_txns_result', $this->input_->isStrictRead()); + else + { + $rseqid = 0; + $fname = null; + $mtype = 0; + + $this->input_->readMessageBegin($fname, $mtype, $rseqid); + if ($mtype == TMessageType::EXCEPTION) { + $x = new TApplicationException(); + $x->read($this->input_); + $this->input_->readMessageEnd(); + throw $x; + } + $result = new \metastore\ThriftHiveMetastore_get_open_txns_result(); + $result->read($this->input_); + $this->input_->readMessageEnd(); + } + if ($result->success !== null) { + return $result->success; + } + throw new \Exception("get_open_txns failed: unknown result"); + } + + public function get_open_txns_info() + { + $this->send_get_open_txns_info(); + return $this->recv_get_open_txns_info(); + } + + public function send_get_open_txns_info() + { + $args = new \metastore\ThriftHiveMetastore_get_open_txns_info_args(); + $bin_accel = ($this->output_ instanceof TProtocol::$TBINARYPROTOCOLACCELERATED) && function_exists('thrift_protocol_write_binary'); + if ($bin_accel) + { + thrift_protocol_write_binary($this->output_, 'get_open_txns_info', TMessageType::CALL, $args, $this->seqid_, $this->output_->isStrictWrite()); + } + else + { + $this->output_->writeMessageBegin('get_open_txns_info', TMessageType::CALL, $this->seqid_); + $args->write($this->output_); + $this->output_->writeMessageEnd(); + $this->output_->getTransport()->flush(); + } + } + + public function recv_get_open_txns_info() + { + $bin_accel = ($this->input_ instanceof TProtocol::$TBINARYPROTOCOLACCELERATED) && function_exists('thrift_protocol_read_binary'); + if ($bin_accel) $result = thrift_protocol_read_binary($this->input_, '\metastore\ThriftHiveMetastore_get_open_txns_info_result', $this->input_->isStrictRead()); + else + { + $rseqid = 0; + $fname = null; + $mtype = 0; + + $this->input_->readMessageBegin($fname, $mtype, $rseqid); + if ($mtype == TMessageType::EXCEPTION) { + $x = new TApplicationException(); + $x->read($this->input_); + $this->input_->readMessageEnd(); + throw $x; + } + $result = new \metastore\ThriftHiveMetastore_get_open_txns_info_result(); + $result->read($this->input_); + $this->input_->readMessageEnd(); + } + if ($result->success !== null) { + return $result->success; + } + throw new \Exception("get_open_txns_info failed: unknown result"); + } + + public function open_txns(\metastore\OpenTxnRequest $rqst) + { + $this->send_open_txns($rqst); + return $this->recv_open_txns(); + } + + public function send_open_txns(\metastore\OpenTxnRequest $rqst) + { + $args = new \metastore\ThriftHiveMetastore_open_txns_args(); + $args->rqst = $rqst; + $bin_accel = ($this->output_ instanceof TProtocol::$TBINARYPROTOCOLACCELERATED) && function_exists('thrift_protocol_write_binary'); + if ($bin_accel) + { + thrift_protocol_write_binary($this->output_, 'open_txns', TMessageType::CALL, $args, $this->seqid_, $this->output_->isStrictWrite()); + } + else + { + $this->output_->writeMessageBegin('open_txns', TMessageType::CALL, $this->seqid_); + $args->write($this->output_); + $this->output_->writeMessageEnd(); + $this->output_->getTransport()->flush(); + } + } + + public function recv_open_txns() + { + $bin_accel = ($this->input_ instanceof TProtocol::$TBINARYPROTOCOLACCELERATED) && function_exists('thrift_protocol_read_binary'); + if ($bin_accel) $result = thrift_protocol_read_binary($this->input_, '\metastore\ThriftHiveMetastore_open_txns_result', $this->input_->isStrictRead()); + else + { + $rseqid = 0; + $fname = null; + $mtype = 0; + + $this->input_->readMessageBegin($fname, $mtype, $rseqid); + if ($mtype == TMessageType::EXCEPTION) { + $x = new TApplicationException(); + $x->read($this->input_); + $this->input_->readMessageEnd(); + throw $x; + } + $result = new \metastore\ThriftHiveMetastore_open_txns_result(); + $result->read($this->input_); + $this->input_->readMessageEnd(); + } + if ($result->success !== null) { + return $result->success; + } + throw new \Exception("open_txns failed: unknown result"); + } + + public function abort_txn(\metastore\AbortTxnRequest $rqst) + { + $this->send_abort_txn($rqst); + $this->recv_abort_txn(); + } + + public function send_abort_txn(\metastore\AbortTxnRequest $rqst) + { + $args = new \metastore\ThriftHiveMetastore_abort_txn_args(); + $args->rqst = $rqst; + $bin_accel = ($this->output_ instanceof TProtocol::$TBINARYPROTOCOLACCELERATED) && function_exists('thrift_protocol_write_binary'); + if ($bin_accel) + { + thrift_protocol_write_binary($this->output_, 'abort_txn', TMessageType::CALL, $args, $this->seqid_, $this->output_->isStrictWrite()); + } + else + { + $this->output_->writeMessageBegin('abort_txn', TMessageType::CALL, $this->seqid_); + $args->write($this->output_); + $this->output_->writeMessageEnd(); + $this->output_->getTransport()->flush(); + } + } + + public function recv_abort_txn() + { + $bin_accel = ($this->input_ instanceof TProtocol::$TBINARYPROTOCOLACCELERATED) && function_exists('thrift_protocol_read_binary'); + if ($bin_accel) $result = thrift_protocol_read_binary($this->input_, '\metastore\ThriftHiveMetastore_abort_txn_result', $this->input_->isStrictRead()); + else + { + $rseqid = 0; + $fname = null; + $mtype = 0; + + $this->input_->readMessageBegin($fname, $mtype, $rseqid); + if ($mtype == TMessageType::EXCEPTION) { + $x = new TApplicationException(); + $x->read($this->input_); + $this->input_->readMessageEnd(); + throw $x; + } + $result = new \metastore\ThriftHiveMetastore_abort_txn_result(); + $result->read($this->input_); + $this->input_->readMessageEnd(); + } + if ($result->o1 !== null) { + throw $result->o1; + } + return; + } + + public function commit_txn(\metastore\CommitTxnRequest $rqst) + { + $this->send_commit_txn($rqst); + $this->recv_commit_txn(); + } + + public function send_commit_txn(\metastore\CommitTxnRequest $rqst) + { + $args = new \metastore\ThriftHiveMetastore_commit_txn_args(); + $args->rqst = $rqst; + $bin_accel = ($this->output_ instanceof TProtocol::$TBINARYPROTOCOLACCELERATED) && function_exists('thrift_protocol_write_binary'); + if ($bin_accel) + { + thrift_protocol_write_binary($this->output_, 'commit_txn', TMessageType::CALL, $args, $this->seqid_, $this->output_->isStrictWrite()); + } + else + { + $this->output_->writeMessageBegin('commit_txn', TMessageType::CALL, $this->seqid_); + $args->write($this->output_); + $this->output_->writeMessageEnd(); + $this->output_->getTransport()->flush(); + } + } + + public function recv_commit_txn() + { + $bin_accel = ($this->input_ instanceof TProtocol::$TBINARYPROTOCOLACCELERATED) && function_exists('thrift_protocol_read_binary'); + if ($bin_accel) $result = thrift_protocol_read_binary($this->input_, '\metastore\ThriftHiveMetastore_commit_txn_result', $this->input_->isStrictRead()); + else + { + $rseqid = 0; + $fname = null; + $mtype = 0; + + $this->input_->readMessageBegin($fname, $mtype, $rseqid); + if ($mtype == TMessageType::EXCEPTION) { + $x = new TApplicationException(); + $x->read($this->input_); + $this->input_->readMessageEnd(); + throw $x; + } + $result = new \metastore\ThriftHiveMetastore_commit_txn_result(); + $result->read($this->input_); + $this->input_->readMessageEnd(); + } + if ($result->o1 !== null) { + throw $result->o1; + } + if ($result->o2 !== null) { + throw $result->o2; + } + return; + } + + public function lock(\metastore\LockRequest $rqst) + { + $this->send_lock($rqst); + return $this->recv_lock(); + } + + public function send_lock(\metastore\LockRequest $rqst) + { + $args = new \metastore\ThriftHiveMetastore_lock_args(); + $args->rqst = $rqst; + $bin_accel = ($this->output_ instanceof TProtocol::$TBINARYPROTOCOLACCELERATED) && function_exists('thrift_protocol_write_binary'); + if ($bin_accel) + { + thrift_protocol_write_binary($this->output_, 'lock', TMessageType::CALL, $args, $this->seqid_, $this->output_->isStrictWrite()); + } + else + { + $this->output_->writeMessageBegin('lock', TMessageType::CALL, $this->seqid_); + $args->write($this->output_); + $this->output_->writeMessageEnd(); + $this->output_->getTransport()->flush(); + } + } + + public function recv_lock() + { + $bin_accel = ($this->input_ instanceof TProtocol::$TBINARYPROTOCOLACCELERATED) && function_exists('thrift_protocol_read_binary'); + if ($bin_accel) $result = thrift_protocol_read_binary($this->input_, '\metastore\ThriftHiveMetastore_lock_result', $this->input_->isStrictRead()); + else + { + $rseqid = 0; + $fname = null; + $mtype = 0; + + $this->input_->readMessageBegin($fname, $mtype, $rseqid); + if ($mtype == TMessageType::EXCEPTION) { + $x = new TApplicationException(); + $x->read($this->input_); + $this->input_->readMessageEnd(); + throw $x; + } + $result = new \metastore\ThriftHiveMetastore_lock_result(); + $result->read($this->input_); + $this->input_->readMessageEnd(); + } + if ($result->success !== null) { + return $result->success; + } + if ($result->o1 !== null) { + throw $result->o1; + } + if ($result->o2 !== null) { + throw $result->o2; + } + throw new \Exception("lock failed: unknown result"); + } + + public function check_lock(\metastore\CheckLockRequest $rqst) + { + $this->send_check_lock($rqst); + return $this->recv_check_lock(); + } + + public function send_check_lock(\metastore\CheckLockRequest $rqst) + { + $args = new \metastore\ThriftHiveMetastore_check_lock_args(); + $args->rqst = $rqst; + $bin_accel = ($this->output_ instanceof TProtocol::$TBINARYPROTOCOLACCELERATED) && function_exists('thrift_protocol_write_binary'); + if ($bin_accel) + { + thrift_protocol_write_binary($this->output_, 'check_lock', TMessageType::CALL, $args, $this->seqid_, $this->output_->isStrictWrite()); + } + else + { + $this->output_->writeMessageBegin('check_lock', TMessageType::CALL, $this->seqid_); + $args->write($this->output_); + $this->output_->writeMessageEnd(); + $this->output_->getTransport()->flush(); + } + } + + public function recv_check_lock() + { + $bin_accel = ($this->input_ instanceof TProtocol::$TBINARYPROTOCOLACCELERATED) && function_exists('thrift_protocol_read_binary'); + if ($bin_accel) $result = thrift_protocol_read_binary($this->input_, '\metastore\ThriftHiveMetastore_check_lock_result', $this->input_->isStrictRead()); + else + { + $rseqid = 0; + $fname = null; + $mtype = 0; + + $this->input_->readMessageBegin($fname, $mtype, $rseqid); + if ($mtype == TMessageType::EXCEPTION) { + $x = new TApplicationException(); + $x->read($this->input_); + $this->input_->readMessageEnd(); + throw $x; + } + $result = new \metastore\ThriftHiveMetastore_check_lock_result(); + $result->read($this->input_); + $this->input_->readMessageEnd(); + } + if ($result->success !== null) { + return $result->success; + } + if ($result->o1 !== null) { + throw $result->o1; + } + if ($result->o2 !== null) { + throw $result->o2; + } + if ($result->o3 !== null) { + throw $result->o3; + } + throw new \Exception("check_lock failed: unknown result"); + } + + public function unlock(\metastore\UnlockRequest $rqst) + { + $this->send_unlock($rqst); + $this->recv_unlock(); + } + + public function send_unlock(\metastore\UnlockRequest $rqst) + { + $args = new \metastore\ThriftHiveMetastore_unlock_args(); + $args->rqst = $rqst; + $bin_accel = ($this->output_ instanceof TProtocol::$TBINARYPROTOCOLACCELERATED) && function_exists('thrift_protocol_write_binary'); + if ($bin_accel) + { + thrift_protocol_write_binary($this->output_, 'unlock', TMessageType::CALL, $args, $this->seqid_, $this->output_->isStrictWrite()); + } + else + { + $this->output_->writeMessageBegin('unlock', TMessageType::CALL, $this->seqid_); + $args->write($this->output_); + $this->output_->writeMessageEnd(); + $this->output_->getTransport()->flush(); + } + } + + public function recv_unlock() + { + $bin_accel = ($this->input_ instanceof TProtocol::$TBINARYPROTOCOLACCELERATED) && function_exists('thrift_protocol_read_binary'); + if ($bin_accel) $result = thrift_protocol_read_binary($this->input_, '\metastore\ThriftHiveMetastore_unlock_result', $this->input_->isStrictRead()); + else + { + $rseqid = 0; + $fname = null; + $mtype = 0; + + $this->input_->readMessageBegin($fname, $mtype, $rseqid); + if ($mtype == TMessageType::EXCEPTION) { + $x = new TApplicationException(); + $x->read($this->input_); + $this->input_->readMessageEnd(); + throw $x; + } + $result = new \metastore\ThriftHiveMetastore_unlock_result(); + $result->read($this->input_); + $this->input_->readMessageEnd(); + } + if ($result->o1 !== null) { + throw $result->o1; + } + if ($result->o2 !== null) { + throw $result->o2; + } + return; + } + + public function show_locks(\metastore\ShowLocksRequest $rqst) + { + $this->send_show_locks($rqst); + return $this->recv_show_locks(); + } + + public function send_show_locks(\metastore\ShowLocksRequest $rqst) + { + $args = new \metastore\ThriftHiveMetastore_show_locks_args(); + $args->rqst = $rqst; + $bin_accel = ($this->output_ instanceof TProtocol::$TBINARYPROTOCOLACCELERATED) && function_exists('thrift_protocol_write_binary'); + if ($bin_accel) + { + thrift_protocol_write_binary($this->output_, 'show_locks', TMessageType::CALL, $args, $this->seqid_, $this->output_->isStrictWrite()); + } + else + { + $this->output_->writeMessageBegin('show_locks', TMessageType::CALL, $this->seqid_); + $args->write($this->output_); + $this->output_->writeMessageEnd(); + $this->output_->getTransport()->flush(); + } + } + + public function recv_show_locks() + { + $bin_accel = ($this->input_ instanceof TProtocol::$TBINARYPROTOCOLACCELERATED) && function_exists('thrift_protocol_read_binary'); + if ($bin_accel) $result = thrift_protocol_read_binary($this->input_, '\metastore\ThriftHiveMetastore_show_locks_result', $this->input_->isStrictRead()); + else + { + $rseqid = 0; + $fname = null; + $mtype = 0; + + $this->input_->readMessageBegin($fname, $mtype, $rseqid); + if ($mtype == TMessageType::EXCEPTION) { + $x = new TApplicationException(); + $x->read($this->input_); + $this->input_->readMessageEnd(); + throw $x; + } + $result = new \metastore\ThriftHiveMetastore_show_locks_result(); + $result->read($this->input_); + $this->input_->readMessageEnd(); + } + if ($result->success !== null) { + return $result->success; + } + throw new \Exception("show_locks failed: unknown result"); + } + + public function heartbeat(\metastore\HeartbeatRequest $ids) + { + $this->send_heartbeat($ids); + $this->recv_heartbeat(); + } + + public function send_heartbeat(\metastore\HeartbeatRequest $ids) + { + $args = new \metastore\ThriftHiveMetastore_heartbeat_args(); + $args->ids = $ids; + $bin_accel = ($this->output_ instanceof TProtocol::$TBINARYPROTOCOLACCELERATED) && function_exists('thrift_protocol_write_binary'); + if ($bin_accel) + { + thrift_protocol_write_binary($this->output_, 'heartbeat', TMessageType::CALL, $args, $this->seqid_, $this->output_->isStrictWrite()); + } + else + { + $this->output_->writeMessageBegin('heartbeat', TMessageType::CALL, $this->seqid_); + $args->write($this->output_); + $this->output_->writeMessageEnd(); + $this->output_->getTransport()->flush(); + } + } + + public function recv_heartbeat() + { + $bin_accel = ($this->input_ instanceof TProtocol::$TBINARYPROTOCOLACCELERATED) && function_exists('thrift_protocol_read_binary'); + if ($bin_accel) $result = thrift_protocol_read_binary($this->input_, '\metastore\ThriftHiveMetastore_heartbeat_result', $this->input_->isStrictRead()); + else + { + $rseqid = 0; + $fname = null; + $mtype = 0; + + $this->input_->readMessageBegin($fname, $mtype, $rseqid); + if ($mtype == TMessageType::EXCEPTION) { + $x = new TApplicationException(); + $x->read($this->input_); + $this->input_->readMessageEnd(); + throw $x; + } + $result = new \metastore\ThriftHiveMetastore_heartbeat_result(); + $result->read($this->input_); + $this->input_->readMessageEnd(); + } + if ($result->o1 !== null) { + throw $result->o1; + } + if ($result->o2 !== null) { + throw $result->o2; + } + if ($result->o3 !== null) { + throw $result->o3; + } + return; + } + + public function compact(\metastore\CompactionRequest $rqst) + { + $this->send_compact($rqst); + $this->recv_compact(); + } + + public function send_compact(\metastore\CompactionRequest $rqst) + { + $args = new \metastore\ThriftHiveMetastore_compact_args(); + $args->rqst = $rqst; + $bin_accel = ($this->output_ instanceof TProtocol::$TBINARYPROTOCOLACCELERATED) && function_exists('thrift_protocol_write_binary'); + if ($bin_accel) + { + thrift_protocol_write_binary($this->output_, 'compact', TMessageType::CALL, $args, $this->seqid_, $this->output_->isStrictWrite()); + } + else + { + $this->output_->writeMessageBegin('compact', TMessageType::CALL, $this->seqid_); + $args->write($this->output_); + $this->output_->writeMessageEnd(); + $this->output_->getTransport()->flush(); + } + } + + public function recv_compact() + { + $bin_accel = ($this->input_ instanceof TProtocol::$TBINARYPROTOCOLACCELERATED) && function_exists('thrift_protocol_read_binary'); + if ($bin_accel) $result = thrift_protocol_read_binary($this->input_, '\metastore\ThriftHiveMetastore_compact_result', $this->input_->isStrictRead()); + else + { + $rseqid = 0; + $fname = null; + $mtype = 0; + + $this->input_->readMessageBegin($fname, $mtype, $rseqid); + if ($mtype == TMessageType::EXCEPTION) { + $x = new TApplicationException(); + $x->read($this->input_); + $this->input_->readMessageEnd(); + throw $x; + } + $result = new \metastore\ThriftHiveMetastore_compact_result(); + $result->read($this->input_); + $this->input_->readMessageEnd(); + } + return; + } + + public function show_compact(\metastore\ShowCompactRequest $rqst) + { + $this->send_show_compact($rqst); + return $this->recv_show_compact(); + } + + public function send_show_compact(\metastore\ShowCompactRequest $rqst) + { + $args = new \metastore\ThriftHiveMetastore_show_compact_args(); + $args->rqst = $rqst; + $bin_accel = ($this->output_ instanceof TProtocol::$TBINARYPROTOCOLACCELERATED) && function_exists('thrift_protocol_write_binary'); + if ($bin_accel) + { + thrift_protocol_write_binary($this->output_, 'show_compact', TMessageType::CALL, $args, $this->seqid_, $this->output_->isStrictWrite()); + } + else + { + $this->output_->writeMessageBegin('show_compact', TMessageType::CALL, $this->seqid_); + $args->write($this->output_); + $this->output_->writeMessageEnd(); + $this->output_->getTransport()->flush(); + } + } + + public function recv_show_compact() + { + $bin_accel = ($this->input_ instanceof TProtocol::$TBINARYPROTOCOLACCELERATED) && function_exists('thrift_protocol_read_binary'); + if ($bin_accel) $result = thrift_protocol_read_binary($this->input_, '\metastore\ThriftHiveMetastore_show_compact_result', $this->input_->isStrictRead()); + else + { + $rseqid = 0; + $fname = null; + $mtype = 0; + + $this->input_->readMessageBegin($fname, $mtype, $rseqid); + if ($mtype == TMessageType::EXCEPTION) { + $x = new TApplicationException(); + $x->read($this->input_); + $this->input_->readMessageEnd(); + throw $x; + } + $result = new \metastore\ThriftHiveMetastore_show_compact_result(); + $result->read($this->input_); + $this->input_->readMessageEnd(); + } + if ($result->success !== null) { + return $result->success; + } + throw new \Exception("show_compact failed: unknown result"); + } + +} + +// HELPER FUNCTIONS AND STRUCTURES + +class ThriftHiveMetastore_create_database_args { + static $_TSPEC; + + public $database = null; + + public function __construct($vals=null) { + if (!isset(self::$_TSPEC)) { + self::$_TSPEC = array( + 1 => array( + 'var' => 'database', + 'type' => TType::STRUCT, + 'class' => '\metastore\Database', + ), + ); + } + if (is_array($vals)) { + if (isset($vals['database'])) { + $this->database = $vals['database']; + } + } + } + + public function getName() { + return 'ThriftHiveMetastore_create_database_args'; + } + + public function read($input) + { + $xfer = 0; + $fname = null; + $ftype = 0; + $fid = 0; + $xfer += $input->readStructBegin($fname); + while (true) + { + $xfer += $input->readFieldBegin($fname, $ftype, $fid); + if ($ftype == TType::STOP) { + break; + } + switch ($fid) + { + case 1: + if ($ftype == TType::STRUCT) { + $this->database = new \metastore\Database(); + $xfer += $this->database->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; + default: + $xfer += $input->skip($ftype); + break; + } + $xfer += $input->readFieldEnd(); + } + $xfer += $input->readStructEnd(); + return $xfer; + } + + public function write($output) { + $xfer = 0; + $xfer += $output->writeStructBegin('ThriftHiveMetastore_create_database_args'); + if ($this->database !== null) { + if (!is_object($this->database)) { + throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); + } + $xfer += $output->writeFieldBegin('database', TType::STRUCT, 1); + $xfer += $this->database->write($output); + $xfer += $output->writeFieldEnd(); + } + $xfer += $output->writeFieldStop(); + $xfer += $output->writeStructEnd(); + return $xfer; + } + +} + +class ThriftHiveMetastore_create_database_result { + static $_TSPEC; + + public $o1 = null; + public $o2 = null; + public $o3 = null; + + public function __construct($vals=null) { + if (!isset(self::$_TSPEC)) { + self::$_TSPEC = array( + 1 => array( + 'var' => 'o1', + 'type' => TType::STRUCT, + 'class' => '\metastore\AlreadyExistsException', + ), + 2 => array( + 'var' => 'o2', + 'type' => TType::STRUCT, + 'class' => '\metastore\InvalidObjectException', + ), + 3 => array( + 'var' => 'o3', + 'type' => TType::STRUCT, + 'class' => '\metastore\MetaException', + ), + ); + } + if (is_array($vals)) { + if (isset($vals['o1'])) { + $this->o1 = $vals['o1']; + } + if (isset($vals['o2'])) { + $this->o2 = $vals['o2']; + } + if (isset($vals['o3'])) { + $this->o3 = $vals['o3']; + } + } + } + + public function getName() { + return 'ThriftHiveMetastore_create_database_result'; + } + + public function read($input) + { + $xfer = 0; + $fname = null; + $ftype = 0; + $fid = 0; + $xfer += $input->readStructBegin($fname); + while (true) + { + $xfer += $input->readFieldBegin($fname, $ftype, $fid); + if ($ftype == TType::STOP) { + break; + } + switch ($fid) + { + case 1: + if ($ftype == TType::STRUCT) { + $this->o1 = new \metastore\AlreadyExistsException(); + $xfer += $this->o1->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; + case 2: + if ($ftype == TType::STRUCT) { + $this->o2 = new \metastore\InvalidObjectException(); + $xfer += $this->o2->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; + case 3: + if ($ftype == TType::STRUCT) { + $this->o3 = new \metastore\MetaException(); + $xfer += $this->o3->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; + default: + $xfer += $input->skip($ftype); + break; + } + $xfer += $input->readFieldEnd(); + } + $xfer += $input->readStructEnd(); + return $xfer; + } + + public function write($output) { + $xfer = 0; + $xfer += $output->writeStructBegin('ThriftHiveMetastore_create_database_result'); + if ($this->o1 !== null) { + $xfer += $output->writeFieldBegin('o1', TType::STRUCT, 1); + $xfer += $this->o1->write($output); + $xfer += $output->writeFieldEnd(); + } + if ($this->o2 !== null) { + $xfer += $output->writeFieldBegin('o2', TType::STRUCT, 2); + $xfer += $this->o2->write($output); + $xfer += $output->writeFieldEnd(); + } + if ($this->o3 !== null) { + $xfer += $output->writeFieldBegin('o3', TType::STRUCT, 3); + $xfer += $this->o3->write($output); + $xfer += $output->writeFieldEnd(); + } + $xfer += $output->writeFieldStop(); + $xfer += $output->writeStructEnd(); + return $xfer; + } + +} + +class ThriftHiveMetastore_get_database_args { + static $_TSPEC; + + public $name = null; + + public function __construct($vals=null) { + if (!isset(self::$_TSPEC)) { + self::$_TSPEC = array( + 1 => array( + 'var' => 'name', + 'type' => TType::STRING, + ), + ); + } + if (is_array($vals)) { + if (isset($vals['name'])) { + $this->name = $vals['name']; + } + } + } + + public function getName() { + return 'ThriftHiveMetastore_get_database_args'; + } + + public function read($input) + { + $xfer = 0; + $fname = null; + $ftype = 0; + $fid = 0; + $xfer += $input->readStructBegin($fname); + while (true) + { + $xfer += $input->readFieldBegin($fname, $ftype, $fid); + if ($ftype == TType::STOP) { + break; + } + switch ($fid) + { + case 1: + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->name); + } else { + $xfer += $input->skip($ftype); + } + break; + default: + $xfer += $input->skip($ftype); + break; + } + $xfer += $input->readFieldEnd(); + } + $xfer += $input->readStructEnd(); + return $xfer; + } + + public function write($output) { + $xfer = 0; + $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_database_args'); + if ($this->name !== null) { + $xfer += $output->writeFieldBegin('name', TType::STRING, 1); + $xfer += $output->writeString($this->name); + $xfer += $output->writeFieldEnd(); + } + $xfer += $output->writeFieldStop(); + $xfer += $output->writeStructEnd(); + return $xfer; + } + +} + +class ThriftHiveMetastore_get_database_result { + static $_TSPEC; + + public $success = null; + public $o1 = null; + public $o2 = null; + + public function __construct($vals=null) { + if (!isset(self::$_TSPEC)) { + self::$_TSPEC = array( + 0 => array( + 'var' => 'success', + 'type' => TType::STRUCT, + 'class' => '\metastore\Database', + ), + 1 => array( + 'var' => 'o1', + 'type' => TType::STRUCT, + 'class' => '\metastore\NoSuchObjectException', + ), + 2 => array( + 'var' => 'o2', + 'type' => TType::STRUCT, + 'class' => '\metastore\MetaException', + ), + ); + } + if (is_array($vals)) { + if (isset($vals['success'])) { + $this->success = $vals['success']; + } + if (isset($vals['o1'])) { + $this->o1 = $vals['o1']; + } + if (isset($vals['o2'])) { + $this->o2 = $vals['o2']; + } + } + } + + public function getName() { + return 'ThriftHiveMetastore_get_database_result'; + } + + public function read($input) + { + $xfer = 0; + $fname = null; + $ftype = 0; + $fid = 0; + $xfer += $input->readStructBegin($fname); + while (true) + { + $xfer += $input->readFieldBegin($fname, $ftype, $fid); + if ($ftype == TType::STOP) { + break; + } + switch ($fid) + { + case 0: + if ($ftype == TType::STRUCT) { + $this->success = new \metastore\Database(); + $xfer += $this->success->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; + case 1: + if ($ftype == TType::STRUCT) { + $this->o1 = new \metastore\NoSuchObjectException(); + $xfer += $this->o1->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; + case 2: + if ($ftype == TType::STRUCT) { + $this->o2 = new \metastore\MetaException(); + $xfer += $this->o2->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; + default: + $xfer += $input->skip($ftype); + break; + } + $xfer += $input->readFieldEnd(); + } + $xfer += $input->readStructEnd(); + return $xfer; + } + + public function write($output) { + $xfer = 0; + $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_database_result'); + if ($this->success !== null) { + if (!is_object($this->success)) { + throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); + } + $xfer += $output->writeFieldBegin('success', TType::STRUCT, 0); + $xfer += $this->success->write($output); + $xfer += $output->writeFieldEnd(); + } + if ($this->o1 !== null) { + $xfer += $output->writeFieldBegin('o1', TType::STRUCT, 1); + $xfer += $this->o1->write($output); + $xfer += $output->writeFieldEnd(); + } + if ($this->o2 !== null) { + $xfer += $output->writeFieldBegin('o2', TType::STRUCT, 2); + $xfer += $this->o2->write($output); + $xfer += $output->writeFieldEnd(); + } + $xfer += $output->writeFieldStop(); + $xfer += $output->writeStructEnd(); + return $xfer; + } + +} + +class ThriftHiveMetastore_drop_database_args { + static $_TSPEC; + + public $name = null; + public $deleteData = null; + public $cascade = null; + + public function __construct($vals=null) { + if (!isset(self::$_TSPEC)) { + self::$_TSPEC = array( + 1 => array( + 'var' => 'name', + 'type' => TType::STRING, + ), + 2 => array( + 'var' => 'deleteData', + 'type' => TType::BOOL, + ), + 3 => array( + 'var' => 'cascade', + 'type' => TType::BOOL, + ), + ); + } + if (is_array($vals)) { + if (isset($vals['name'])) { + $this->name = $vals['name']; + } + if (isset($vals['deleteData'])) { + $this->deleteData = $vals['deleteData']; + } + if (isset($vals['cascade'])) { + $this->cascade = $vals['cascade']; + } + } + } + + public function getName() { + return 'ThriftHiveMetastore_drop_database_args'; + } + + public function read($input) + { + $xfer = 0; + $fname = null; + $ftype = 0; + $fid = 0; + $xfer += $input->readStructBegin($fname); + while (true) + { + $xfer += $input->readFieldBegin($fname, $ftype, $fid); + if ($ftype == TType::STOP) { + break; + } + switch ($fid) + { + case 1: + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->name); + } else { + $xfer += $input->skip($ftype); + } + break; + case 2: + if ($ftype == TType::BOOL) { + $xfer += $input->readBool($this->deleteData); + } else { + $xfer += $input->skip($ftype); + } + break; + case 3: + if ($ftype == TType::BOOL) { + $xfer += $input->readBool($this->cascade); + } else { + $xfer += $input->skip($ftype); + } + break; + default: + $xfer += $input->skip($ftype); + break; + } + $xfer += $input->readFieldEnd(); + } + $xfer += $input->readStructEnd(); + return $xfer; + } + + public function write($output) { + $xfer = 0; + $xfer += $output->writeStructBegin('ThriftHiveMetastore_drop_database_args'); + if ($this->name !== null) { + $xfer += $output->writeFieldBegin('name', TType::STRING, 1); + $xfer += $output->writeString($this->name); + $xfer += $output->writeFieldEnd(); + } + if ($this->deleteData !== null) { + $xfer += $output->writeFieldBegin('deleteData', TType::BOOL, 2); + $xfer += $output->writeBool($this->deleteData); + $xfer += $output->writeFieldEnd(); + } + if ($this->cascade !== null) { + $xfer += $output->writeFieldBegin('cascade', TType::BOOL, 3); + $xfer += $output->writeBool($this->cascade); + $xfer += $output->writeFieldEnd(); + } + $xfer += $output->writeFieldStop(); + $xfer += $output->writeStructEnd(); + return $xfer; + } + +} + +class ThriftHiveMetastore_drop_database_result { + static $_TSPEC; + + public $o1 = null; + public $o2 = null; + public $o3 = null; + + public function __construct($vals=null) { + if (!isset(self::$_TSPEC)) { + self::$_TSPEC = array( + 1 => array( + 'var' => 'o1', + 'type' => TType::STRUCT, + 'class' => '\metastore\NoSuchObjectException', + ), + 2 => array( + 'var' => 'o2', + 'type' => TType::STRUCT, + 'class' => '\metastore\InvalidOperationException', + ), + 3 => array( + 'var' => 'o3', + 'type' => TType::STRUCT, + 'class' => '\metastore\MetaException', + ), + ); + } + if (is_array($vals)) { + if (isset($vals['o1'])) { + $this->o1 = $vals['o1']; + } + if (isset($vals['o2'])) { + $this->o2 = $vals['o2']; + } + if (isset($vals['o3'])) { + $this->o3 = $vals['o3']; + } + } + } + + public function getName() { + return 'ThriftHiveMetastore_drop_database_result'; + } + + public function read($input) + { + $xfer = 0; + $fname = null; + $ftype = 0; + $fid = 0; + $xfer += $input->readStructBegin($fname); + while (true) + { + $xfer += $input->readFieldBegin($fname, $ftype, $fid); + if ($ftype == TType::STOP) { + break; + } + switch ($fid) + { + case 1: + if ($ftype == TType::STRUCT) { + $this->o1 = new \metastore\NoSuchObjectException(); + $xfer += $this->o1->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; + case 2: + if ($ftype == TType::STRUCT) { + $this->o2 = new \metastore\InvalidOperationException(); + $xfer += $this->o2->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; + case 3: + if ($ftype == TType::STRUCT) { + $this->o3 = new \metastore\MetaException(); + $xfer += $this->o3->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; + default: + $xfer += $input->skip($ftype); + break; + } + $xfer += $input->readFieldEnd(); + } + $xfer += $input->readStructEnd(); + return $xfer; + } + + public function write($output) { + $xfer = 0; + $xfer += $output->writeStructBegin('ThriftHiveMetastore_drop_database_result'); + if ($this->o1 !== null) { + $xfer += $output->writeFieldBegin('o1', TType::STRUCT, 1); + $xfer += $this->o1->write($output); + $xfer += $output->writeFieldEnd(); + } + if ($this->o2 !== null) { + $xfer += $output->writeFieldBegin('o2', TType::STRUCT, 2); + $xfer += $this->o2->write($output); + $xfer += $output->writeFieldEnd(); + } + if ($this->o3 !== null) { + $xfer += $output->writeFieldBegin('o3', TType::STRUCT, 3); + $xfer += $this->o3->write($output); + $xfer += $output->writeFieldEnd(); + } + $xfer += $output->writeFieldStop(); + $xfer += $output->writeStructEnd(); + return $xfer; + } + +} + +class ThriftHiveMetastore_get_databases_args { + static $_TSPEC; + + public $pattern = null; + + public function __construct($vals=null) { + if (!isset(self::$_TSPEC)) { + self::$_TSPEC = array( + 1 => array( + 'var' => 'pattern', + 'type' => TType::STRING, + ), + ); + } + if (is_array($vals)) { + if (isset($vals['pattern'])) { + $this->pattern = $vals['pattern']; + } + } + } + + public function getName() { + return 'ThriftHiveMetastore_get_databases_args'; + } + + public function read($input) + { + $xfer = 0; + $fname = null; + $ftype = 0; + $fid = 0; + $xfer += $input->readStructBegin($fname); + while (true) + { + $xfer += $input->readFieldBegin($fname, $ftype, $fid); + if ($ftype == TType::STOP) { + break; + } + switch ($fid) + { + case 1: + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->pattern); + } else { + $xfer += $input->skip($ftype); + } + break; + default: + $xfer += $input->skip($ftype); + break; + } + $xfer += $input->readFieldEnd(); + } + $xfer += $input->readStructEnd(); + return $xfer; + } + + public function write($output) { + $xfer = 0; + $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_databases_args'); + if ($this->pattern !== null) { + $xfer += $output->writeFieldBegin('pattern', TType::STRING, 1); + $xfer += $output->writeString($this->pattern); + $xfer += $output->writeFieldEnd(); + } + $xfer += $output->writeFieldStop(); + $xfer += $output->writeStructEnd(); + return $xfer; + } + +} + +class ThriftHiveMetastore_get_databases_result { + static $_TSPEC; + + public $success = null; + public $o1 = null; + + public function __construct($vals=null) { + if (!isset(self::$_TSPEC)) { + self::$_TSPEC = array( + 0 => array( + 'var' => 'success', + 'type' => TType::LST, + 'etype' => TType::STRING, + 'elem' => array( + 'type' => TType::STRING, + ), + ), + 1 => array( + 'var' => 'o1', + 'type' => TType::STRUCT, + 'class' => '\metastore\MetaException', + ), + ); + } + if (is_array($vals)) { + if (isset($vals['success'])) { + $this->success = $vals['success']; + } + if (isset($vals['o1'])) { + $this->o1 = $vals['o1']; + } + } + } + + public function getName() { + return 'ThriftHiveMetastore_get_databases_result'; + } + + public function read($input) + { + $xfer = 0; + $fname = null; + $ftype = 0; + $fid = 0; + $xfer += $input->readStructBegin($fname); + while (true) + { + $xfer += $input->readFieldBegin($fname, $ftype, $fid); + if ($ftype == TType::STOP) { + break; + } + switch ($fid) + { + case 0: + if ($ftype == TType::LST) { + $this->success = array(); + $_size363 = 0; + $_etype366 = 0; + $xfer += $input->readListBegin($_etype366, $_size363); + for ($_i367 = 0; $_i367 < $_size363; ++$_i367) + { + $elem368 = null; + $xfer += $input->readString($elem368); + $this->success []= $elem368; + } + $xfer += $input->readListEnd(); + } else { + $xfer += $input->skip($ftype); + } + break; + case 1: + if ($ftype == TType::STRUCT) { + $this->o1 = new \metastore\MetaException(); + $xfer += $this->o1->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; + default: + $xfer += $input->skip($ftype); + break; + } + $xfer += $input->readFieldEnd(); + } + $xfer += $input->readStructEnd(); + return $xfer; + } + + public function write($output) { + $xfer = 0; + $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_databases_result'); + if ($this->success !== null) { + if (!is_array($this->success)) { + throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); + } + $xfer += $output->writeFieldBegin('success', TType::LST, 0); + { + $output->writeListBegin(TType::STRING, count($this->success)); + { + foreach ($this->success as $iter369) + { + $xfer += $output->writeString($iter369); + } + } + $output->writeListEnd(); + } + $xfer += $output->writeFieldEnd(); + } + if ($this->o1 !== null) { + $xfer += $output->writeFieldBegin('o1', TType::STRUCT, 1); + $xfer += $this->o1->write($output); + $xfer += $output->writeFieldEnd(); + } + $xfer += $output->writeFieldStop(); + $xfer += $output->writeStructEnd(); + return $xfer; + } + +} + +class ThriftHiveMetastore_get_all_databases_args { + static $_TSPEC; + + + public function __construct() { + if (!isset(self::$_TSPEC)) { + self::$_TSPEC = array( + ); + } + } + + public function getName() { + return 'ThriftHiveMetastore_get_all_databases_args'; + } + + public function read($input) + { + $xfer = 0; + $fname = null; + $ftype = 0; + $fid = 0; + $xfer += $input->readStructBegin($fname); + while (true) + { + $xfer += $input->readFieldBegin($fname, $ftype, $fid); + if ($ftype == TType::STOP) { + break; + } + switch ($fid) + { + default: + $xfer += $input->skip($ftype); + break; + } + $xfer += $input->readFieldEnd(); + } + $xfer += $input->readStructEnd(); + return $xfer; + } + + public function write($output) { + $xfer = 0; + $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_all_databases_args'); + $xfer += $output->writeFieldStop(); + $xfer += $output->writeStructEnd(); + return $xfer; + } + +} + +class ThriftHiveMetastore_get_all_databases_result { + static $_TSPEC; + + public $success = null; + public $o1 = null; + + public function __construct($vals=null) { + if (!isset(self::$_TSPEC)) { + self::$_TSPEC = array( + 0 => array( + 'var' => 'success', + 'type' => TType::LST, + 'etype' => TType::STRING, + 'elem' => array( + 'type' => TType::STRING, + ), + ), + 1 => array( + 'var' => 'o1', + 'type' => TType::STRUCT, + 'class' => '\metastore\MetaException', + ), + ); + } + if (is_array($vals)) { + if (isset($vals['success'])) { + $this->success = $vals['success']; + } + if (isset($vals['o1'])) { + $this->o1 = $vals['o1']; + } + } + } + + public function getName() { + return 'ThriftHiveMetastore_get_all_databases_result'; + } + + public function read($input) + { + $xfer = 0; + $fname = null; + $ftype = 0; + $fid = 0; + $xfer += $input->readStructBegin($fname); + while (true) + { + $xfer += $input->readFieldBegin($fname, $ftype, $fid); + if ($ftype == TType::STOP) { + break; + } + switch ($fid) + { + case 0: + if ($ftype == TType::LST) { + $this->success = array(); + $_size370 = 0; + $_etype373 = 0; + $xfer += $input->readListBegin($_etype373, $_size370); + for ($_i374 = 0; $_i374 < $_size370; ++$_i374) + { + $elem375 = null; + $xfer += $input->readString($elem375); + $this->success []= $elem375; + } + $xfer += $input->readListEnd(); + } else { + $xfer += $input->skip($ftype); + } + break; + case 1: + if ($ftype == TType::STRUCT) { + $this->o1 = new \metastore\MetaException(); + $xfer += $this->o1->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; + default: + $xfer += $input->skip($ftype); + break; + } + $xfer += $input->readFieldEnd(); + } + $xfer += $input->readStructEnd(); + return $xfer; + } + + public function write($output) { + $xfer = 0; + $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_all_databases_result'); + if ($this->success !== null) { + if (!is_array($this->success)) { + throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); + } + $xfer += $output->writeFieldBegin('success', TType::LST, 0); + { + $output->writeListBegin(TType::STRING, count($this->success)); + { + foreach ($this->success as $iter376) + { + $xfer += $output->writeString($iter376); + } + } + $output->writeListEnd(); + } + $xfer += $output->writeFieldEnd(); + } + if ($this->o1 !== null) { + $xfer += $output->writeFieldBegin('o1', TType::STRUCT, 1); + $xfer += $this->o1->write($output); + $xfer += $output->writeFieldEnd(); + } + $xfer += $output->writeFieldStop(); + $xfer += $output->writeStructEnd(); + return $xfer; + } + +} + +class ThriftHiveMetastore_alter_database_args { + static $_TSPEC; + + public $dbname = null; + public $db = null; + + public function __construct($vals=null) { + if (!isset(self::$_TSPEC)) { + self::$_TSPEC = array( + 1 => array( + 'var' => 'dbname', + 'type' => TType::STRING, + ), + 2 => array( + 'var' => 'db', + 'type' => TType::STRUCT, + 'class' => '\metastore\Database', + ), + ); + } + if (is_array($vals)) { + if (isset($vals['dbname'])) { + $this->dbname = $vals['dbname']; + } + if (isset($vals['db'])) { + $this->db = $vals['db']; + } + } + } + + public function getName() { + return 'ThriftHiveMetastore_alter_database_args'; + } + + public function read($input) + { + $xfer = 0; + $fname = null; + $ftype = 0; + $fid = 0; + $xfer += $input->readStructBegin($fname); + while (true) + { + $xfer += $input->readFieldBegin($fname, $ftype, $fid); + if ($ftype == TType::STOP) { + break; + } + switch ($fid) + { + case 1: + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->dbname); + } else { + $xfer += $input->skip($ftype); + } + break; + case 2: + if ($ftype == TType::STRUCT) { + $this->db = new \metastore\Database(); + $xfer += $this->db->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; + default: + $xfer += $input->skip($ftype); + break; + } + $xfer += $input->readFieldEnd(); + } + $xfer += $input->readStructEnd(); + return $xfer; + } + + public function write($output) { + $xfer = 0; + $xfer += $output->writeStructBegin('ThriftHiveMetastore_alter_database_args'); + if ($this->dbname !== null) { + $xfer += $output->writeFieldBegin('dbname', TType::STRING, 1); + $xfer += $output->writeString($this->dbname); + $xfer += $output->writeFieldEnd(); + } + if ($this->db !== null) { + if (!is_object($this->db)) { + throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); + } + $xfer += $output->writeFieldBegin('db', TType::STRUCT, 2); + $xfer += $this->db->write($output); + $xfer += $output->writeFieldEnd(); + } + $xfer += $output->writeFieldStop(); + $xfer += $output->writeStructEnd(); + return $xfer; + } + +} + +class ThriftHiveMetastore_alter_database_result { + static $_TSPEC; + + public $o1 = null; + public $o2 = null; + + public function __construct($vals=null) { + if (!isset(self::$_TSPEC)) { + self::$_TSPEC = array( + 1 => array( + 'var' => 'o1', + 'type' => TType::STRUCT, + 'class' => '\metastore\MetaException', + ), + 2 => array( + 'var' => 'o2', + 'type' => TType::STRUCT, + 'class' => '\metastore\NoSuchObjectException', + ), + ); + } + if (is_array($vals)) { + if (isset($vals['o1'])) { + $this->o1 = $vals['o1']; + } + if (isset($vals['o2'])) { + $this->o2 = $vals['o2']; + } + } + } + + public function getName() { + return 'ThriftHiveMetastore_alter_database_result'; + } + + public function read($input) + { + $xfer = 0; + $fname = null; + $ftype = 0; + $fid = 0; + $xfer += $input->readStructBegin($fname); + while (true) + { + $xfer += $input->readFieldBegin($fname, $ftype, $fid); + if ($ftype == TType::STOP) { + break; + } + switch ($fid) + { + case 1: + if ($ftype == TType::STRUCT) { + $this->o1 = new \metastore\MetaException(); + $xfer += $this->o1->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; + case 2: + if ($ftype == TType::STRUCT) { + $this->o2 = new \metastore\NoSuchObjectException(); + $xfer += $this->o2->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; + default: + $xfer += $input->skip($ftype); + break; + } + $xfer += $input->readFieldEnd(); + } + $xfer += $input->readStructEnd(); + return $xfer; + } + + public function write($output) { + $xfer = 0; + $xfer += $output->writeStructBegin('ThriftHiveMetastore_alter_database_result'); + if ($this->o1 !== null) { + $xfer += $output->writeFieldBegin('o1', TType::STRUCT, 1); + $xfer += $this->o1->write($output); + $xfer += $output->writeFieldEnd(); + } + if ($this->o2 !== null) { + $xfer += $output->writeFieldBegin('o2', TType::STRUCT, 2); + $xfer += $this->o2->write($output); + $xfer += $output->writeFieldEnd(); + } + $xfer += $output->writeFieldStop(); + $xfer += $output->writeStructEnd(); + return $xfer; + } + +} + +class ThriftHiveMetastore_get_type_args { + static $_TSPEC; + + public $name = null; + + public function __construct($vals=null) { + if (!isset(self::$_TSPEC)) { + self::$_TSPEC = array( + 1 => array( + 'var' => 'name', + 'type' => TType::STRING, + ), + ); + } + if (is_array($vals)) { + if (isset($vals['name'])) { + $this->name = $vals['name']; + } + } + } + + public function getName() { + return 'ThriftHiveMetastore_get_type_args'; + } + + public function read($input) + { + $xfer = 0; + $fname = null; + $ftype = 0; + $fid = 0; + $xfer += $input->readStructBegin($fname); + while (true) + { + $xfer += $input->readFieldBegin($fname, $ftype, $fid); + if ($ftype == TType::STOP) { + break; + } + switch ($fid) + { + case 1: + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->name); + } else { + $xfer += $input->skip($ftype); + } + break; + default: + $xfer += $input->skip($ftype); + break; + } + $xfer += $input->readFieldEnd(); + } + $xfer += $input->readStructEnd(); + return $xfer; + } + + public function write($output) { + $xfer = 0; + $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_type_args'); + if ($this->name !== null) { + $xfer += $output->writeFieldBegin('name', TType::STRING, 1); + $xfer += $output->writeString($this->name); + $xfer += $output->writeFieldEnd(); + } + $xfer += $output->writeFieldStop(); + $xfer += $output->writeStructEnd(); + return $xfer; + } + +} + +class ThriftHiveMetastore_get_type_result { + static $_TSPEC; + + public $success = null; + public $o1 = null; + public $o2 = null; + + public function __construct($vals=null) { + if (!isset(self::$_TSPEC)) { + self::$_TSPEC = array( + 0 => array( + 'var' => 'success', + 'type' => TType::STRUCT, + 'class' => '\metastore\Type', + ), + 1 => array( + 'var' => 'o1', + 'type' => TType::STRUCT, + 'class' => '\metastore\MetaException', + ), + 2 => array( + 'var' => 'o2', + 'type' => TType::STRUCT, + 'class' => '\metastore\NoSuchObjectException', + ), + ); + } + if (is_array($vals)) { + if (isset($vals['success'])) { + $this->success = $vals['success']; + } + if (isset($vals['o1'])) { + $this->o1 = $vals['o1']; + } + if (isset($vals['o2'])) { + $this->o2 = $vals['o2']; + } + } + } + + public function getName() { + return 'ThriftHiveMetastore_get_type_result'; + } + + public function read($input) + { + $xfer = 0; + $fname = null; + $ftype = 0; + $fid = 0; + $xfer += $input->readStructBegin($fname); + while (true) + { + $xfer += $input->readFieldBegin($fname, $ftype, $fid); + if ($ftype == TType::STOP) { + break; + } + switch ($fid) + { + case 0: + if ($ftype == TType::STRUCT) { + $this->success = new \metastore\Type(); + $xfer += $this->success->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; + case 1: + if ($ftype == TType::STRUCT) { + $this->o1 = new \metastore\MetaException(); + $xfer += $this->o1->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; + case 2: + if ($ftype == TType::STRUCT) { + $this->o2 = new \metastore\NoSuchObjectException(); + $xfer += $this->o2->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; + default: + $xfer += $input->skip($ftype); + break; + } + $xfer += $input->readFieldEnd(); + } + $xfer += $input->readStructEnd(); + return $xfer; + } + + public function write($output) { + $xfer = 0; + $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_type_result'); + if ($this->success !== null) { + if (!is_object($this->success)) { + throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); + } + $xfer += $output->writeFieldBegin('success', TType::STRUCT, 0); + $xfer += $this->success->write($output); + $xfer += $output->writeFieldEnd(); + } + if ($this->o1 !== null) { + $xfer += $output->writeFieldBegin('o1', TType::STRUCT, 1); + $xfer += $this->o1->write($output); + $xfer += $output->writeFieldEnd(); + } + if ($this->o2 !== null) { + $xfer += $output->writeFieldBegin('o2', TType::STRUCT, 2); + $xfer += $this->o2->write($output); + $xfer += $output->writeFieldEnd(); + } + $xfer += $output->writeFieldStop(); + $xfer += $output->writeStructEnd(); + return $xfer; + } + +} + +class ThriftHiveMetastore_create_type_args { + static $_TSPEC; + + public $type = null; + + public function __construct($vals=null) { + if (!isset(self::$_TSPEC)) { + self::$_TSPEC = array( + 1 => array( + 'var' => 'type', + 'type' => TType::STRUCT, + 'class' => '\metastore\Type', + ), + ); + } + if (is_array($vals)) { + if (isset($vals['type'])) { + $this->type = $vals['type']; + } + } + } + + public function getName() { + return 'ThriftHiveMetastore_create_type_args'; + } + + public function read($input) + { + $xfer = 0; + $fname = null; + $ftype = 0; + $fid = 0; + $xfer += $input->readStructBegin($fname); + while (true) + { + $xfer += $input->readFieldBegin($fname, $ftype, $fid); + if ($ftype == TType::STOP) { + break; + } + switch ($fid) + { + case 1: + if ($ftype == TType::STRUCT) { + $this->type = new \metastore\Type(); + $xfer += $this->type->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; + default: + $xfer += $input->skip($ftype); + break; + } + $xfer += $input->readFieldEnd(); + } + $xfer += $input->readStructEnd(); + return $xfer; + } + + public function write($output) { + $xfer = 0; + $xfer += $output->writeStructBegin('ThriftHiveMetastore_create_type_args'); + if ($this->type !== null) { + if (!is_object($this->type)) { + throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); + } + $xfer += $output->writeFieldBegin('type', TType::STRUCT, 1); + $xfer += $this->type->write($output); + $xfer += $output->writeFieldEnd(); + } + $xfer += $output->writeFieldStop(); + $xfer += $output->writeStructEnd(); + return $xfer; + } + +} + +class ThriftHiveMetastore_create_type_result { + static $_TSPEC; + + public $success = null; + public $o1 = null; + public $o2 = null; + public $o3 = null; + + public function __construct($vals=null) { + if (!isset(self::$_TSPEC)) { + self::$_TSPEC = array( + 0 => array( + 'var' => 'success', + 'type' => TType::BOOL, + ), + 1 => array( + 'var' => 'o1', + 'type' => TType::STRUCT, + 'class' => '\metastore\AlreadyExistsException', + ), + 2 => array( + 'var' => 'o2', + 'type' => TType::STRUCT, + 'class' => '\metastore\InvalidObjectException', + ), + 3 => array( + 'var' => 'o3', + 'type' => TType::STRUCT, + 'class' => '\metastore\MetaException', + ), + ); + } + if (is_array($vals)) { + if (isset($vals['success'])) { + $this->success = $vals['success']; + } + if (isset($vals['o1'])) { + $this->o1 = $vals['o1']; + } + if (isset($vals['o2'])) { + $this->o2 = $vals['o2']; + } + if (isset($vals['o3'])) { + $this->o3 = $vals['o3']; + } + } + } + + public function getName() { + return 'ThriftHiveMetastore_create_type_result'; + } + + public function read($input) + { + $xfer = 0; + $fname = null; + $ftype = 0; + $fid = 0; + $xfer += $input->readStructBegin($fname); + while (true) + { + $xfer += $input->readFieldBegin($fname, $ftype, $fid); + if ($ftype == TType::STOP) { + break; + } + switch ($fid) + { + case 0: + if ($ftype == TType::BOOL) { + $xfer += $input->readBool($this->success); + } else { + $xfer += $input->skip($ftype); + } + break; + case 1: + if ($ftype == TType::STRUCT) { + $this->o1 = new \metastore\AlreadyExistsException(); + $xfer += $this->o1->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; + case 2: + if ($ftype == TType::STRUCT) { + $this->o2 = new \metastore\InvalidObjectException(); + $xfer += $this->o2->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; + case 3: + if ($ftype == TType::STRUCT) { + $this->o3 = new \metastore\MetaException(); + $xfer += $this->o3->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; + default: + $xfer += $input->skip($ftype); + break; + } + $xfer += $input->readFieldEnd(); + } + $xfer += $input->readStructEnd(); + return $xfer; + } + + public function write($output) { + $xfer = 0; + $xfer += $output->writeStructBegin('ThriftHiveMetastore_create_type_result'); + if ($this->success !== null) { + $xfer += $output->writeFieldBegin('success', TType::BOOL, 0); + $xfer += $output->writeBool($this->success); + $xfer += $output->writeFieldEnd(); + } + if ($this->o1 !== null) { + $xfer += $output->writeFieldBegin('o1', TType::STRUCT, 1); + $xfer += $this->o1->write($output); + $xfer += $output->writeFieldEnd(); + } + if ($this->o2 !== null) { + $xfer += $output->writeFieldBegin('o2', TType::STRUCT, 2); + $xfer += $this->o2->write($output); + $xfer += $output->writeFieldEnd(); + } + if ($this->o3 !== null) { + $xfer += $output->writeFieldBegin('o3', TType::STRUCT, 3); + $xfer += $this->o3->write($output); + $xfer += $output->writeFieldEnd(); + } + $xfer += $output->writeFieldStop(); + $xfer += $output->writeStructEnd(); + return $xfer; + } + +} + +class ThriftHiveMetastore_drop_type_args { + static $_TSPEC; + + public $type = null; + + public function __construct($vals=null) { + if (!isset(self::$_TSPEC)) { + self::$_TSPEC = array( + 1 => array( + 'var' => 'type', + 'type' => TType::STRING, + ), + ); + } + if (is_array($vals)) { + if (isset($vals['type'])) { + $this->type = $vals['type']; + } + } + } + + public function getName() { + return 'ThriftHiveMetastore_drop_type_args'; + } + + public function read($input) + { + $xfer = 0; + $fname = null; + $ftype = 0; + $fid = 0; + $xfer += $input->readStructBegin($fname); + while (true) + { + $xfer += $input->readFieldBegin($fname, $ftype, $fid); + if ($ftype == TType::STOP) { + break; + } + switch ($fid) + { + case 1: + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->type); + } else { + $xfer += $input->skip($ftype); + } + break; + default: + $xfer += $input->skip($ftype); + break; + } + $xfer += $input->readFieldEnd(); + } + $xfer += $input->readStructEnd(); + return $xfer; + } + + public function write($output) { + $xfer = 0; + $xfer += $output->writeStructBegin('ThriftHiveMetastore_drop_type_args'); + if ($this->type !== null) { + $xfer += $output->writeFieldBegin('type', TType::STRING, 1); + $xfer += $output->writeString($this->type); + $xfer += $output->writeFieldEnd(); + } + $xfer += $output->writeFieldStop(); + $xfer += $output->writeStructEnd(); + return $xfer; + } + +} + +class ThriftHiveMetastore_drop_type_result { + static $_TSPEC; + + public $success = null; + public $o1 = null; + public $o2 = null; + + public function __construct($vals=null) { + if (!isset(self::$_TSPEC)) { + self::$_TSPEC = array( + 0 => array( + 'var' => 'success', + 'type' => TType::BOOL, + ), + 1 => array( + 'var' => 'o1', + 'type' => TType::STRUCT, + 'class' => '\metastore\MetaException', + ), + 2 => array( + 'var' => 'o2', + 'type' => TType::STRUCT, + 'class' => '\metastore\NoSuchObjectException', + ), + ); + } + if (is_array($vals)) { + if (isset($vals['success'])) { + $this->success = $vals['success']; + } + if (isset($vals['o1'])) { + $this->o1 = $vals['o1']; + } + if (isset($vals['o2'])) { + $this->o2 = $vals['o2']; + } + } + } + + public function getName() { + return 'ThriftHiveMetastore_drop_type_result'; + } + + public function read($input) + { + $xfer = 0; + $fname = null; + $ftype = 0; + $fid = 0; + $xfer += $input->readStructBegin($fname); + while (true) + { + $xfer += $input->readFieldBegin($fname, $ftype, $fid); + if ($ftype == TType::STOP) { + break; + } + switch ($fid) + { + case 0: + if ($ftype == TType::BOOL) { + $xfer += $input->readBool($this->success); + } else { + $xfer += $input->skip($ftype); + } + break; + case 1: + if ($ftype == TType::STRUCT) { + $this->o1 = new \metastore\MetaException(); + $xfer += $this->o1->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; + case 2: + if ($ftype == TType::STRUCT) { + $this->o2 = new \metastore\NoSuchObjectException(); + $xfer += $this->o2->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; + default: + $xfer += $input->skip($ftype); + break; + } + $xfer += $input->readFieldEnd(); + } + $xfer += $input->readStructEnd(); + return $xfer; + } + + public function write($output) { + $xfer = 0; + $xfer += $output->writeStructBegin('ThriftHiveMetastore_drop_type_result'); + if ($this->success !== null) { + $xfer += $output->writeFieldBegin('success', TType::BOOL, 0); + $xfer += $output->writeBool($this->success); + $xfer += $output->writeFieldEnd(); + } + if ($this->o1 !== null) { + $xfer += $output->writeFieldBegin('o1', TType::STRUCT, 1); + $xfer += $this->o1->write($output); + $xfer += $output->writeFieldEnd(); + } + if ($this->o2 !== null) { + $xfer += $output->writeFieldBegin('o2', TType::STRUCT, 2); + $xfer += $this->o2->write($output); + $xfer += $output->writeFieldEnd(); + } + $xfer += $output->writeFieldStop(); + $xfer += $output->writeStructEnd(); + return $xfer; + } + +} + +class ThriftHiveMetastore_get_type_all_args { + static $_TSPEC; + + public $name = null; + + public function __construct($vals=null) { + if (!isset(self::$_TSPEC)) { + self::$_TSPEC = array( + 1 => array( + 'var' => 'name', + 'type' => TType::STRING, + ), + ); + } + if (is_array($vals)) { + if (isset($vals['name'])) { + $this->name = $vals['name']; + } + } + } + + public function getName() { + return 'ThriftHiveMetastore_get_type_all_args'; + } + + public function read($input) + { + $xfer = 0; + $fname = null; + $ftype = 0; + $fid = 0; + $xfer += $input->readStructBegin($fname); + while (true) + { + $xfer += $input->readFieldBegin($fname, $ftype, $fid); + if ($ftype == TType::STOP) { + break; + } + switch ($fid) + { + case 1: + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->name); + } else { + $xfer += $input->skip($ftype); + } + break; + default: + $xfer += $input->skip($ftype); + break; + } + $xfer += $input->readFieldEnd(); + } + $xfer += $input->readStructEnd(); + return $xfer; + } + + public function write($output) { + $xfer = 0; + $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_type_all_args'); + if ($this->name !== null) { + $xfer += $output->writeFieldBegin('name', TType::STRING, 1); + $xfer += $output->writeString($this->name); + $xfer += $output->writeFieldEnd(); + } + $xfer += $output->writeFieldStop(); + $xfer += $output->writeStructEnd(); + return $xfer; + } + +} + +class ThriftHiveMetastore_get_type_all_result { + static $_TSPEC; + + public $success = null; + public $o2 = null; + + public function __construct($vals=null) { + if (!isset(self::$_TSPEC)) { + self::$_TSPEC = array( + 0 => array( + 'var' => 'success', + 'type' => TType::MAP, + 'ktype' => TType::STRING, + 'vtype' => TType::STRUCT, + 'key' => array( + 'type' => TType::STRING, + ), + 'val' => array( + 'type' => TType::STRUCT, + 'class' => '\metastore\Type', + ), + ), + 1 => array( + 'var' => 'o2', + 'type' => TType::STRUCT, + 'class' => '\metastore\MetaException', + ), + ); + } + if (is_array($vals)) { + if (isset($vals['success'])) { + $this->success = $vals['success']; + } + if (isset($vals['o2'])) { + $this->o2 = $vals['o2']; + } + } + } + + public function getName() { + return 'ThriftHiveMetastore_get_type_all_result'; + } + + public function read($input) + { + $xfer = 0; + $fname = null; + $ftype = 0; + $fid = 0; + $xfer += $input->readStructBegin($fname); + while (true) + { + $xfer += $input->readFieldBegin($fname, $ftype, $fid); + if ($ftype == TType::STOP) { + break; + } + switch ($fid) + { + case 0: + if ($ftype == TType::MAP) { + $this->success = array(); + $_size377 = 0; + $_ktype378 = 0; + $_vtype379 = 0; + $xfer += $input->readMapBegin($_ktype378, $_vtype379, $_size377); + for ($_i381 = 0; $_i381 < $_size377; ++$_i381) + { + $key382 = ''; + $val383 = new \metastore\Type(); + $xfer += $input->readString($key382); + $val383 = new \metastore\Type(); + $xfer += $val383->read($input); + $this->success[$key382] = $val383; + } + $xfer += $input->readMapEnd(); + } else { + $xfer += $input->skip($ftype); + } + break; + case 1: + if ($ftype == TType::STRUCT) { + $this->o2 = new \metastore\MetaException(); + $xfer += $this->o2->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; + default: + $xfer += $input->skip($ftype); + break; + } + $xfer += $input->readFieldEnd(); + } + $xfer += $input->readStructEnd(); + return $xfer; + } + + public function write($output) { + $xfer = 0; + $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_type_all_result'); + if ($this->success !== null) { + if (!is_array($this->success)) { + throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); + } + $xfer += $output->writeFieldBegin('success', TType::MAP, 0); + { + $output->writeMapBegin(TType::STRING, TType::STRUCT, count($this->success)); + { + foreach ($this->success as $kiter384 => $viter385) + { + $xfer += $output->writeString($kiter384); + $xfer += $viter385->write($output); + } + } + $output->writeMapEnd(); + } + $xfer += $output->writeFieldEnd(); + } + if ($this->o2 !== null) { + $xfer += $output->writeFieldBegin('o2', TType::STRUCT, 1); + $xfer += $this->o2->write($output); + $xfer += $output->writeFieldEnd(); + } + $xfer += $output->writeFieldStop(); + $xfer += $output->writeStructEnd(); + return $xfer; + } + +} + +class ThriftHiveMetastore_get_fields_args { + static $_TSPEC; + + public $db_name = null; + public $table_name = null; + + public function __construct($vals=null) { + if (!isset(self::$_TSPEC)) { + self::$_TSPEC = array( + 1 => array( + 'var' => 'db_name', + 'type' => TType::STRING, + ), + 2 => array( + 'var' => 'table_name', + 'type' => TType::STRING, + ), + ); + } + if (is_array($vals)) { + if (isset($vals['db_name'])) { + $this->db_name = $vals['db_name']; + } + if (isset($vals['table_name'])) { + $this->table_name = $vals['table_name']; + } + } + } + + public function getName() { + return 'ThriftHiveMetastore_get_fields_args'; + } + + public function read($input) + { + $xfer = 0; + $fname = null; + $ftype = 0; + $fid = 0; + $xfer += $input->readStructBegin($fname); + while (true) + { + $xfer += $input->readFieldBegin($fname, $ftype, $fid); + if ($ftype == TType::STOP) { + break; + } + switch ($fid) + { + case 1: + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->db_name); + } else { + $xfer += $input->skip($ftype); + } + break; + case 2: + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->table_name); + } else { + $xfer += $input->skip($ftype); + } + break; + default: + $xfer += $input->skip($ftype); + break; + } + $xfer += $input->readFieldEnd(); + } + $xfer += $input->readStructEnd(); + return $xfer; + } + + public function write($output) { + $xfer = 0; + $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_fields_args'); + if ($this->db_name !== null) { + $xfer += $output->writeFieldBegin('db_name', TType::STRING, 1); + $xfer += $output->writeString($this->db_name); + $xfer += $output->writeFieldEnd(); + } + if ($this->table_name !== null) { + $xfer += $output->writeFieldBegin('table_name', TType::STRING, 2); + $xfer += $output->writeString($this->table_name); + $xfer += $output->writeFieldEnd(); + } + $xfer += $output->writeFieldStop(); + $xfer += $output->writeStructEnd(); + return $xfer; + } + +} + +class ThriftHiveMetastore_get_fields_result { + static $_TSPEC; + + public $success = null; + public $o1 = null; + public $o2 = null; + public $o3 = null; + + public function __construct($vals=null) { + if (!isset(self::$_TSPEC)) { + self::$_TSPEC = array( + 0 => array( + 'var' => 'success', + 'type' => TType::LST, + 'etype' => TType::STRUCT, + 'elem' => array( + 'type' => TType::STRUCT, + 'class' => '\metastore\FieldSchema', + ), + ), + 1 => array( + 'var' => 'o1', + 'type' => TType::STRUCT, + 'class' => '\metastore\MetaException', + ), + 2 => array( + 'var' => 'o2', + 'type' => TType::STRUCT, + 'class' => '\metastore\UnknownTableException', + ), + 3 => array( + 'var' => 'o3', + 'type' => TType::STRUCT, + 'class' => '\metastore\UnknownDBException', + ), + ); + } + if (is_array($vals)) { + if (isset($vals['success'])) { + $this->success = $vals['success']; + } + if (isset($vals['o1'])) { + $this->o1 = $vals['o1']; + } + if (isset($vals['o2'])) { + $this->o2 = $vals['o2']; + } + if (isset($vals['o3'])) { + $this->o3 = $vals['o3']; + } + } + } + + public function getName() { + return 'ThriftHiveMetastore_get_fields_result'; + } + + public function read($input) + { + $xfer = 0; + $fname = null; + $ftype = 0; + $fid = 0; + $xfer += $input->readStructBegin($fname); + while (true) + { + $xfer += $input->readFieldBegin($fname, $ftype, $fid); + if ($ftype == TType::STOP) { + break; + } + switch ($fid) + { + case 0: + if ($ftype == TType::LST) { + $this->success = array(); + $_size386 = 0; + $_etype389 = 0; + $xfer += $input->readListBegin($_etype389, $_size386); + for ($_i390 = 0; $_i390 < $_size386; ++$_i390) + { + $elem391 = null; + $elem391 = new \metastore\FieldSchema(); + $xfer += $elem391->read($input); + $this->success []= $elem391; + } + $xfer += $input->readListEnd(); + } else { + $xfer += $input->skip($ftype); + } + break; + case 1: + if ($ftype == TType::STRUCT) { + $this->o1 = new \metastore\MetaException(); + $xfer += $this->o1->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; + case 2: + if ($ftype == TType::STRUCT) { + $this->o2 = new \metastore\UnknownTableException(); + $xfer += $this->o2->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; + case 3: + if ($ftype == TType::STRUCT) { + $this->o3 = new \metastore\UnknownDBException(); + $xfer += $this->o3->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; + default: + $xfer += $input->skip($ftype); + break; + } + $xfer += $input->readFieldEnd(); + } + $xfer += $input->readStructEnd(); + return $xfer; + } + + public function write($output) { + $xfer = 0; + $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_fields_result'); + if ($this->success !== null) { + if (!is_array($this->success)) { + throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); + } + $xfer += $output->writeFieldBegin('success', TType::LST, 0); + { + $output->writeListBegin(TType::STRUCT, count($this->success)); + { + foreach ($this->success as $iter392) + { + $xfer += $iter392->write($output); + } + } + $output->writeListEnd(); + } + $xfer += $output->writeFieldEnd(); + } + if ($this->o1 !== null) { + $xfer += $output->writeFieldBegin('o1', TType::STRUCT, 1); + $xfer += $this->o1->write($output); + $xfer += $output->writeFieldEnd(); + } + if ($this->o2 !== null) { + $xfer += $output->writeFieldBegin('o2', TType::STRUCT, 2); + $xfer += $this->o2->write($output); + $xfer += $output->writeFieldEnd(); + } + if ($this->o3 !== null) { + $xfer += $output->writeFieldBegin('o3', TType::STRUCT, 3); + $xfer += $this->o3->write($output); + $xfer += $output->writeFieldEnd(); + } + $xfer += $output->writeFieldStop(); + $xfer += $output->writeStructEnd(); + return $xfer; + } + +} + +class ThriftHiveMetastore_get_schema_args { + static $_TSPEC; + + public $db_name = null; + public $table_name = null; + + public function __construct($vals=null) { + if (!isset(self::$_TSPEC)) { + self::$_TSPEC = array( + 1 => array( + 'var' => 'db_name', + 'type' => TType::STRING, + ), + 2 => array( + 'var' => 'table_name', + 'type' => TType::STRING, + ), + ); + } + if (is_array($vals)) { + if (isset($vals['db_name'])) { + $this->db_name = $vals['db_name']; + } + if (isset($vals['table_name'])) { + $this->table_name = $vals['table_name']; + } + } + } + + public function getName() { + return 'ThriftHiveMetastore_get_schema_args'; + } + + public function read($input) + { + $xfer = 0; + $fname = null; + $ftype = 0; + $fid = 0; + $xfer += $input->readStructBegin($fname); + while (true) + { + $xfer += $input->readFieldBegin($fname, $ftype, $fid); + if ($ftype == TType::STOP) { + break; + } + switch ($fid) + { + case 1: + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->db_name); + } else { + $xfer += $input->skip($ftype); + } + break; + case 2: + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->table_name); + } else { + $xfer += $input->skip($ftype); + } + break; + default: + $xfer += $input->skip($ftype); + break; + } + $xfer += $input->readFieldEnd(); + } + $xfer += $input->readStructEnd(); + return $xfer; + } + + public function write($output) { + $xfer = 0; + $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_schema_args'); + if ($this->db_name !== null) { + $xfer += $output->writeFieldBegin('db_name', TType::STRING, 1); + $xfer += $output->writeString($this->db_name); + $xfer += $output->writeFieldEnd(); + } + if ($this->table_name !== null) { + $xfer += $output->writeFieldBegin('table_name', TType::STRING, 2); + $xfer += $output->writeString($this->table_name); + $xfer += $output->writeFieldEnd(); + } + $xfer += $output->writeFieldStop(); + $xfer += $output->writeStructEnd(); + return $xfer; + } + } -// HELPER FUNCTIONS AND STRUCTURES +class ThriftHiveMetastore_get_schema_result { + static $_TSPEC; + + public $success = null; + public $o1 = null; + public $o2 = null; + public $o3 = null; + + public function __construct($vals=null) { + if (!isset(self::$_TSPEC)) { + self::$_TSPEC = array( + 0 => array( + 'var' => 'success', + 'type' => TType::LST, + 'etype' => TType::STRUCT, + 'elem' => array( + 'type' => TType::STRUCT, + 'class' => '\metastore\FieldSchema', + ), + ), + 1 => array( + 'var' => 'o1', + 'type' => TType::STRUCT, + 'class' => '\metastore\MetaException', + ), + 2 => array( + 'var' => 'o2', + 'type' => TType::STRUCT, + 'class' => '\metastore\UnknownTableException', + ), + 3 => array( + 'var' => 'o3', + 'type' => TType::STRUCT, + 'class' => '\metastore\UnknownDBException', + ), + ); + } + if (is_array($vals)) { + if (isset($vals['success'])) { + $this->success = $vals['success']; + } + if (isset($vals['o1'])) { + $this->o1 = $vals['o1']; + } + if (isset($vals['o2'])) { + $this->o2 = $vals['o2']; + } + if (isset($vals['o3'])) { + $this->o3 = $vals['o3']; + } + } + } + + public function getName() { + return 'ThriftHiveMetastore_get_schema_result'; + } + + public function read($input) + { + $xfer = 0; + $fname = null; + $ftype = 0; + $fid = 0; + $xfer += $input->readStructBegin($fname); + while (true) + { + $xfer += $input->readFieldBegin($fname, $ftype, $fid); + if ($ftype == TType::STOP) { + break; + } + switch ($fid) + { + case 0: + if ($ftype == TType::LST) { + $this->success = array(); + $_size393 = 0; + $_etype396 = 0; + $xfer += $input->readListBegin($_etype396, $_size393); + for ($_i397 = 0; $_i397 < $_size393; ++$_i397) + { + $elem398 = null; + $elem398 = new \metastore\FieldSchema(); + $xfer += $elem398->read($input); + $this->success []= $elem398; + } + $xfer += $input->readListEnd(); + } else { + $xfer += $input->skip($ftype); + } + break; + case 1: + if ($ftype == TType::STRUCT) { + $this->o1 = new \metastore\MetaException(); + $xfer += $this->o1->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; + case 2: + if ($ftype == TType::STRUCT) { + $this->o2 = new \metastore\UnknownTableException(); + $xfer += $this->o2->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; + case 3: + if ($ftype == TType::STRUCT) { + $this->o3 = new \metastore\UnknownDBException(); + $xfer += $this->o3->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; + default: + $xfer += $input->skip($ftype); + break; + } + $xfer += $input->readFieldEnd(); + } + $xfer += $input->readStructEnd(); + return $xfer; + } + + public function write($output) { + $xfer = 0; + $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_schema_result'); + if ($this->success !== null) { + if (!is_array($this->success)) { + throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); + } + $xfer += $output->writeFieldBegin('success', TType::LST, 0); + { + $output->writeListBegin(TType::STRUCT, count($this->success)); + { + foreach ($this->success as $iter399) + { + $xfer += $iter399->write($output); + } + } + $output->writeListEnd(); + } + $xfer += $output->writeFieldEnd(); + } + if ($this->o1 !== null) { + $xfer += $output->writeFieldBegin('o1', TType::STRUCT, 1); + $xfer += $this->o1->write($output); + $xfer += $output->writeFieldEnd(); + } + if ($this->o2 !== null) { + $xfer += $output->writeFieldBegin('o2', TType::STRUCT, 2); + $xfer += $this->o2->write($output); + $xfer += $output->writeFieldEnd(); + } + if ($this->o3 !== null) { + $xfer += $output->writeFieldBegin('o3', TType::STRUCT, 3); + $xfer += $this->o3->write($output); + $xfer += $output->writeFieldEnd(); + } + $xfer += $output->writeFieldStop(); + $xfer += $output->writeStructEnd(); + return $xfer; + } + +} -class ThriftHiveMetastore_create_database_args { +class ThriftHiveMetastore_create_table_args { static $_TSPEC; - public $database = null; + public $tbl = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 1 => array( - 'var' => 'database', + 'var' => 'tbl', 'type' => TType::STRUCT, - 'class' => '\metastore\Database', + 'class' => '\metastore\Table', ), ); } if (is_array($vals)) { - if (isset($vals['database'])) { - $this->database = $vals['database']; + if (isset($vals['tbl'])) { + $this->tbl = $vals['tbl']; } } } public function getName() { - return 'ThriftHiveMetastore_create_database_args'; + return 'ThriftHiveMetastore_create_table_args'; } public function read($input) @@ -5543,8 +8676,8 @@ class ThriftHiveMetastore_create_database_args { { case 1: if ($ftype == TType::STRUCT) { - $this->database = new \metastore\Database(); - $xfer += $this->database->read($input); + $this->tbl = new \metastore\Table(); + $xfer += $this->tbl->read($input); } else { $xfer += $input->skip($ftype); } @@ -5561,13 +8694,13 @@ class ThriftHiveMetastore_create_database_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_create_database_args'); - if ($this->database !== null) { - if (!is_object($this->database)) { + $xfer += $output->writeStructBegin('ThriftHiveMetastore_create_table_args'); + if ($this->tbl !== null) { + if (!is_object($this->tbl)) { throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); } - $xfer += $output->writeFieldBegin('database', TType::STRUCT, 1); - $xfer += $this->database->write($output); + $xfer += $output->writeFieldBegin('tbl', TType::STRUCT, 1); + $xfer += $this->tbl->write($output); $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -5577,12 +8710,13 @@ class ThriftHiveMetastore_create_database_args { } -class ThriftHiveMetastore_create_database_result { +class ThriftHiveMetastore_create_table_result { static $_TSPEC; public $o1 = null; public $o2 = null; public $o3 = null; + public $o4 = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { @@ -5602,6 +8736,11 @@ class ThriftHiveMetastore_create_database_result { 'type' => TType::STRUCT, 'class' => '\metastore\MetaException', ), + 4 => array( + 'var' => 'o4', + 'type' => TType::STRUCT, + 'class' => '\metastore\NoSuchObjectException', + ), ); } if (is_array($vals)) { @@ -5614,11 +8753,14 @@ class ThriftHiveMetastore_create_database_result { if (isset($vals['o3'])) { $this->o3 = $vals['o3']; } + if (isset($vals['o4'])) { + $this->o4 = $vals['o4']; + } } } public function getName() { - return 'ThriftHiveMetastore_create_database_result'; + return 'ThriftHiveMetastore_create_table_result'; } public function read($input) @@ -5660,6 +8802,14 @@ class ThriftHiveMetastore_create_database_result { $xfer += $input->skip($ftype); } break; + case 4: + if ($ftype == TType::STRUCT) { + $this->o4 = new \metastore\NoSuchObjectException(); + $xfer += $this->o4->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; default: $xfer += $input->skip($ftype); break; @@ -5672,7 +8822,7 @@ class ThriftHiveMetastore_create_database_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_create_database_result'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_create_table_result'); if ($this->o1 !== null) { $xfer += $output->writeFieldBegin('o1', TType::STRUCT, 1); $xfer += $this->o1->write($output); @@ -5688,6 +8838,11 @@ class ThriftHiveMetastore_create_database_result { $xfer += $this->o3->write($output); $xfer += $output->writeFieldEnd(); } + if ($this->o4 !== null) { + $xfer += $output->writeFieldBegin('o4', TType::STRUCT, 4); + $xfer += $this->o4->write($output); + $xfer += $output->writeFieldEnd(); + } $xfer += $output->writeFieldStop(); $xfer += $output->writeStructEnd(); return $xfer; @@ -5695,29 +8850,39 @@ class ThriftHiveMetastore_create_database_result { } -class ThriftHiveMetastore_get_database_args { +class ThriftHiveMetastore_create_table_with_environment_context_args { static $_TSPEC; - public $name = null; + public $tbl = null; + public $environment_context = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 1 => array( - 'var' => 'name', - 'type' => TType::STRING, + 'var' => 'tbl', + 'type' => TType::STRUCT, + 'class' => '\metastore\Table', + ), + 2 => array( + 'var' => 'environment_context', + 'type' => TType::STRUCT, + 'class' => '\metastore\EnvironmentContext', ), ); } if (is_array($vals)) { - if (isset($vals['name'])) { - $this->name = $vals['name']; + if (isset($vals['tbl'])) { + $this->tbl = $vals['tbl']; + } + if (isset($vals['environment_context'])) { + $this->environment_context = $vals['environment_context']; } } } public function getName() { - return 'ThriftHiveMetastore_get_database_args'; + return 'ThriftHiveMetastore_create_table_with_environment_context_args'; } public function read($input) @@ -5736,8 +8901,17 @@ class ThriftHiveMetastore_get_database_args { switch ($fid) { case 1: - if ($ftype == TType::STRING) { - $xfer += $input->readString($this->name); + if ($ftype == TType::STRUCT) { + $this->tbl = new \metastore\Table(); + $xfer += $this->tbl->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; + case 2: + if ($ftype == TType::STRUCT) { + $this->environment_context = new \metastore\EnvironmentContext(); + $xfer += $this->environment_context->read($input); } else { $xfer += $input->skip($ftype); } @@ -5754,10 +8928,21 @@ class ThriftHiveMetastore_get_database_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_database_args'); - if ($this->name !== null) { - $xfer += $output->writeFieldBegin('name', TType::STRING, 1); - $xfer += $output->writeString($this->name); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_create_table_with_environment_context_args'); + if ($this->tbl !== null) { + if (!is_object($this->tbl)) { + throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); + } + $xfer += $output->writeFieldBegin('tbl', TType::STRUCT, 1); + $xfer += $this->tbl->write($output); + $xfer += $output->writeFieldEnd(); + } + if ($this->environment_context !== null) { + if (!is_object($this->environment_context)) { + throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); + } + $xfer += $output->writeFieldBegin('environment_context', TType::STRUCT, 2); + $xfer += $this->environment_context->write($output); $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -5767,48 +8952,57 @@ class ThriftHiveMetastore_get_database_args { } -class ThriftHiveMetastore_get_database_result { +class ThriftHiveMetastore_create_table_with_environment_context_result { static $_TSPEC; - public $success = null; public $o1 = null; public $o2 = null; + public $o3 = null; + public $o4 = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( - 0 => array( - 'var' => 'success', - 'type' => TType::STRUCT, - 'class' => '\metastore\Database', - ), 1 => array( 'var' => 'o1', 'type' => TType::STRUCT, - 'class' => '\metastore\NoSuchObjectException', + 'class' => '\metastore\AlreadyExistsException', ), 2 => array( 'var' => 'o2', 'type' => TType::STRUCT, + 'class' => '\metastore\InvalidObjectException', + ), + 3 => array( + 'var' => 'o3', + 'type' => TType::STRUCT, 'class' => '\metastore\MetaException', ), + 4 => array( + 'var' => 'o4', + 'type' => TType::STRUCT, + 'class' => '\metastore\NoSuchObjectException', + ), ); } if (is_array($vals)) { - if (isset($vals['success'])) { - $this->success = $vals['success']; - } if (isset($vals['o1'])) { $this->o1 = $vals['o1']; } if (isset($vals['o2'])) { $this->o2 = $vals['o2']; } + if (isset($vals['o3'])) { + $this->o3 = $vals['o3']; + } + if (isset($vals['o4'])) { + $this->o4 = $vals['o4']; + } } } public function getName() { - return 'ThriftHiveMetastore_get_database_result'; + return 'ThriftHiveMetastore_create_table_with_environment_context_result'; } public function read($input) @@ -5826,26 +9020,34 @@ class ThriftHiveMetastore_get_database_result { } switch ($fid) { - case 0: + case 1: if ($ftype == TType::STRUCT) { - $this->success = new \metastore\Database(); - $xfer += $this->success->read($input); + $this->o1 = new \metastore\AlreadyExistsException(); + $xfer += $this->o1->read($input); } else { $xfer += $input->skip($ftype); } break; - case 1: + case 2: if ($ftype == TType::STRUCT) { - $this->o1 = new \metastore\NoSuchObjectException(); - $xfer += $this->o1->read($input); + $this->o2 = new \metastore\InvalidObjectException(); + $xfer += $this->o2->read($input); } else { $xfer += $input->skip($ftype); } break; - case 2: + case 3: if ($ftype == TType::STRUCT) { - $this->o2 = new \metastore\MetaException(); - $xfer += $this->o2->read($input); + $this->o3 = new \metastore\MetaException(); + $xfer += $this->o3->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; + case 4: + if ($ftype == TType::STRUCT) { + $this->o4 = new \metastore\NoSuchObjectException(); + $xfer += $this->o4->read($input); } else { $xfer += $input->skip($ftype); } @@ -5862,15 +9064,7 @@ class ThriftHiveMetastore_get_database_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_database_result'); - if ($this->success !== null) { - if (!is_object($this->success)) { - throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); - } - $xfer += $output->writeFieldBegin('success', TType::STRUCT, 0); - $xfer += $this->success->write($output); - $xfer += $output->writeFieldEnd(); - } + $xfer += $output->writeStructBegin('ThriftHiveMetastore_create_table_with_environment_context_result'); if ($this->o1 !== null) { $xfer += $output->writeFieldBegin('o1', TType::STRUCT, 1); $xfer += $this->o1->write($output); @@ -5881,6 +9075,16 @@ class ThriftHiveMetastore_get_database_result { $xfer += $this->o2->write($output); $xfer += $output->writeFieldEnd(); } + if ($this->o3 !== null) { + $xfer += $output->writeFieldBegin('o3', TType::STRUCT, 3); + $xfer += $this->o3->write($output); + $xfer += $output->writeFieldEnd(); + } + if ($this->o4 !== null) { + $xfer += $output->writeFieldBegin('o4', TType::STRUCT, 4); + $xfer += $this->o4->write($output); + $xfer += $output->writeFieldEnd(); + } $xfer += $output->writeFieldStop(); $xfer += $output->writeStructEnd(); return $xfer; @@ -5888,45 +9092,45 @@ class ThriftHiveMetastore_get_database_result { } -class ThriftHiveMetastore_drop_database_args { +class ThriftHiveMetastore_drop_table_args { static $_TSPEC; + public $dbname = null; public $name = null; public $deleteData = null; - public $cascade = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 1 => array( - 'var' => 'name', + 'var' => 'dbname', 'type' => TType::STRING, ), 2 => array( - 'var' => 'deleteData', - 'type' => TType::BOOL, + 'var' => 'name', + 'type' => TType::STRING, ), 3 => array( - 'var' => 'cascade', + 'var' => 'deleteData', 'type' => TType::BOOL, ), ); } if (is_array($vals)) { + if (isset($vals['dbname'])) { + $this->dbname = $vals['dbname']; + } if (isset($vals['name'])) { $this->name = $vals['name']; } if (isset($vals['deleteData'])) { $this->deleteData = $vals['deleteData']; } - if (isset($vals['cascade'])) { - $this->cascade = $vals['cascade']; - } } } public function getName() { - return 'ThriftHiveMetastore_drop_database_args'; + return 'ThriftHiveMetastore_drop_table_args'; } public function read($input) @@ -5946,21 +9150,21 @@ class ThriftHiveMetastore_drop_database_args { { case 1: if ($ftype == TType::STRING) { - $xfer += $input->readString($this->name); + $xfer += $input->readString($this->dbname); } else { $xfer += $input->skip($ftype); } break; case 2: - if ($ftype == TType::BOOL) { - $xfer += $input->readBool($this->deleteData); + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->name); } else { $xfer += $input->skip($ftype); } break; case 3: if ($ftype == TType::BOOL) { - $xfer += $input->readBool($this->cascade); + $xfer += $input->readBool($this->deleteData); } else { $xfer += $input->skip($ftype); } @@ -5977,22 +9181,22 @@ class ThriftHiveMetastore_drop_database_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_drop_database_args'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_drop_table_args'); + if ($this->dbname !== null) { + $xfer += $output->writeFieldBegin('dbname', TType::STRING, 1); + $xfer += $output->writeString($this->dbname); + $xfer += $output->writeFieldEnd(); + } if ($this->name !== null) { - $xfer += $output->writeFieldBegin('name', TType::STRING, 1); + $xfer += $output->writeFieldBegin('name', TType::STRING, 2); $xfer += $output->writeString($this->name); $xfer += $output->writeFieldEnd(); } if ($this->deleteData !== null) { - $xfer += $output->writeFieldBegin('deleteData', TType::BOOL, 2); + $xfer += $output->writeFieldBegin('deleteData', TType::BOOL, 3); $xfer += $output->writeBool($this->deleteData); $xfer += $output->writeFieldEnd(); } - if ($this->cascade !== null) { - $xfer += $output->writeFieldBegin('cascade', TType::BOOL, 3); - $xfer += $output->writeBool($this->cascade); - $xfer += $output->writeFieldEnd(); - } $xfer += $output->writeFieldStop(); $xfer += $output->writeStructEnd(); return $xfer; @@ -6000,11 +9204,10 @@ class ThriftHiveMetastore_drop_database_args { } -class ThriftHiveMetastore_drop_database_result { +class ThriftHiveMetastore_drop_table_result { static $_TSPEC; public $o1 = null; - public $o2 = null; public $o3 = null; public function __construct($vals=null) { @@ -6016,11 +9219,6 @@ class ThriftHiveMetastore_drop_database_result { 'class' => '\metastore\NoSuchObjectException', ), 2 => array( - 'var' => 'o2', - 'type' => TType::STRUCT, - 'class' => '\metastore\InvalidOperationException', - ), - 3 => array( 'var' => 'o3', 'type' => TType::STRUCT, 'class' => '\metastore\MetaException', @@ -6031,9 +9229,6 @@ class ThriftHiveMetastore_drop_database_result { if (isset($vals['o1'])) { $this->o1 = $vals['o1']; } - if (isset($vals['o2'])) { - $this->o2 = $vals['o2']; - } if (isset($vals['o3'])) { $this->o3 = $vals['o3']; } @@ -6041,7 +9236,7 @@ class ThriftHiveMetastore_drop_database_result { } public function getName() { - return 'ThriftHiveMetastore_drop_database_result'; + return 'ThriftHiveMetastore_drop_table_result'; } public function read($input) @@ -6069,14 +9264,6 @@ class ThriftHiveMetastore_drop_database_result { break; case 2: if ($ftype == TType::STRUCT) { - $this->o2 = new \metastore\InvalidOperationException(); - $xfer += $this->o2->read($input); - } else { - $xfer += $input->skip($ftype); - } - break; - case 3: - if ($ftype == TType::STRUCT) { $this->o3 = new \metastore\MetaException(); $xfer += $this->o3->read($input); } else { @@ -6095,19 +9282,14 @@ class ThriftHiveMetastore_drop_database_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_drop_database_result'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_drop_table_result'); if ($this->o1 !== null) { $xfer += $output->writeFieldBegin('o1', TType::STRUCT, 1); $xfer += $this->o1->write($output); $xfer += $output->writeFieldEnd(); } - if ($this->o2 !== null) { - $xfer += $output->writeFieldBegin('o2', TType::STRUCT, 2); - $xfer += $this->o2->write($output); - $xfer += $output->writeFieldEnd(); - } if ($this->o3 !== null) { - $xfer += $output->writeFieldBegin('o3', TType::STRUCT, 3); + $xfer += $output->writeFieldBegin('o3', TType::STRUCT, 2); $xfer += $this->o3->write($output); $xfer += $output->writeFieldEnd(); } @@ -6118,29 +9300,54 @@ class ThriftHiveMetastore_drop_database_result { } -class ThriftHiveMetastore_get_databases_args { +class ThriftHiveMetastore_drop_table_with_environment_context_args { static $_TSPEC; - public $pattern = null; + public $dbname = null; + public $name = null; + public $deleteData = null; + public $environment_context = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 1 => array( - 'var' => 'pattern', + 'var' => 'dbname', + 'type' => TType::STRING, + ), + 2 => array( + 'var' => 'name', 'type' => TType::STRING, ), + 3 => array( + 'var' => 'deleteData', + 'type' => TType::BOOL, + ), + 4 => array( + 'var' => 'environment_context', + 'type' => TType::STRUCT, + 'class' => '\metastore\EnvironmentContext', + ), ); } if (is_array($vals)) { - if (isset($vals['pattern'])) { - $this->pattern = $vals['pattern']; + if (isset($vals['dbname'])) { + $this->dbname = $vals['dbname']; + } + if (isset($vals['name'])) { + $this->name = $vals['name']; + } + if (isset($vals['deleteData'])) { + $this->deleteData = $vals['deleteData']; + } + if (isset($vals['environment_context'])) { + $this->environment_context = $vals['environment_context']; } } } public function getName() { - return 'ThriftHiveMetastore_get_databases_args'; + return 'ThriftHiveMetastore_drop_table_with_environment_context_args'; } public function read($input) @@ -6160,7 +9367,29 @@ class ThriftHiveMetastore_get_databases_args { { case 1: if ($ftype == TType::STRING) { - $xfer += $input->readString($this->pattern); + $xfer += $input->readString($this->dbname); + } else { + $xfer += $input->skip($ftype); + } + break; + case 2: + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->name); + } else { + $xfer += $input->skip($ftype); + } + break; + case 3: + if ($ftype == TType::BOOL) { + $xfer += $input->readBool($this->deleteData); + } else { + $xfer += $input->skip($ftype); + } + break; + case 4: + if ($ftype == TType::STRUCT) { + $this->environment_context = new \metastore\EnvironmentContext(); + $xfer += $this->environment_context->read($input); } else { $xfer += $input->skip($ftype); } @@ -6177,10 +9406,28 @@ class ThriftHiveMetastore_get_databases_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_databases_args'); - if ($this->pattern !== null) { - $xfer += $output->writeFieldBegin('pattern', TType::STRING, 1); - $xfer += $output->writeString($this->pattern); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_drop_table_with_environment_context_args'); + if ($this->dbname !== null) { + $xfer += $output->writeFieldBegin('dbname', TType::STRING, 1); + $xfer += $output->writeString($this->dbname); + $xfer += $output->writeFieldEnd(); + } + if ($this->name !== null) { + $xfer += $output->writeFieldBegin('name', TType::STRING, 2); + $xfer += $output->writeString($this->name); + $xfer += $output->writeFieldEnd(); + } + if ($this->deleteData !== null) { + $xfer += $output->writeFieldBegin('deleteData', TType::BOOL, 3); + $xfer += $output->writeBool($this->deleteData); + $xfer += $output->writeFieldEnd(); + } + if ($this->environment_context !== null) { + if (!is_object($this->environment_context)) { + throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); + } + $xfer += $output->writeFieldBegin('environment_context', TType::STRUCT, 4); + $xfer += $this->environment_context->write($output); $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -6190,42 +9437,39 @@ class ThriftHiveMetastore_get_databases_args { } -class ThriftHiveMetastore_get_databases_result { +class ThriftHiveMetastore_drop_table_with_environment_context_result { static $_TSPEC; - public $success = null; public $o1 = null; + public $o3 = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( - 0 => array( - 'var' => 'success', - 'type' => TType::LST, - 'etype' => TType::STRING, - 'elem' => array( - 'type' => TType::STRING, - ), - ), 1 => array( 'var' => 'o1', 'type' => TType::STRUCT, + 'class' => '\metastore\NoSuchObjectException', + ), + 2 => array( + 'var' => 'o3', + 'type' => TType::STRUCT, 'class' => '\metastore\MetaException', ), ); } if (is_array($vals)) { - if (isset($vals['success'])) { - $this->success = $vals['success']; - } if (isset($vals['o1'])) { $this->o1 = $vals['o1']; } + if (isset($vals['o3'])) { + $this->o3 = $vals['o3']; + } } } public function getName() { - return 'ThriftHiveMetastore_get_databases_result'; + return 'ThriftHiveMetastore_drop_table_with_environment_context_result'; } public function read($input) @@ -6243,27 +9487,18 @@ class ThriftHiveMetastore_get_databases_result { } switch ($fid) { - case 0: - if ($ftype == TType::LST) { - $this->success = array(); - $_size320 = 0; - $_etype323 = 0; - $xfer += $input->readListBegin($_etype323, $_size320); - for ($_i324 = 0; $_i324 < $_size320; ++$_i324) - { - $elem325 = null; - $xfer += $input->readString($elem325); - $this->success []= $elem325; - } - $xfer += $input->readListEnd(); + case 1: + if ($ftype == TType::STRUCT) { + $this->o1 = new \metastore\NoSuchObjectException(); + $xfer += $this->o1->read($input); } else { $xfer += $input->skip($ftype); } break; - case 1: + case 2: if ($ftype == TType::STRUCT) { - $this->o1 = new \metastore\MetaException(); - $xfer += $this->o1->read($input); + $this->o3 = new \metastore\MetaException(); + $xfer += $this->o3->read($input); } else { $xfer += $input->skip($ftype); } @@ -6280,29 +9515,17 @@ class ThriftHiveMetastore_get_databases_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_databases_result'); - if ($this->success !== null) { - if (!is_array($this->success)) { - throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); - } - $xfer += $output->writeFieldBegin('success', TType::LST, 0); - { - $output->writeListBegin(TType::STRING, count($this->success)); - { - foreach ($this->success as $iter326) - { - $xfer += $output->writeString($iter326); - } - } - $output->writeListEnd(); - } - $xfer += $output->writeFieldEnd(); - } + $xfer += $output->writeStructBegin('ThriftHiveMetastore_drop_table_with_environment_context_result'); if ($this->o1 !== null) { $xfer += $output->writeFieldBegin('o1', TType::STRUCT, 1); $xfer += $this->o1->write($output); $xfer += $output->writeFieldEnd(); } + if ($this->o3 !== null) { + $xfer += $output->writeFieldBegin('o3', TType::STRUCT, 2); + $xfer += $this->o3->write($output); + $xfer += $output->writeFieldEnd(); + } $xfer += $output->writeFieldStop(); $xfer += $output->writeStructEnd(); return $xfer; @@ -6310,19 +9533,37 @@ class ThriftHiveMetastore_get_databases_result { } -class ThriftHiveMetastore_get_all_databases_args { +class ThriftHiveMetastore_get_tables_args { static $_TSPEC; + public $db_name = null; + public $pattern = null; - public function __construct() { + public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( + 1 => array( + 'var' => 'db_name', + 'type' => TType::STRING, + ), + 2 => array( + 'var' => 'pattern', + 'type' => TType::STRING, + ), ); } + if (is_array($vals)) { + if (isset($vals['db_name'])) { + $this->db_name = $vals['db_name']; + } + if (isset($vals['pattern'])) { + $this->pattern = $vals['pattern']; + } + } } public function getName() { - return 'ThriftHiveMetastore_get_all_databases_args'; + return 'ThriftHiveMetastore_get_tables_args'; } public function read($input) @@ -6340,6 +9581,20 @@ class ThriftHiveMetastore_get_all_databases_args { } switch ($fid) { + case 1: + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->db_name); + } else { + $xfer += $input->skip($ftype); + } + break; + case 2: + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->pattern); + } else { + $xfer += $input->skip($ftype); + } + break; default: $xfer += $input->skip($ftype); break; @@ -6352,7 +9607,17 @@ class ThriftHiveMetastore_get_all_databases_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_all_databases_args'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_tables_args'); + if ($this->db_name !== null) { + $xfer += $output->writeFieldBegin('db_name', TType::STRING, 1); + $xfer += $output->writeString($this->db_name); + $xfer += $output->writeFieldEnd(); + } + if ($this->pattern !== null) { + $xfer += $output->writeFieldBegin('pattern', TType::STRING, 2); + $xfer += $output->writeString($this->pattern); + $xfer += $output->writeFieldEnd(); + } $xfer += $output->writeFieldStop(); $xfer += $output->writeStructEnd(); return $xfer; @@ -6360,7 +9625,7 @@ class ThriftHiveMetastore_get_all_databases_args { } -class ThriftHiveMetastore_get_all_databases_result { +class ThriftHiveMetastore_get_tables_result { static $_TSPEC; public $success = null; @@ -6395,7 +9660,7 @@ class ThriftHiveMetastore_get_all_databases_result { } public function getName() { - return 'ThriftHiveMetastore_get_all_databases_result'; + return 'ThriftHiveMetastore_get_tables_result'; } public function read($input) @@ -6416,14 +9681,14 @@ class ThriftHiveMetastore_get_all_databases_result { case 0: if ($ftype == TType::LST) { $this->success = array(); - $_size327 = 0; - $_etype330 = 0; - $xfer += $input->readListBegin($_etype330, $_size327); - for ($_i331 = 0; $_i331 < $_size327; ++$_i331) + $_size400 = 0; + $_etype403 = 0; + $xfer += $input->readListBegin($_etype403, $_size400); + for ($_i404 = 0; $_i404 < $_size400; ++$_i404) { - $elem332 = null; - $xfer += $input->readString($elem332); - $this->success []= $elem332; + $elem405 = null; + $xfer += $input->readString($elem405); + $this->success []= $elem405; } $xfer += $input->readListEnd(); } else { @@ -6450,7 +9715,7 @@ class ThriftHiveMetastore_get_all_databases_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_all_databases_result'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_tables_result'); if ($this->success !== null) { if (!is_array($this->success)) { throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); @@ -6459,9 +9724,9 @@ class ThriftHiveMetastore_get_all_databases_result { { $output->writeListBegin(TType::STRING, count($this->success)); { - foreach ($this->success as $iter333) + foreach ($this->success as $iter406) { - $xfer += $output->writeString($iter333); + $xfer += $output->writeString($iter406); } } $output->writeListEnd(); @@ -6480,38 +9745,29 @@ class ThriftHiveMetastore_get_all_databases_result { } -class ThriftHiveMetastore_alter_database_args { +class ThriftHiveMetastore_get_all_tables_args { static $_TSPEC; - public $dbname = null; - public $db = null; + public $db_name = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 1 => array( - 'var' => 'dbname', + 'var' => 'db_name', 'type' => TType::STRING, - ), - 2 => array( - 'var' => 'db', - 'type' => TType::STRUCT, - 'class' => '\metastore\Database', - ), - ); - } - if (is_array($vals)) { - if (isset($vals['dbname'])) { - $this->dbname = $vals['dbname']; - } - if (isset($vals['db'])) { - $this->db = $vals['db']; + ), + ); + } + if (is_array($vals)) { + if (isset($vals['db_name'])) { + $this->db_name = $vals['db_name']; } } } public function getName() { - return 'ThriftHiveMetastore_alter_database_args'; + return 'ThriftHiveMetastore_get_all_tables_args'; } public function read($input) @@ -6531,15 +9787,7 @@ class ThriftHiveMetastore_alter_database_args { { case 1: if ($ftype == TType::STRING) { - $xfer += $input->readString($this->dbname); - } else { - $xfer += $input->skip($ftype); - } - break; - case 2: - if ($ftype == TType::STRUCT) { - $this->db = new \metastore\Database(); - $xfer += $this->db->read($input); + $xfer += $input->readString($this->db_name); } else { $xfer += $input->skip($ftype); } @@ -6556,18 +9804,10 @@ class ThriftHiveMetastore_alter_database_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_alter_database_args'); - if ($this->dbname !== null) { - $xfer += $output->writeFieldBegin('dbname', TType::STRING, 1); - $xfer += $output->writeString($this->dbname); - $xfer += $output->writeFieldEnd(); - } - if ($this->db !== null) { - if (!is_object($this->db)) { - throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); - } - $xfer += $output->writeFieldBegin('db', TType::STRUCT, 2); - $xfer += $this->db->write($output); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_all_tables_args'); + if ($this->db_name !== null) { + $xfer += $output->writeFieldBegin('db_name', TType::STRING, 1); + $xfer += $output->writeString($this->db_name); $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -6577,39 +9817,42 @@ class ThriftHiveMetastore_alter_database_args { } -class ThriftHiveMetastore_alter_database_result { +class ThriftHiveMetastore_get_all_tables_result { static $_TSPEC; + public $success = null; public $o1 = null; - public $o2 = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( + 0 => array( + 'var' => 'success', + 'type' => TType::LST, + 'etype' => TType::STRING, + 'elem' => array( + 'type' => TType::STRING, + ), + ), 1 => array( 'var' => 'o1', 'type' => TType::STRUCT, 'class' => '\metastore\MetaException', ), - 2 => array( - 'var' => 'o2', - 'type' => TType::STRUCT, - 'class' => '\metastore\NoSuchObjectException', - ), ); } if (is_array($vals)) { + if (isset($vals['success'])) { + $this->success = $vals['success']; + } if (isset($vals['o1'])) { $this->o1 = $vals['o1']; } - if (isset($vals['o2'])) { - $this->o2 = $vals['o2']; - } } } public function getName() { - return 'ThriftHiveMetastore_alter_database_result'; + return 'ThriftHiveMetastore_get_all_tables_result'; } public function read($input) @@ -6627,18 +9870,27 @@ class ThriftHiveMetastore_alter_database_result { } switch ($fid) { - case 1: - if ($ftype == TType::STRUCT) { - $this->o1 = new \metastore\MetaException(); - $xfer += $this->o1->read($input); + case 0: + if ($ftype == TType::LST) { + $this->success = array(); + $_size407 = 0; + $_etype410 = 0; + $xfer += $input->readListBegin($_etype410, $_size407); + for ($_i411 = 0; $_i411 < $_size407; ++$_i411) + { + $elem412 = null; + $xfer += $input->readString($elem412); + $this->success []= $elem412; + } + $xfer += $input->readListEnd(); } else { $xfer += $input->skip($ftype); } break; - case 2: + case 1: if ($ftype == TType::STRUCT) { - $this->o2 = new \metastore\NoSuchObjectException(); - $xfer += $this->o2->read($input); + $this->o1 = new \metastore\MetaException(); + $xfer += $this->o1->read($input); } else { $xfer += $input->skip($ftype); } @@ -6655,17 +9907,29 @@ class ThriftHiveMetastore_alter_database_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_alter_database_result'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_all_tables_result'); + if ($this->success !== null) { + if (!is_array($this->success)) { + throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); + } + $xfer += $output->writeFieldBegin('success', TType::LST, 0); + { + $output->writeListBegin(TType::STRING, count($this->success)); + { + foreach ($this->success as $iter413) + { + $xfer += $output->writeString($iter413); + } + } + $output->writeListEnd(); + } + $xfer += $output->writeFieldEnd(); + } if ($this->o1 !== null) { $xfer += $output->writeFieldBegin('o1', TType::STRUCT, 1); $xfer += $this->o1->write($output); $xfer += $output->writeFieldEnd(); } - if ($this->o2 !== null) { - $xfer += $output->writeFieldBegin('o2', TType::STRUCT, 2); - $xfer += $this->o2->write($output); - $xfer += $output->writeFieldEnd(); - } $xfer += $output->writeFieldStop(); $xfer += $output->writeStructEnd(); return $xfer; @@ -6673,29 +9937,37 @@ class ThriftHiveMetastore_alter_database_result { } -class ThriftHiveMetastore_get_type_args { +class ThriftHiveMetastore_get_table_args { static $_TSPEC; - public $name = null; + public $dbname = null; + public $tbl_name = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 1 => array( - 'var' => 'name', + 'var' => 'dbname', + 'type' => TType::STRING, + ), + 2 => array( + 'var' => 'tbl_name', 'type' => TType::STRING, ), ); } if (is_array($vals)) { - if (isset($vals['name'])) { - $this->name = $vals['name']; + if (isset($vals['dbname'])) { + $this->dbname = $vals['dbname']; + } + if (isset($vals['tbl_name'])) { + $this->tbl_name = $vals['tbl_name']; } } } public function getName() { - return 'ThriftHiveMetastore_get_type_args'; + return 'ThriftHiveMetastore_get_table_args'; } public function read($input) @@ -6715,7 +9987,14 @@ class ThriftHiveMetastore_get_type_args { { case 1: if ($ftype == TType::STRING) { - $xfer += $input->readString($this->name); + $xfer += $input->readString($this->dbname); + } else { + $xfer += $input->skip($ftype); + } + break; + case 2: + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->tbl_name); } else { $xfer += $input->skip($ftype); } @@ -6732,10 +10011,15 @@ class ThriftHiveMetastore_get_type_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_type_args'); - if ($this->name !== null) { - $xfer += $output->writeFieldBegin('name', TType::STRING, 1); - $xfer += $output->writeString($this->name); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_table_args'); + if ($this->dbname !== null) { + $xfer += $output->writeFieldBegin('dbname', TType::STRING, 1); + $xfer += $output->writeString($this->dbname); + $xfer += $output->writeFieldEnd(); + } + if ($this->tbl_name !== null) { + $xfer += $output->writeFieldBegin('tbl_name', TType::STRING, 2); + $xfer += $output->writeString($this->tbl_name); $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -6745,7 +10029,7 @@ class ThriftHiveMetastore_get_type_args { } -class ThriftHiveMetastore_get_type_result { +class ThriftHiveMetastore_get_table_result { static $_TSPEC; public $success = null; @@ -6758,7 +10042,7 @@ class ThriftHiveMetastore_get_type_result { 0 => array( 'var' => 'success', 'type' => TType::STRUCT, - 'class' => '\metastore\Type', + 'class' => '\metastore\Table', ), 1 => array( 'var' => 'o1', @@ -6786,7 +10070,7 @@ class ThriftHiveMetastore_get_type_result { } public function getName() { - return 'ThriftHiveMetastore_get_type_result'; + return 'ThriftHiveMetastore_get_table_result'; } public function read($input) @@ -6806,7 +10090,7 @@ class ThriftHiveMetastore_get_type_result { { case 0: if ($ftype == TType::STRUCT) { - $this->success = new \metastore\Type(); + $this->success = new \metastore\Table(); $xfer += $this->success->read($input); } else { $xfer += $input->skip($ftype); @@ -6840,7 +10124,7 @@ class ThriftHiveMetastore_get_type_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_type_result'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_table_result'); if ($this->success !== null) { if (!is_object($this->success)) { throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); @@ -6866,30 +10150,41 @@ class ThriftHiveMetastore_get_type_result { } -class ThriftHiveMetastore_create_type_args { +class ThriftHiveMetastore_get_table_objects_by_name_args { static $_TSPEC; - public $type = null; + public $dbname = null; + public $tbl_names = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 1 => array( - 'var' => 'type', - 'type' => TType::STRUCT, - 'class' => '\metastore\Type', + 'var' => 'dbname', + 'type' => TType::STRING, + ), + 2 => array( + 'var' => 'tbl_names', + 'type' => TType::LST, + 'etype' => TType::STRING, + 'elem' => array( + 'type' => TType::STRING, + ), ), ); } if (is_array($vals)) { - if (isset($vals['type'])) { - $this->type = $vals['type']; + if (isset($vals['dbname'])) { + $this->dbname = $vals['dbname']; + } + if (isset($vals['tbl_names'])) { + $this->tbl_names = $vals['tbl_names']; } } } public function getName() { - return 'ThriftHiveMetastore_create_type_args'; + return 'ThriftHiveMetastore_get_table_objects_by_name_args'; } public function read($input) @@ -6908,9 +10203,25 @@ class ThriftHiveMetastore_create_type_args { switch ($fid) { case 1: - if ($ftype == TType::STRUCT) { - $this->type = new \metastore\Type(); - $xfer += $this->type->read($input); + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->dbname); + } else { + $xfer += $input->skip($ftype); + } + break; + case 2: + if ($ftype == TType::LST) { + $this->tbl_names = array(); + $_size414 = 0; + $_etype417 = 0; + $xfer += $input->readListBegin($_etype417, $_size414); + for ($_i418 = 0; $_i418 < $_size414; ++$_i418) + { + $elem419 = null; + $xfer += $input->readString($elem419); + $this->tbl_names []= $elem419; + } + $xfer += $input->readListEnd(); } else { $xfer += $input->skip($ftype); } @@ -6927,13 +10238,27 @@ class ThriftHiveMetastore_create_type_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_create_type_args'); - if ($this->type !== null) { - if (!is_object($this->type)) { + $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_table_objects_by_name_args'); + if ($this->dbname !== null) { + $xfer += $output->writeFieldBegin('dbname', TType::STRING, 1); + $xfer += $output->writeString($this->dbname); + $xfer += $output->writeFieldEnd(); + } + if ($this->tbl_names !== null) { + if (!is_array($this->tbl_names)) { throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); } - $xfer += $output->writeFieldBegin('type', TType::STRUCT, 1); - $xfer += $this->type->write($output); + $xfer += $output->writeFieldBegin('tbl_names', TType::LST, 2); + { + $output->writeListBegin(TType::STRING, count($this->tbl_names)); + { + foreach ($this->tbl_names as $iter420) + { + $xfer += $output->writeString($iter420); + } + } + $output->writeListEnd(); + } $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -6943,7 +10268,7 @@ class ThriftHiveMetastore_create_type_args { } -class ThriftHiveMetastore_create_type_result { +class ThriftHiveMetastore_get_table_objects_by_name_result { static $_TSPEC; public $success = null; @@ -6956,22 +10281,27 @@ class ThriftHiveMetastore_create_type_result { self::$_TSPEC = array( 0 => array( 'var' => 'success', - 'type' => TType::BOOL, + 'type' => TType::LST, + 'etype' => TType::STRUCT, + 'elem' => array( + 'type' => TType::STRUCT, + 'class' => '\metastore\Table', + ), ), 1 => array( 'var' => 'o1', 'type' => TType::STRUCT, - 'class' => '\metastore\AlreadyExistsException', + 'class' => '\metastore\MetaException', ), 2 => array( 'var' => 'o2', 'type' => TType::STRUCT, - 'class' => '\metastore\InvalidObjectException', + 'class' => '\metastore\InvalidOperationException', ), 3 => array( 'var' => 'o3', 'type' => TType::STRUCT, - 'class' => '\metastore\MetaException', + 'class' => '\metastore\UnknownDBException', ), ); } @@ -6992,7 +10322,7 @@ class ThriftHiveMetastore_create_type_result { } public function getName() { - return 'ThriftHiveMetastore_create_type_result'; + return 'ThriftHiveMetastore_get_table_objects_by_name_result'; } public function read($input) @@ -7011,15 +10341,26 @@ class ThriftHiveMetastore_create_type_result { switch ($fid) { case 0: - if ($ftype == TType::BOOL) { - $xfer += $input->readBool($this->success); + if ($ftype == TType::LST) { + $this->success = array(); + $_size421 = 0; + $_etype424 = 0; + $xfer += $input->readListBegin($_etype424, $_size421); + for ($_i425 = 0; $_i425 < $_size421; ++$_i425) + { + $elem426 = null; + $elem426 = new \metastore\Table(); + $xfer += $elem426->read($input); + $this->success []= $elem426; + } + $xfer += $input->readListEnd(); } else { $xfer += $input->skip($ftype); } break; case 1: if ($ftype == TType::STRUCT) { - $this->o1 = new \metastore\AlreadyExistsException(); + $this->o1 = new \metastore\MetaException(); $xfer += $this->o1->read($input); } else { $xfer += $input->skip($ftype); @@ -7027,7 +10368,7 @@ class ThriftHiveMetastore_create_type_result { break; case 2: if ($ftype == TType::STRUCT) { - $this->o2 = new \metastore\InvalidObjectException(); + $this->o2 = new \metastore\InvalidOperationException(); $xfer += $this->o2->read($input); } else { $xfer += $input->skip($ftype); @@ -7035,7 +10376,7 @@ class ThriftHiveMetastore_create_type_result { break; case 3: if ($ftype == TType::STRUCT) { - $this->o3 = new \metastore\MetaException(); + $this->o3 = new \metastore\UnknownDBException(); $xfer += $this->o3->read($input); } else { $xfer += $input->skip($ftype); @@ -7053,10 +10394,22 @@ class ThriftHiveMetastore_create_type_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_create_type_result'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_table_objects_by_name_result'); if ($this->success !== null) { - $xfer += $output->writeFieldBegin('success', TType::BOOL, 0); - $xfer += $output->writeBool($this->success); + if (!is_array($this->success)) { + throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); + } + $xfer += $output->writeFieldBegin('success', TType::LST, 0); + { + $output->writeListBegin(TType::STRUCT, count($this->success)); + { + foreach ($this->success as $iter427) + { + $xfer += $iter427->write($output); + } + } + $output->writeListEnd(); + } $xfer += $output->writeFieldEnd(); } if ($this->o1 !== null) { @@ -7081,29 +10434,45 @@ class ThriftHiveMetastore_create_type_result { } -class ThriftHiveMetastore_drop_type_args { +class ThriftHiveMetastore_get_table_names_by_filter_args { static $_TSPEC; - public $type = null; + public $dbname = null; + public $filter = null; + public $max_tables = -1; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 1 => array( - 'var' => 'type', + 'var' => 'dbname', + 'type' => TType::STRING, + ), + 2 => array( + 'var' => 'filter', 'type' => TType::STRING, ), + 3 => array( + 'var' => 'max_tables', + 'type' => TType::I16, + ), ); } if (is_array($vals)) { - if (isset($vals['type'])) { - $this->type = $vals['type']; + if (isset($vals['dbname'])) { + $this->dbname = $vals['dbname']; + } + if (isset($vals['filter'])) { + $this->filter = $vals['filter']; + } + if (isset($vals['max_tables'])) { + $this->max_tables = $vals['max_tables']; } } } public function getName() { - return 'ThriftHiveMetastore_drop_type_args'; + return 'ThriftHiveMetastore_get_table_names_by_filter_args'; } public function read($input) @@ -7123,7 +10492,21 @@ class ThriftHiveMetastore_drop_type_args { { case 1: if ($ftype == TType::STRING) { - $xfer += $input->readString($this->type); + $xfer += $input->readString($this->dbname); + } else { + $xfer += $input->skip($ftype); + } + break; + case 2: + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->filter); + } else { + $xfer += $input->skip($ftype); + } + break; + case 3: + if ($ftype == TType::I16) { + $xfer += $input->readI16($this->max_tables); } else { $xfer += $input->skip($ftype); } @@ -7140,10 +10523,20 @@ class ThriftHiveMetastore_drop_type_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_drop_type_args'); - if ($this->type !== null) { - $xfer += $output->writeFieldBegin('type', TType::STRING, 1); - $xfer += $output->writeString($this->type); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_table_names_by_filter_args'); + if ($this->dbname !== null) { + $xfer += $output->writeFieldBegin('dbname', TType::STRING, 1); + $xfer += $output->writeString($this->dbname); + $xfer += $output->writeFieldEnd(); + } + if ($this->filter !== null) { + $xfer += $output->writeFieldBegin('filter', TType::STRING, 2); + $xfer += $output->writeString($this->filter); + $xfer += $output->writeFieldEnd(); + } + if ($this->max_tables !== null) { + $xfer += $output->writeFieldBegin('max_tables', TType::I16, 3); + $xfer += $output->writeI16($this->max_tables); $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -7153,19 +10546,24 @@ class ThriftHiveMetastore_drop_type_args { } -class ThriftHiveMetastore_drop_type_result { +class ThriftHiveMetastore_get_table_names_by_filter_result { static $_TSPEC; public $success = null; public $o1 = null; public $o2 = null; + public $o3 = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 0 => array( 'var' => 'success', - 'type' => TType::BOOL, + 'type' => TType::LST, + 'etype' => TType::STRING, + 'elem' => array( + 'type' => TType::STRING, + ), ), 1 => array( 'var' => 'o1', @@ -7175,7 +10573,12 @@ class ThriftHiveMetastore_drop_type_result { 2 => array( 'var' => 'o2', 'type' => TType::STRUCT, - 'class' => '\metastore\NoSuchObjectException', + 'class' => '\metastore\InvalidOperationException', + ), + 3 => array( + 'var' => 'o3', + 'type' => TType::STRUCT, + 'class' => '\metastore\UnknownDBException', ), ); } @@ -7189,11 +10592,14 @@ class ThriftHiveMetastore_drop_type_result { if (isset($vals['o2'])) { $this->o2 = $vals['o2']; } + if (isset($vals['o3'])) { + $this->o3 = $vals['o3']; + } } } public function getName() { - return 'ThriftHiveMetastore_drop_type_result'; + return 'ThriftHiveMetastore_get_table_names_by_filter_result'; } public function read($input) @@ -7212,8 +10618,18 @@ class ThriftHiveMetastore_drop_type_result { switch ($fid) { case 0: - if ($ftype == TType::BOOL) { - $xfer += $input->readBool($this->success); + if ($ftype == TType::LST) { + $this->success = array(); + $_size428 = 0; + $_etype431 = 0; + $xfer += $input->readListBegin($_etype431, $_size428); + for ($_i432 = 0; $_i432 < $_size428; ++$_i432) + { + $elem433 = null; + $xfer += $input->readString($elem433); + $this->success []= $elem433; + } + $xfer += $input->readListEnd(); } else { $xfer += $input->skip($ftype); } @@ -7228,12 +10644,20 @@ class ThriftHiveMetastore_drop_type_result { break; case 2: if ($ftype == TType::STRUCT) { - $this->o2 = new \metastore\NoSuchObjectException(); + $this->o2 = new \metastore\InvalidOperationException(); $xfer += $this->o2->read($input); } else { $xfer += $input->skip($ftype); } break; + case 3: + if ($ftype == TType::STRUCT) { + $this->o3 = new \metastore\UnknownDBException(); + $xfer += $this->o3->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; default: $xfer += $input->skip($ftype); break; @@ -7246,10 +10670,22 @@ class ThriftHiveMetastore_drop_type_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_drop_type_result'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_table_names_by_filter_result'); if ($this->success !== null) { - $xfer += $output->writeFieldBegin('success', TType::BOOL, 0); - $xfer += $output->writeBool($this->success); + if (!is_array($this->success)) { + throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); + } + $xfer += $output->writeFieldBegin('success', TType::LST, 0); + { + $output->writeListBegin(TType::STRING, count($this->success)); + { + foreach ($this->success as $iter434) + { + $xfer += $output->writeString($iter434); + } + } + $output->writeListEnd(); + } $xfer += $output->writeFieldEnd(); } if ($this->o1 !== null) { @@ -7262,6 +10698,11 @@ class ThriftHiveMetastore_drop_type_result { $xfer += $this->o2->write($output); $xfer += $output->writeFieldEnd(); } + if ($this->o3 !== null) { + $xfer += $output->writeFieldBegin('o3', TType::STRUCT, 3); + $xfer += $this->o3->write($output); + $xfer += $output->writeFieldEnd(); + } $xfer += $output->writeFieldStop(); $xfer += $output->writeStructEnd(); return $xfer; @@ -7269,29 +10710,46 @@ class ThriftHiveMetastore_drop_type_result { } -class ThriftHiveMetastore_get_type_all_args { +class ThriftHiveMetastore_alter_table_args { static $_TSPEC; - public $name = null; + public $dbname = null; + public $tbl_name = null; + public $new_tbl = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 1 => array( - 'var' => 'name', + 'var' => 'dbname', + 'type' => TType::STRING, + ), + 2 => array( + 'var' => 'tbl_name', 'type' => TType::STRING, ), + 3 => array( + 'var' => 'new_tbl', + 'type' => TType::STRUCT, + 'class' => '\metastore\Table', + ), ); } if (is_array($vals)) { - if (isset($vals['name'])) { - $this->name = $vals['name']; + if (isset($vals['dbname'])) { + $this->dbname = $vals['dbname']; + } + if (isset($vals['tbl_name'])) { + $this->tbl_name = $vals['tbl_name']; + } + if (isset($vals['new_tbl'])) { + $this->new_tbl = $vals['new_tbl']; } } } public function getName() { - return 'ThriftHiveMetastore_get_type_all_args'; + return 'ThriftHiveMetastore_alter_table_args'; } public function read($input) @@ -7311,7 +10769,22 @@ class ThriftHiveMetastore_get_type_all_args { { case 1: if ($ftype == TType::STRING) { - $xfer += $input->readString($this->name); + $xfer += $input->readString($this->dbname); + } else { + $xfer += $input->skip($ftype); + } + break; + case 2: + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->tbl_name); + } else { + $xfer += $input->skip($ftype); + } + break; + case 3: + if ($ftype == TType::STRUCT) { + $this->new_tbl = new \metastore\Table(); + $xfer += $this->new_tbl->read($input); } else { $xfer += $input->skip($ftype); } @@ -7328,10 +10801,23 @@ class ThriftHiveMetastore_get_type_all_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_type_all_args'); - if ($this->name !== null) { - $xfer += $output->writeFieldBegin('name', TType::STRING, 1); - $xfer += $output->writeString($this->name); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_alter_table_args'); + if ($this->dbname !== null) { + $xfer += $output->writeFieldBegin('dbname', TType::STRING, 1); + $xfer += $output->writeString($this->dbname); + $xfer += $output->writeFieldEnd(); + } + if ($this->tbl_name !== null) { + $xfer += $output->writeFieldBegin('tbl_name', TType::STRING, 2); + $xfer += $output->writeString($this->tbl_name); + $xfer += $output->writeFieldEnd(); + } + if ($this->new_tbl !== null) { + if (!is_object($this->new_tbl)) { + throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); + } + $xfer += $output->writeFieldBegin('new_tbl', TType::STRUCT, 3); + $xfer += $this->new_tbl->write($output); $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -7341,29 +10827,21 @@ class ThriftHiveMetastore_get_type_all_args { } -class ThriftHiveMetastore_get_type_all_result { +class ThriftHiveMetastore_alter_table_result { static $_TSPEC; - public $success = null; + public $o1 = null; public $o2 = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( - 0 => array( - 'var' => 'success', - 'type' => TType::MAP, - 'ktype' => TType::STRING, - 'vtype' => TType::STRUCT, - 'key' => array( - 'type' => TType::STRING, - ), - 'val' => array( - 'type' => TType::STRUCT, - 'class' => '\metastore\Type', - ), - ), 1 => array( + 'var' => 'o1', + 'type' => TType::STRUCT, + 'class' => '\metastore\InvalidOperationException', + ), + 2 => array( 'var' => 'o2', 'type' => TType::STRUCT, 'class' => '\metastore\MetaException', @@ -7371,8 +10849,8 @@ class ThriftHiveMetastore_get_type_all_result { ); } if (is_array($vals)) { - if (isset($vals['success'])) { - $this->success = $vals['success']; + if (isset($vals['o1'])) { + $this->o1 = $vals['o1']; } if (isset($vals['o2'])) { $this->o2 = $vals['o2']; @@ -7381,7 +10859,7 @@ class ThriftHiveMetastore_get_type_all_result { } public function getName() { - return 'ThriftHiveMetastore_get_type_all_result'; + return 'ThriftHiveMetastore_alter_table_result'; } public function read($input) @@ -7399,28 +10877,15 @@ class ThriftHiveMetastore_get_type_all_result { } switch ($fid) { - case 0: - if ($ftype == TType::MAP) { - $this->success = array(); - $_size334 = 0; - $_ktype335 = 0; - $_vtype336 = 0; - $xfer += $input->readMapBegin($_ktype335, $_vtype336, $_size334); - for ($_i338 = 0; $_i338 < $_size334; ++$_i338) - { - $key339 = ''; - $val340 = new \metastore\Type(); - $xfer += $input->readString($key339); - $val340 = new \metastore\Type(); - $xfer += $val340->read($input); - $this->success[$key339] = $val340; - } - $xfer += $input->readMapEnd(); + case 1: + if ($ftype == TType::STRUCT) { + $this->o1 = new \metastore\InvalidOperationException(); + $xfer += $this->o1->read($input); } else { $xfer += $input->skip($ftype); } break; - case 1: + case 2: if ($ftype == TType::STRUCT) { $this->o2 = new \metastore\MetaException(); $xfer += $this->o2->read($input); @@ -7440,27 +10905,14 @@ class ThriftHiveMetastore_get_type_all_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_type_all_result'); - if ($this->success !== null) { - if (!is_array($this->success)) { - throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); - } - $xfer += $output->writeFieldBegin('success', TType::MAP, 0); - { - $output->writeMapBegin(TType::STRING, TType::STRUCT, count($this->success)); - { - foreach ($this->success as $kiter341 => $viter342) - { - $xfer += $output->writeString($kiter341); - $xfer += $viter342->write($output); - } - } - $output->writeMapEnd(); - } + $xfer += $output->writeStructBegin('ThriftHiveMetastore_alter_table_result'); + if ($this->o1 !== null) { + $xfer += $output->writeFieldBegin('o1', TType::STRUCT, 1); + $xfer += $this->o1->write($output); $xfer += $output->writeFieldEnd(); } if ($this->o2 !== null) { - $xfer += $output->writeFieldBegin('o2', TType::STRUCT, 1); + $xfer += $output->writeFieldBegin('o2', TType::STRUCT, 2); $xfer += $this->o2->write($output); $xfer += $output->writeFieldEnd(); } @@ -7471,37 +10923,55 @@ class ThriftHiveMetastore_get_type_all_result { } -class ThriftHiveMetastore_get_fields_args { +class ThriftHiveMetastore_alter_table_with_environment_context_args { static $_TSPEC; - public $db_name = null; - public $table_name = null; - + public $dbname = null; + public $tbl_name = null; + public $new_tbl = null; + public $environment_context = null; + public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 1 => array( - 'var' => 'db_name', + 'var' => 'dbname', 'type' => TType::STRING, ), 2 => array( - 'var' => 'table_name', + 'var' => 'tbl_name', 'type' => TType::STRING, ), + 3 => array( + 'var' => 'new_tbl', + 'type' => TType::STRUCT, + 'class' => '\metastore\Table', + ), + 4 => array( + 'var' => 'environment_context', + 'type' => TType::STRUCT, + 'class' => '\metastore\EnvironmentContext', + ), ); } if (is_array($vals)) { - if (isset($vals['db_name'])) { - $this->db_name = $vals['db_name']; + if (isset($vals['dbname'])) { + $this->dbname = $vals['dbname']; } - if (isset($vals['table_name'])) { - $this->table_name = $vals['table_name']; + if (isset($vals['tbl_name'])) { + $this->tbl_name = $vals['tbl_name']; + } + if (isset($vals['new_tbl'])) { + $this->new_tbl = $vals['new_tbl']; + } + if (isset($vals['environment_context'])) { + $this->environment_context = $vals['environment_context']; } } } public function getName() { - return 'ThriftHiveMetastore_get_fields_args'; + return 'ThriftHiveMetastore_alter_table_with_environment_context_args'; } public function read($input) @@ -7521,14 +10991,30 @@ class ThriftHiveMetastore_get_fields_args { { case 1: if ($ftype == TType::STRING) { - $xfer += $input->readString($this->db_name); + $xfer += $input->readString($this->dbname); } else { $xfer += $input->skip($ftype); } break; case 2: if ($ftype == TType::STRING) { - $xfer += $input->readString($this->table_name); + $xfer += $input->readString($this->tbl_name); + } else { + $xfer += $input->skip($ftype); + } + break; + case 3: + if ($ftype == TType::STRUCT) { + $this->new_tbl = new \metastore\Table(); + $xfer += $this->new_tbl->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; + case 4: + if ($ftype == TType::STRUCT) { + $this->environment_context = new \metastore\EnvironmentContext(); + $xfer += $this->environment_context->read($input); } else { $xfer += $input->skip($ftype); } @@ -7545,15 +11031,31 @@ class ThriftHiveMetastore_get_fields_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_fields_args'); - if ($this->db_name !== null) { - $xfer += $output->writeFieldBegin('db_name', TType::STRING, 1); - $xfer += $output->writeString($this->db_name); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_alter_table_with_environment_context_args'); + if ($this->dbname !== null) { + $xfer += $output->writeFieldBegin('dbname', TType::STRING, 1); + $xfer += $output->writeString($this->dbname); $xfer += $output->writeFieldEnd(); } - if ($this->table_name !== null) { - $xfer += $output->writeFieldBegin('table_name', TType::STRING, 2); - $xfer += $output->writeString($this->table_name); + if ($this->tbl_name !== null) { + $xfer += $output->writeFieldBegin('tbl_name', TType::STRING, 2); + $xfer += $output->writeString($this->tbl_name); + $xfer += $output->writeFieldEnd(); + } + if ($this->new_tbl !== null) { + if (!is_object($this->new_tbl)) { + throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); + } + $xfer += $output->writeFieldBegin('new_tbl', TType::STRUCT, 3); + $xfer += $this->new_tbl->write($output); + $xfer += $output->writeFieldEnd(); + } + if ($this->environment_context !== null) { + if (!is_object($this->environment_context)) { + throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); + } + $xfer += $output->writeFieldBegin('environment_context', TType::STRUCT, 4); + $xfer += $this->environment_context->write($output); $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -7563,61 +11065,39 @@ class ThriftHiveMetastore_get_fields_args { } -class ThriftHiveMetastore_get_fields_result { +class ThriftHiveMetastore_alter_table_with_environment_context_result { static $_TSPEC; - public $success = null; public $o1 = null; public $o2 = null; - public $o3 = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( - 0 => array( - 'var' => 'success', - 'type' => TType::LST, - 'etype' => TType::STRUCT, - 'elem' => array( - 'type' => TType::STRUCT, - 'class' => '\metastore\FieldSchema', - ), - ), 1 => array( 'var' => 'o1', 'type' => TType::STRUCT, - 'class' => '\metastore\MetaException', + 'class' => '\metastore\InvalidOperationException', ), 2 => array( 'var' => 'o2', 'type' => TType::STRUCT, - 'class' => '\metastore\UnknownTableException', - ), - 3 => array( - 'var' => 'o3', - 'type' => TType::STRUCT, - 'class' => '\metastore\UnknownDBException', + 'class' => '\metastore\MetaException', ), ); } if (is_array($vals)) { - if (isset($vals['success'])) { - $this->success = $vals['success']; - } if (isset($vals['o1'])) { $this->o1 = $vals['o1']; } if (isset($vals['o2'])) { $this->o2 = $vals['o2']; } - if (isset($vals['o3'])) { - $this->o3 = $vals['o3']; - } } } public function getName() { - return 'ThriftHiveMetastore_get_fields_result'; + return 'ThriftHiveMetastore_alter_table_with_environment_context_result'; } public function read($input) @@ -7635,27 +11115,9 @@ class ThriftHiveMetastore_get_fields_result { } switch ($fid) { - case 0: - if ($ftype == TType::LST) { - $this->success = array(); - $_size343 = 0; - $_etype346 = 0; - $xfer += $input->readListBegin($_etype346, $_size343); - for ($_i347 = 0; $_i347 < $_size343; ++$_i347) - { - $elem348 = null; - $elem348 = new \metastore\FieldSchema(); - $xfer += $elem348->read($input); - $this->success []= $elem348; - } - $xfer += $input->readListEnd(); - } else { - $xfer += $input->skip($ftype); - } - break; case 1: if ($ftype == TType::STRUCT) { - $this->o1 = new \metastore\MetaException(); + $this->o1 = new \metastore\InvalidOperationException(); $xfer += $this->o1->read($input); } else { $xfer += $input->skip($ftype); @@ -7663,20 +11125,12 @@ class ThriftHiveMetastore_get_fields_result { break; case 2: if ($ftype == TType::STRUCT) { - $this->o2 = new \metastore\UnknownTableException(); + $this->o2 = new \metastore\MetaException(); $xfer += $this->o2->read($input); } else { $xfer += $input->skip($ftype); } break; - case 3: - if ($ftype == TType::STRUCT) { - $this->o3 = new \metastore\UnknownDBException(); - $xfer += $this->o3->read($input); - } else { - $xfer += $input->skip($ftype); - } - break; default: $xfer += $input->skip($ftype); break; @@ -7689,24 +11143,7 @@ class ThriftHiveMetastore_get_fields_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_fields_result'); - if ($this->success !== null) { - if (!is_array($this->success)) { - throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); - } - $xfer += $output->writeFieldBegin('success', TType::LST, 0); - { - $output->writeListBegin(TType::STRUCT, count($this->success)); - { - foreach ($this->success as $iter349) - { - $xfer += $iter349->write($output); - } - } - $output->writeListEnd(); - } - $xfer += $output->writeFieldEnd(); - } + $xfer += $output->writeStructBegin('ThriftHiveMetastore_alter_table_with_environment_context_result'); if ($this->o1 !== null) { $xfer += $output->writeFieldBegin('o1', TType::STRUCT, 1); $xfer += $this->o1->write($output); @@ -7717,11 +11154,6 @@ class ThriftHiveMetastore_get_fields_result { $xfer += $this->o2->write($output); $xfer += $output->writeFieldEnd(); } - if ($this->o3 !== null) { - $xfer += $output->writeFieldBegin('o3', TType::STRUCT, 3); - $xfer += $this->o3->write($output); - $xfer += $output->writeFieldEnd(); - } $xfer += $output->writeFieldStop(); $xfer += $output->writeStructEnd(); return $xfer; @@ -7729,37 +11161,30 @@ class ThriftHiveMetastore_get_fields_result { } -class ThriftHiveMetastore_get_schema_args { +class ThriftHiveMetastore_add_partition_args { static $_TSPEC; - public $db_name = null; - public $table_name = null; + public $new_part = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 1 => array( - 'var' => 'db_name', - 'type' => TType::STRING, - ), - 2 => array( - 'var' => 'table_name', - 'type' => TType::STRING, + 'var' => 'new_part', + 'type' => TType::STRUCT, + 'class' => '\metastore\Partition', ), ); } if (is_array($vals)) { - if (isset($vals['db_name'])) { - $this->db_name = $vals['db_name']; - } - if (isset($vals['table_name'])) { - $this->table_name = $vals['table_name']; + if (isset($vals['new_part'])) { + $this->new_part = $vals['new_part']; } } } public function getName() { - return 'ThriftHiveMetastore_get_schema_args'; + return 'ThriftHiveMetastore_add_partition_args'; } public function read($input) @@ -7778,15 +11203,9 @@ class ThriftHiveMetastore_get_schema_args { switch ($fid) { case 1: - if ($ftype == TType::STRING) { - $xfer += $input->readString($this->db_name); - } else { - $xfer += $input->skip($ftype); - } - break; - case 2: - if ($ftype == TType::STRING) { - $xfer += $input->readString($this->table_name); + if ($ftype == TType::STRUCT) { + $this->new_part = new \metastore\Partition(); + $xfer += $this->new_part->read($input); } else { $xfer += $input->skip($ftype); } @@ -7803,15 +11222,13 @@ class ThriftHiveMetastore_get_schema_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_schema_args'); - if ($this->db_name !== null) { - $xfer += $output->writeFieldBegin('db_name', TType::STRING, 1); - $xfer += $output->writeString($this->db_name); - $xfer += $output->writeFieldEnd(); - } - if ($this->table_name !== null) { - $xfer += $output->writeFieldBegin('table_name', TType::STRING, 2); - $xfer += $output->writeString($this->table_name); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_add_partition_args'); + if ($this->new_part !== null) { + if (!is_object($this->new_part)) { + throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); + } + $xfer += $output->writeFieldBegin('new_part', TType::STRUCT, 1); + $xfer += $this->new_part->write($output); $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -7821,7 +11238,7 @@ class ThriftHiveMetastore_get_schema_args { } -class ThriftHiveMetastore_get_schema_result { +class ThriftHiveMetastore_add_partition_result { static $_TSPEC; public $success = null; @@ -7834,27 +11251,23 @@ class ThriftHiveMetastore_get_schema_result { self::$_TSPEC = array( 0 => array( 'var' => 'success', - 'type' => TType::LST, - 'etype' => TType::STRUCT, - 'elem' => array( - 'type' => TType::STRUCT, - 'class' => '\metastore\FieldSchema', - ), + 'type' => TType::STRUCT, + 'class' => '\metastore\Partition', ), 1 => array( 'var' => 'o1', 'type' => TType::STRUCT, - 'class' => '\metastore\MetaException', + 'class' => '\metastore\InvalidObjectException', ), 2 => array( 'var' => 'o2', 'type' => TType::STRUCT, - 'class' => '\metastore\UnknownTableException', + 'class' => '\metastore\AlreadyExistsException', ), 3 => array( 'var' => 'o3', 'type' => TType::STRUCT, - 'class' => '\metastore\UnknownDBException', + 'class' => '\metastore\MetaException', ), ); } @@ -7875,7 +11288,7 @@ class ThriftHiveMetastore_get_schema_result { } public function getName() { - return 'ThriftHiveMetastore_get_schema_result'; + return 'ThriftHiveMetastore_add_partition_result'; } public function read($input) @@ -7894,26 +11307,16 @@ class ThriftHiveMetastore_get_schema_result { switch ($fid) { case 0: - if ($ftype == TType::LST) { - $this->success = array(); - $_size350 = 0; - $_etype353 = 0; - $xfer += $input->readListBegin($_etype353, $_size350); - for ($_i354 = 0; $_i354 < $_size350; ++$_i354) - { - $elem355 = null; - $elem355 = new \metastore\FieldSchema(); - $xfer += $elem355->read($input); - $this->success []= $elem355; - } - $xfer += $input->readListEnd(); + if ($ftype == TType::STRUCT) { + $this->success = new \metastore\Partition(); + $xfer += $this->success->read($input); } else { $xfer += $input->skip($ftype); } break; case 1: if ($ftype == TType::STRUCT) { - $this->o1 = new \metastore\MetaException(); + $this->o1 = new \metastore\InvalidObjectException(); $xfer += $this->o1->read($input); } else { $xfer += $input->skip($ftype); @@ -7921,7 +11324,7 @@ class ThriftHiveMetastore_get_schema_result { break; case 2: if ($ftype == TType::STRUCT) { - $this->o2 = new \metastore\UnknownTableException(); + $this->o2 = new \metastore\AlreadyExistsException(); $xfer += $this->o2->read($input); } else { $xfer += $input->skip($ftype); @@ -7929,7 +11332,7 @@ class ThriftHiveMetastore_get_schema_result { break; case 3: if ($ftype == TType::STRUCT) { - $this->o3 = new \metastore\UnknownDBException(); + $this->o3 = new \metastore\MetaException(); $xfer += $this->o3->read($input); } else { $xfer += $input->skip($ftype); @@ -7947,22 +11350,13 @@ class ThriftHiveMetastore_get_schema_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_schema_result'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_add_partition_result'); if ($this->success !== null) { - if (!is_array($this->success)) { - throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); - } - $xfer += $output->writeFieldBegin('success', TType::LST, 0); - { - $output->writeListBegin(TType::STRUCT, count($this->success)); - { - foreach ($this->success as $iter356) - { - $xfer += $iter356->write($output); - } - } - $output->writeListEnd(); + if (!is_object($this->success)) { + throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); } + $xfer += $output->writeFieldBegin('success', TType::STRUCT, 0); + $xfer += $this->success->write($output); $xfer += $output->writeFieldEnd(); } if ($this->o1 !== null) { @@ -7987,30 +11381,39 @@ class ThriftHiveMetastore_get_schema_result { } -class ThriftHiveMetastore_create_table_args { +class ThriftHiveMetastore_add_partition_with_environment_context_args { static $_TSPEC; - public $tbl = null; + public $new_part = null; + public $environment_context = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 1 => array( - 'var' => 'tbl', + 'var' => 'new_part', 'type' => TType::STRUCT, - 'class' => '\metastore\Table', + 'class' => '\metastore\Partition', + ), + 2 => array( + 'var' => 'environment_context', + 'type' => TType::STRUCT, + 'class' => '\metastore\EnvironmentContext', ), ); } if (is_array($vals)) { - if (isset($vals['tbl'])) { - $this->tbl = $vals['tbl']; + if (isset($vals['new_part'])) { + $this->new_part = $vals['new_part']; + } + if (isset($vals['environment_context'])) { + $this->environment_context = $vals['environment_context']; } } } public function getName() { - return 'ThriftHiveMetastore_create_table_args'; + return 'ThriftHiveMetastore_add_partition_with_environment_context_args'; } public function read($input) @@ -8030,8 +11433,16 @@ class ThriftHiveMetastore_create_table_args { { case 1: if ($ftype == TType::STRUCT) { - $this->tbl = new \metastore\Table(); - $xfer += $this->tbl->read($input); + $this->new_part = new \metastore\Partition(); + $xfer += $this->new_part->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; + case 2: + if ($ftype == TType::STRUCT) { + $this->environment_context = new \metastore\EnvironmentContext(); + $xfer += $this->environment_context->read($input); } else { $xfer += $input->skip($ftype); } @@ -8048,13 +11459,21 @@ class ThriftHiveMetastore_create_table_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_create_table_args'); - if ($this->tbl !== null) { - if (!is_object($this->tbl)) { + $xfer += $output->writeStructBegin('ThriftHiveMetastore_add_partition_with_environment_context_args'); + if ($this->new_part !== null) { + if (!is_object($this->new_part)) { throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); } - $xfer += $output->writeFieldBegin('tbl', TType::STRUCT, 1); - $xfer += $this->tbl->write($output); + $xfer += $output->writeFieldBegin('new_part', TType::STRUCT, 1); + $xfer += $this->new_part->write($output); + $xfer += $output->writeFieldEnd(); + } + if ($this->environment_context !== null) { + if (!is_object($this->environment_context)) { + throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); + } + $xfer += $output->writeFieldBegin('environment_context', TType::STRUCT, 2); + $xfer += $this->environment_context->write($output); $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -8064,40 +11483,43 @@ class ThriftHiveMetastore_create_table_args { } -class ThriftHiveMetastore_create_table_result { +class ThriftHiveMetastore_add_partition_with_environment_context_result { static $_TSPEC; + public $success = null; public $o1 = null; public $o2 = null; public $o3 = null; - public $o4 = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( + 0 => array( + 'var' => 'success', + 'type' => TType::STRUCT, + 'class' => '\metastore\Partition', + ), 1 => array( 'var' => 'o1', 'type' => TType::STRUCT, - 'class' => '\metastore\AlreadyExistsException', + 'class' => '\metastore\InvalidObjectException', ), 2 => array( 'var' => 'o2', 'type' => TType::STRUCT, - 'class' => '\metastore\InvalidObjectException', + 'class' => '\metastore\AlreadyExistsException', ), 3 => array( 'var' => 'o3', 'type' => TType::STRUCT, 'class' => '\metastore\MetaException', ), - 4 => array( - 'var' => 'o4', - 'type' => TType::STRUCT, - 'class' => '\metastore\NoSuchObjectException', - ), ); } if (is_array($vals)) { + if (isset($vals['success'])) { + $this->success = $vals['success']; + } if (isset($vals['o1'])) { $this->o1 = $vals['o1']; } @@ -8107,14 +11529,11 @@ class ThriftHiveMetastore_create_table_result { if (isset($vals['o3'])) { $this->o3 = $vals['o3']; } - if (isset($vals['o4'])) { - $this->o4 = $vals['o4']; - } } } public function getName() { - return 'ThriftHiveMetastore_create_table_result'; + return 'ThriftHiveMetastore_add_partition_with_environment_context_result'; } public function read($input) @@ -8132,9 +11551,17 @@ class ThriftHiveMetastore_create_table_result { } switch ($fid) { + case 0: + if ($ftype == TType::STRUCT) { + $this->success = new \metastore\Partition(); + $xfer += $this->success->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; case 1: if ($ftype == TType::STRUCT) { - $this->o1 = new \metastore\AlreadyExistsException(); + $this->o1 = new \metastore\InvalidObjectException(); $xfer += $this->o1->read($input); } else { $xfer += $input->skip($ftype); @@ -8142,7 +11569,7 @@ class ThriftHiveMetastore_create_table_result { break; case 2: if ($ftype == TType::STRUCT) { - $this->o2 = new \metastore\InvalidObjectException(); + $this->o2 = new \metastore\AlreadyExistsException(); $xfer += $this->o2->read($input); } else { $xfer += $input->skip($ftype); @@ -8156,14 +11583,6 @@ class ThriftHiveMetastore_create_table_result { $xfer += $input->skip($ftype); } break; - case 4: - if ($ftype == TType::STRUCT) { - $this->o4 = new \metastore\NoSuchObjectException(); - $xfer += $this->o4->read($input); - } else { - $xfer += $input->skip($ftype); - } - break; default: $xfer += $input->skip($ftype); break; @@ -8176,7 +11595,15 @@ class ThriftHiveMetastore_create_table_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_create_table_result'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_add_partition_with_environment_context_result'); + if ($this->success !== null) { + if (!is_object($this->success)) { + throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); + } + $xfer += $output->writeFieldBegin('success', TType::STRUCT, 0); + $xfer += $this->success->write($output); + $xfer += $output->writeFieldEnd(); + } if ($this->o1 !== null) { $xfer += $output->writeFieldBegin('o1', TType::STRUCT, 1); $xfer += $this->o1->write($output); @@ -8192,11 +11619,6 @@ class ThriftHiveMetastore_create_table_result { $xfer += $this->o3->write($output); $xfer += $output->writeFieldEnd(); } - if ($this->o4 !== null) { - $xfer += $output->writeFieldBegin('o4', TType::STRUCT, 4); - $xfer += $this->o4->write($output); - $xfer += $output->writeFieldEnd(); - } $xfer += $output->writeFieldStop(); $xfer += $output->writeStructEnd(); return $xfer; @@ -8204,39 +11626,34 @@ class ThriftHiveMetastore_create_table_result { } -class ThriftHiveMetastore_create_table_with_environment_context_args { +class ThriftHiveMetastore_add_partitions_args { static $_TSPEC; - public $tbl = null; - public $environment_context = null; + public $new_parts = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 1 => array( - 'var' => 'tbl', - 'type' => TType::STRUCT, - 'class' => '\metastore\Table', - ), - 2 => array( - 'var' => 'environment_context', - 'type' => TType::STRUCT, - 'class' => '\metastore\EnvironmentContext', + 'var' => 'new_parts', + 'type' => TType::LST, + 'etype' => TType::STRUCT, + 'elem' => array( + 'type' => TType::STRUCT, + 'class' => '\metastore\Partition', + ), ), ); } if (is_array($vals)) { - if (isset($vals['tbl'])) { - $this->tbl = $vals['tbl']; - } - if (isset($vals['environment_context'])) { - $this->environment_context = $vals['environment_context']; + if (isset($vals['new_parts'])) { + $this->new_parts = $vals['new_parts']; } } } public function getName() { - return 'ThriftHiveMetastore_create_table_with_environment_context_args'; + return 'ThriftHiveMetastore_add_partitions_args'; } public function read($input) @@ -8255,17 +11672,19 @@ class ThriftHiveMetastore_create_table_with_environment_context_args { switch ($fid) { case 1: - if ($ftype == TType::STRUCT) { - $this->tbl = new \metastore\Table(); - $xfer += $this->tbl->read($input); - } else { - $xfer += $input->skip($ftype); - } - break; - case 2: - if ($ftype == TType::STRUCT) { - $this->environment_context = new \metastore\EnvironmentContext(); - $xfer += $this->environment_context->read($input); + if ($ftype == TType::LST) { + $this->new_parts = array(); + $_size435 = 0; + $_etype438 = 0; + $xfer += $input->readListBegin($_etype438, $_size435); + for ($_i439 = 0; $_i439 < $_size435; ++$_i439) + { + $elem440 = null; + $elem440 = new \metastore\Partition(); + $xfer += $elem440->read($input); + $this->new_parts []= $elem440; + } + $xfer += $input->readListEnd(); } else { $xfer += $input->skip($ftype); } @@ -8282,21 +11701,22 @@ class ThriftHiveMetastore_create_table_with_environment_context_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_create_table_with_environment_context_args'); - if ($this->tbl !== null) { - if (!is_object($this->tbl)) { + $xfer += $output->writeStructBegin('ThriftHiveMetastore_add_partitions_args'); + if ($this->new_parts !== null) { + if (!is_array($this->new_parts)) { throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); } - $xfer += $output->writeFieldBegin('tbl', TType::STRUCT, 1); - $xfer += $this->tbl->write($output); - $xfer += $output->writeFieldEnd(); - } - if ($this->environment_context !== null) { - if (!is_object($this->environment_context)) { - throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); + $xfer += $output->writeFieldBegin('new_parts', TType::LST, 1); + { + $output->writeListBegin(TType::STRUCT, count($this->new_parts)); + { + foreach ($this->new_parts as $iter441) + { + $xfer += $iter441->write($output); + } + } + $output->writeListEnd(); } - $xfer += $output->writeFieldBegin('environment_context', TType::STRUCT, 2); - $xfer += $this->environment_context->write($output); $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -8306,40 +11726,42 @@ class ThriftHiveMetastore_create_table_with_environment_context_args { } -class ThriftHiveMetastore_create_table_with_environment_context_result { +class ThriftHiveMetastore_add_partitions_result { static $_TSPEC; + public $success = null; public $o1 = null; public $o2 = null; public $o3 = null; - public $o4 = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( + 0 => array( + 'var' => 'success', + 'type' => TType::I32, + ), 1 => array( 'var' => 'o1', 'type' => TType::STRUCT, - 'class' => '\metastore\AlreadyExistsException', + 'class' => '\metastore\InvalidObjectException', ), 2 => array( 'var' => 'o2', 'type' => TType::STRUCT, - 'class' => '\metastore\InvalidObjectException', + 'class' => '\metastore\AlreadyExistsException', ), 3 => array( 'var' => 'o3', 'type' => TType::STRUCT, 'class' => '\metastore\MetaException', ), - 4 => array( - 'var' => 'o4', - 'type' => TType::STRUCT, - 'class' => '\metastore\NoSuchObjectException', - ), ); } if (is_array($vals)) { + if (isset($vals['success'])) { + $this->success = $vals['success']; + } if (isset($vals['o1'])) { $this->o1 = $vals['o1']; } @@ -8349,14 +11771,11 @@ class ThriftHiveMetastore_create_table_with_environment_context_result { if (isset($vals['o3'])) { $this->o3 = $vals['o3']; } - if (isset($vals['o4'])) { - $this->o4 = $vals['o4']; - } } } public function getName() { - return 'ThriftHiveMetastore_create_table_with_environment_context_result'; + return 'ThriftHiveMetastore_add_partitions_result'; } public function read($input) @@ -8374,9 +11793,16 @@ class ThriftHiveMetastore_create_table_with_environment_context_result { } switch ($fid) { + case 0: + if ($ftype == TType::I32) { + $xfer += $input->readI32($this->success); + } else { + $xfer += $input->skip($ftype); + } + break; case 1: if ($ftype == TType::STRUCT) { - $this->o1 = new \metastore\AlreadyExistsException(); + $this->o1 = new \metastore\InvalidObjectException(); $xfer += $this->o1->read($input); } else { $xfer += $input->skip($ftype); @@ -8384,7 +11810,7 @@ class ThriftHiveMetastore_create_table_with_environment_context_result { break; case 2: if ($ftype == TType::STRUCT) { - $this->o2 = new \metastore\InvalidObjectException(); + $this->o2 = new \metastore\AlreadyExistsException(); $xfer += $this->o2->read($input); } else { $xfer += $input->skip($ftype); @@ -8398,14 +11824,6 @@ class ThriftHiveMetastore_create_table_with_environment_context_result { $xfer += $input->skip($ftype); } break; - case 4: - if ($ftype == TType::STRUCT) { - $this->o4 = new \metastore\NoSuchObjectException(); - $xfer += $this->o4->read($input); - } else { - $xfer += $input->skip($ftype); - } - break; default: $xfer += $input->skip($ftype); break; @@ -8418,7 +11836,12 @@ class ThriftHiveMetastore_create_table_with_environment_context_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_create_table_with_environment_context_result'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_add_partitions_result'); + if ($this->success !== null) { + $xfer += $output->writeFieldBegin('success', TType::I32, 0); + $xfer += $output->writeI32($this->success); + $xfer += $output->writeFieldEnd(); + } if ($this->o1 !== null) { $xfer += $output->writeFieldBegin('o1', TType::STRUCT, 1); $xfer += $this->o1->write($output); @@ -8434,11 +11857,6 @@ class ThriftHiveMetastore_create_table_with_environment_context_result { $xfer += $this->o3->write($output); $xfer += $output->writeFieldEnd(); } - if ($this->o4 !== null) { - $xfer += $output->writeFieldBegin('o4', TType::STRUCT, 4); - $xfer += $this->o4->write($output); - $xfer += $output->writeFieldEnd(); - } $xfer += $output->writeFieldStop(); $xfer += $output->writeStructEnd(); return $xfer; @@ -8446,45 +11864,49 @@ class ThriftHiveMetastore_create_table_with_environment_context_result { } -class ThriftHiveMetastore_drop_table_args { +class ThriftHiveMetastore_append_partition_args { static $_TSPEC; - public $dbname = null; - public $name = null; - public $deleteData = null; + public $db_name = null; + public $tbl_name = null; + public $part_vals = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 1 => array( - 'var' => 'dbname', + 'var' => 'db_name', 'type' => TType::STRING, ), 2 => array( - 'var' => 'name', + 'var' => 'tbl_name', 'type' => TType::STRING, ), - 3 => array( - 'var' => 'deleteData', - 'type' => TType::BOOL, + 3 => array( + 'var' => 'part_vals', + 'type' => TType::LST, + 'etype' => TType::STRING, + 'elem' => array( + 'type' => TType::STRING, + ), ), ); } if (is_array($vals)) { - if (isset($vals['dbname'])) { - $this->dbname = $vals['dbname']; + if (isset($vals['db_name'])) { + $this->db_name = $vals['db_name']; } - if (isset($vals['name'])) { - $this->name = $vals['name']; + if (isset($vals['tbl_name'])) { + $this->tbl_name = $vals['tbl_name']; } - if (isset($vals['deleteData'])) { - $this->deleteData = $vals['deleteData']; + if (isset($vals['part_vals'])) { + $this->part_vals = $vals['part_vals']; } } } public function getName() { - return 'ThriftHiveMetastore_drop_table_args'; + return 'ThriftHiveMetastore_append_partition_args'; } public function read($input) @@ -8504,21 +11926,31 @@ class ThriftHiveMetastore_drop_table_args { { case 1: if ($ftype == TType::STRING) { - $xfer += $input->readString($this->dbname); + $xfer += $input->readString($this->db_name); } else { $xfer += $input->skip($ftype); } break; case 2: if ($ftype == TType::STRING) { - $xfer += $input->readString($this->name); + $xfer += $input->readString($this->tbl_name); } else { $xfer += $input->skip($ftype); } break; case 3: - if ($ftype == TType::BOOL) { - $xfer += $input->readBool($this->deleteData); + if ($ftype == TType::LST) { + $this->part_vals = array(); + $_size442 = 0; + $_etype445 = 0; + $xfer += $input->readListBegin($_etype445, $_size442); + for ($_i446 = 0; $_i446 < $_size442; ++$_i446) + { + $elem447 = null; + $xfer += $input->readString($elem447); + $this->part_vals []= $elem447; + } + $xfer += $input->readListEnd(); } else { $xfer += $input->skip($ftype); } @@ -8535,20 +11967,32 @@ class ThriftHiveMetastore_drop_table_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_drop_table_args'); - if ($this->dbname !== null) { - $xfer += $output->writeFieldBegin('dbname', TType::STRING, 1); - $xfer += $output->writeString($this->dbname); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_append_partition_args'); + if ($this->db_name !== null) { + $xfer += $output->writeFieldBegin('db_name', TType::STRING, 1); + $xfer += $output->writeString($this->db_name); $xfer += $output->writeFieldEnd(); } - if ($this->name !== null) { - $xfer += $output->writeFieldBegin('name', TType::STRING, 2); - $xfer += $output->writeString($this->name); + if ($this->tbl_name !== null) { + $xfer += $output->writeFieldBegin('tbl_name', TType::STRING, 2); + $xfer += $output->writeString($this->tbl_name); $xfer += $output->writeFieldEnd(); } - if ($this->deleteData !== null) { - $xfer += $output->writeFieldBegin('deleteData', TType::BOOL, 3); - $xfer += $output->writeBool($this->deleteData); + if ($this->part_vals !== null) { + if (!is_array($this->part_vals)) { + throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); + } + $xfer += $output->writeFieldBegin('part_vals', TType::LST, 3); + { + $output->writeListBegin(TType::STRING, count($this->part_vals)); + { + foreach ($this->part_vals as $iter448) + { + $xfer += $output->writeString($iter448); + } + } + $output->writeListEnd(); + } $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -8558,21 +12002,33 @@ class ThriftHiveMetastore_drop_table_args { } -class ThriftHiveMetastore_drop_table_result { +class ThriftHiveMetastore_append_partition_result { static $_TSPEC; + public $success = null; public $o1 = null; + public $o2 = null; public $o3 = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( + 0 => array( + 'var' => 'success', + 'type' => TType::STRUCT, + 'class' => '\metastore\Partition', + ), 1 => array( 'var' => 'o1', 'type' => TType::STRUCT, - 'class' => '\metastore\NoSuchObjectException', + 'class' => '\metastore\InvalidObjectException', ), 2 => array( + 'var' => 'o2', + 'type' => TType::STRUCT, + 'class' => '\metastore\AlreadyExistsException', + ), + 3 => array( 'var' => 'o3', 'type' => TType::STRUCT, 'class' => '\metastore\MetaException', @@ -8580,9 +12036,15 @@ class ThriftHiveMetastore_drop_table_result { ); } if (is_array($vals)) { + if (isset($vals['success'])) { + $this->success = $vals['success']; + } if (isset($vals['o1'])) { $this->o1 = $vals['o1']; } + if (isset($vals['o2'])) { + $this->o2 = $vals['o2']; + } if (isset($vals['o3'])) { $this->o3 = $vals['o3']; } @@ -8590,7 +12052,7 @@ class ThriftHiveMetastore_drop_table_result { } public function getName() { - return 'ThriftHiveMetastore_drop_table_result'; + return 'ThriftHiveMetastore_append_partition_result'; } public function read($input) @@ -8608,9 +12070,17 @@ class ThriftHiveMetastore_drop_table_result { } switch ($fid) { + case 0: + if ($ftype == TType::STRUCT) { + $this->success = new \metastore\Partition(); + $xfer += $this->success->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; case 1: if ($ftype == TType::STRUCT) { - $this->o1 = new \metastore\NoSuchObjectException(); + $this->o1 = new \metastore\InvalidObjectException(); $xfer += $this->o1->read($input); } else { $xfer += $input->skip($ftype); @@ -8618,6 +12088,14 @@ class ThriftHiveMetastore_drop_table_result { break; case 2: if ($ftype == TType::STRUCT) { + $this->o2 = new \metastore\AlreadyExistsException(); + $xfer += $this->o2->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; + case 3: + if ($ftype == TType::STRUCT) { $this->o3 = new \metastore\MetaException(); $xfer += $this->o3->read($input); } else { @@ -8636,14 +12114,27 @@ class ThriftHiveMetastore_drop_table_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_drop_table_result'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_append_partition_result'); + if ($this->success !== null) { + if (!is_object($this->success)) { + throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); + } + $xfer += $output->writeFieldBegin('success', TType::STRUCT, 0); + $xfer += $this->success->write($output); + $xfer += $output->writeFieldEnd(); + } if ($this->o1 !== null) { $xfer += $output->writeFieldBegin('o1', TType::STRUCT, 1); $xfer += $this->o1->write($output); $xfer += $output->writeFieldEnd(); } + if ($this->o2 !== null) { + $xfer += $output->writeFieldBegin('o2', TType::STRUCT, 2); + $xfer += $this->o2->write($output); + $xfer += $output->writeFieldEnd(); + } if ($this->o3 !== null) { - $xfer += $output->writeFieldBegin('o3', TType::STRUCT, 2); + $xfer += $output->writeFieldBegin('o3', TType::STRUCT, 3); $xfer += $this->o3->write($output); $xfer += $output->writeFieldEnd(); } @@ -8654,54 +12145,30 @@ class ThriftHiveMetastore_drop_table_result { } -class ThriftHiveMetastore_drop_table_with_environment_context_args { +class ThriftHiveMetastore_add_partitions_req_args { static $_TSPEC; - public $dbname = null; - public $name = null; - public $deleteData = null; - public $environment_context = null; + public $request = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 1 => array( - 'var' => 'dbname', - 'type' => TType::STRING, - ), - 2 => array( - 'var' => 'name', - 'type' => TType::STRING, - ), - 3 => array( - 'var' => 'deleteData', - 'type' => TType::BOOL, - ), - 4 => array( - 'var' => 'environment_context', + 'var' => 'request', 'type' => TType::STRUCT, - 'class' => '\metastore\EnvironmentContext', + 'class' => '\metastore\AddPartitionsRequest', ), ); } if (is_array($vals)) { - if (isset($vals['dbname'])) { - $this->dbname = $vals['dbname']; - } - if (isset($vals['name'])) { - $this->name = $vals['name']; - } - if (isset($vals['deleteData'])) { - $this->deleteData = $vals['deleteData']; - } - if (isset($vals['environment_context'])) { - $this->environment_context = $vals['environment_context']; + if (isset($vals['request'])) { + $this->request = $vals['request']; } } } public function getName() { - return 'ThriftHiveMetastore_drop_table_with_environment_context_args'; + return 'ThriftHiveMetastore_add_partitions_req_args'; } public function read($input) @@ -8720,30 +12187,9 @@ class ThriftHiveMetastore_drop_table_with_environment_context_args { switch ($fid) { case 1: - if ($ftype == TType::STRING) { - $xfer += $input->readString($this->dbname); - } else { - $xfer += $input->skip($ftype); - } - break; - case 2: - if ($ftype == TType::STRING) { - $xfer += $input->readString($this->name); - } else { - $xfer += $input->skip($ftype); - } - break; - case 3: - if ($ftype == TType::BOOL) { - $xfer += $input->readBool($this->deleteData); - } else { - $xfer += $input->skip($ftype); - } - break; - case 4: if ($ftype == TType::STRUCT) { - $this->environment_context = new \metastore\EnvironmentContext(); - $xfer += $this->environment_context->read($input); + $this->request = new \metastore\AddPartitionsRequest(); + $xfer += $this->request->read($input); } else { $xfer += $input->skip($ftype); } @@ -8760,28 +12206,13 @@ class ThriftHiveMetastore_drop_table_with_environment_context_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_drop_table_with_environment_context_args'); - if ($this->dbname !== null) { - $xfer += $output->writeFieldBegin('dbname', TType::STRING, 1); - $xfer += $output->writeString($this->dbname); - $xfer += $output->writeFieldEnd(); - } - if ($this->name !== null) { - $xfer += $output->writeFieldBegin('name', TType::STRING, 2); - $xfer += $output->writeString($this->name); - $xfer += $output->writeFieldEnd(); - } - if ($this->deleteData !== null) { - $xfer += $output->writeFieldBegin('deleteData', TType::BOOL, 3); - $xfer += $output->writeBool($this->deleteData); - $xfer += $output->writeFieldEnd(); - } - if ($this->environment_context !== null) { - if (!is_object($this->environment_context)) { + $xfer += $output->writeStructBegin('ThriftHiveMetastore_add_partitions_req_args'); + if ($this->request !== null) { + if (!is_object($this->request)) { throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); } - $xfer += $output->writeFieldBegin('environment_context', TType::STRUCT, 4); - $xfer += $this->environment_context->write($output); + $xfer += $output->writeFieldBegin('request', TType::STRUCT, 1); + $xfer += $this->request->write($output); $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -8791,21 +12222,33 @@ class ThriftHiveMetastore_drop_table_with_environment_context_args { } -class ThriftHiveMetastore_drop_table_with_environment_context_result { +class ThriftHiveMetastore_add_partitions_req_result { static $_TSPEC; + public $success = null; public $o1 = null; + public $o2 = null; public $o3 = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( + 0 => array( + 'var' => 'success', + 'type' => TType::STRUCT, + 'class' => '\metastore\AddPartitionsResult', + ), 1 => array( 'var' => 'o1', 'type' => TType::STRUCT, - 'class' => '\metastore\NoSuchObjectException', + 'class' => '\metastore\InvalidObjectException', ), 2 => array( + 'var' => 'o2', + 'type' => TType::STRUCT, + 'class' => '\metastore\AlreadyExistsException', + ), + 3 => array( 'var' => 'o3', 'type' => TType::STRUCT, 'class' => '\metastore\MetaException', @@ -8813,9 +12256,15 @@ class ThriftHiveMetastore_drop_table_with_environment_context_result { ); } if (is_array($vals)) { + if (isset($vals['success'])) { + $this->success = $vals['success']; + } if (isset($vals['o1'])) { $this->o1 = $vals['o1']; } + if (isset($vals['o2'])) { + $this->o2 = $vals['o2']; + } if (isset($vals['o3'])) { $this->o3 = $vals['o3']; } @@ -8823,7 +12272,7 @@ class ThriftHiveMetastore_drop_table_with_environment_context_result { } public function getName() { - return 'ThriftHiveMetastore_drop_table_with_environment_context_result'; + return 'ThriftHiveMetastore_add_partitions_req_result'; } public function read($input) @@ -8841,9 +12290,17 @@ class ThriftHiveMetastore_drop_table_with_environment_context_result { } switch ($fid) { + case 0: + if ($ftype == TType::STRUCT) { + $this->success = new \metastore\AddPartitionsResult(); + $xfer += $this->success->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; case 1: if ($ftype == TType::STRUCT) { - $this->o1 = new \metastore\NoSuchObjectException(); + $this->o1 = new \metastore\InvalidObjectException(); $xfer += $this->o1->read($input); } else { $xfer += $input->skip($ftype); @@ -8851,6 +12308,14 @@ class ThriftHiveMetastore_drop_table_with_environment_context_result { break; case 2: if ($ftype == TType::STRUCT) { + $this->o2 = new \metastore\AlreadyExistsException(); + $xfer += $this->o2->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; + case 3: + if ($ftype == TType::STRUCT) { $this->o3 = new \metastore\MetaException(); $xfer += $this->o3->read($input); } else { @@ -8869,14 +12334,27 @@ class ThriftHiveMetastore_drop_table_with_environment_context_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_drop_table_with_environment_context_result'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_add_partitions_req_result'); + if ($this->success !== null) { + if (!is_object($this->success)) { + throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); + } + $xfer += $output->writeFieldBegin('success', TType::STRUCT, 0); + $xfer += $this->success->write($output); + $xfer += $output->writeFieldEnd(); + } if ($this->o1 !== null) { $xfer += $output->writeFieldBegin('o1', TType::STRUCT, 1); $xfer += $this->o1->write($output); $xfer += $output->writeFieldEnd(); } + if ($this->o2 !== null) { + $xfer += $output->writeFieldBegin('o2', TType::STRUCT, 2); + $xfer += $this->o2->write($output); + $xfer += $output->writeFieldEnd(); + } if ($this->o3 !== null) { - $xfer += $output->writeFieldBegin('o3', TType::STRUCT, 2); + $xfer += $output->writeFieldBegin('o3', TType::STRUCT, 3); $xfer += $this->o3->write($output); $xfer += $output->writeFieldEnd(); } @@ -8887,11 +12365,13 @@ class ThriftHiveMetastore_drop_table_with_environment_context_result { } -class ThriftHiveMetastore_get_tables_args { +class ThriftHiveMetastore_append_partition_with_environment_context_args { static $_TSPEC; public $db_name = null; - public $pattern = null; + public $tbl_name = null; + public $part_vals = null; + public $environment_context = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { @@ -8901,23 +12381,42 @@ class ThriftHiveMetastore_get_tables_args { 'type' => TType::STRING, ), 2 => array( - 'var' => 'pattern', + 'var' => 'tbl_name', 'type' => TType::STRING, ), + 3 => array( + 'var' => 'part_vals', + 'type' => TType::LST, + 'etype' => TType::STRING, + 'elem' => array( + 'type' => TType::STRING, + ), + ), + 4 => array( + 'var' => 'environment_context', + 'type' => TType::STRUCT, + 'class' => '\metastore\EnvironmentContext', + ), ); } if (is_array($vals)) { if (isset($vals['db_name'])) { $this->db_name = $vals['db_name']; } - if (isset($vals['pattern'])) { - $this->pattern = $vals['pattern']; + if (isset($vals['tbl_name'])) { + $this->tbl_name = $vals['tbl_name']; + } + if (isset($vals['part_vals'])) { + $this->part_vals = $vals['part_vals']; + } + if (isset($vals['environment_context'])) { + $this->environment_context = $vals['environment_context']; } } } public function getName() { - return 'ThriftHiveMetastore_get_tables_args'; + return 'ThriftHiveMetastore_append_partition_with_environment_context_args'; } public function read($input) @@ -8944,7 +12443,32 @@ class ThriftHiveMetastore_get_tables_args { break; case 2: if ($ftype == TType::STRING) { - $xfer += $input->readString($this->pattern); + $xfer += $input->readString($this->tbl_name); + } else { + $xfer += $input->skip($ftype); + } + break; + case 3: + if ($ftype == TType::LST) { + $this->part_vals = array(); + $_size449 = 0; + $_etype452 = 0; + $xfer += $input->readListBegin($_etype452, $_size449); + for ($_i453 = 0; $_i453 < $_size449; ++$_i453) + { + $elem454 = null; + $xfer += $input->readString($elem454); + $this->part_vals []= $elem454; + } + $xfer += $input->readListEnd(); + } else { + $xfer += $input->skip($ftype); + } + break; + case 4: + if ($ftype == TType::STRUCT) { + $this->environment_context = new \metastore\EnvironmentContext(); + $xfer += $this->environment_context->read($input); } else { $xfer += $input->skip($ftype); } @@ -8961,15 +12485,40 @@ class ThriftHiveMetastore_get_tables_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_tables_args'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_append_partition_with_environment_context_args'); if ($this->db_name !== null) { $xfer += $output->writeFieldBegin('db_name', TType::STRING, 1); $xfer += $output->writeString($this->db_name); $xfer += $output->writeFieldEnd(); } - if ($this->pattern !== null) { - $xfer += $output->writeFieldBegin('pattern', TType::STRING, 2); - $xfer += $output->writeString($this->pattern); + if ($this->tbl_name !== null) { + $xfer += $output->writeFieldBegin('tbl_name', TType::STRING, 2); + $xfer += $output->writeString($this->tbl_name); + $xfer += $output->writeFieldEnd(); + } + if ($this->part_vals !== null) { + if (!is_array($this->part_vals)) { + throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); + } + $xfer += $output->writeFieldBegin('part_vals', TType::LST, 3); + { + $output->writeListBegin(TType::STRING, count($this->part_vals)); + { + foreach ($this->part_vals as $iter455) + { + $xfer += $output->writeString($iter455); + } + } + $output->writeListEnd(); + } + $xfer += $output->writeFieldEnd(); + } + if ($this->environment_context !== null) { + if (!is_object($this->environment_context)) { + throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); + } + $xfer += $output->writeFieldBegin('environment_context', TType::STRUCT, 4); + $xfer += $this->environment_context->write($output); $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -8979,26 +12528,35 @@ class ThriftHiveMetastore_get_tables_args { } -class ThriftHiveMetastore_get_tables_result { +class ThriftHiveMetastore_append_partition_with_environment_context_result { static $_TSPEC; public $success = null; public $o1 = null; + public $o2 = null; + public $o3 = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 0 => array( 'var' => 'success', - 'type' => TType::LST, - 'etype' => TType::STRING, - 'elem' => array( - 'type' => TType::STRING, - ), + 'type' => TType::STRUCT, + 'class' => '\metastore\Partition', ), 1 => array( 'var' => 'o1', 'type' => TType::STRUCT, + 'class' => '\metastore\InvalidObjectException', + ), + 2 => array( + 'var' => 'o2', + 'type' => TType::STRUCT, + 'class' => '\metastore\AlreadyExistsException', + ), + 3 => array( + 'var' => 'o3', + 'type' => TType::STRUCT, 'class' => '\metastore\MetaException', ), ); @@ -9010,11 +12568,17 @@ class ThriftHiveMetastore_get_tables_result { if (isset($vals['o1'])) { $this->o1 = $vals['o1']; } + if (isset($vals['o2'])) { + $this->o2 = $vals['o2']; + } + if (isset($vals['o3'])) { + $this->o3 = $vals['o3']; + } } } public function getName() { - return 'ThriftHiveMetastore_get_tables_result'; + return 'ThriftHiveMetastore_append_partition_with_environment_context_result'; } public function read($input) @@ -9033,30 +12597,37 @@ class ThriftHiveMetastore_get_tables_result { switch ($fid) { case 0: - if ($ftype == TType::LST) { - $this->success = array(); - $_size357 = 0; - $_etype360 = 0; - $xfer += $input->readListBegin($_etype360, $_size357); - for ($_i361 = 0; $_i361 < $_size357; ++$_i361) - { - $elem362 = null; - $xfer += $input->readString($elem362); - $this->success []= $elem362; - } - $xfer += $input->readListEnd(); + if ($ftype == TType::STRUCT) { + $this->success = new \metastore\Partition(); + $xfer += $this->success->read($input); } else { $xfer += $input->skip($ftype); } break; case 1: if ($ftype == TType::STRUCT) { - $this->o1 = new \metastore\MetaException(); + $this->o1 = new \metastore\InvalidObjectException(); $xfer += $this->o1->read($input); } else { $xfer += $input->skip($ftype); } break; + case 2: + if ($ftype == TType::STRUCT) { + $this->o2 = new \metastore\AlreadyExistsException(); + $xfer += $this->o2->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; + case 3: + if ($ftype == TType::STRUCT) { + $this->o3 = new \metastore\MetaException(); + $xfer += $this->o3->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; default: $xfer += $input->skip($ftype); break; @@ -9069,22 +12640,13 @@ class ThriftHiveMetastore_get_tables_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_tables_result'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_append_partition_with_environment_context_result'); if ($this->success !== null) { - if (!is_array($this->success)) { + if (!is_object($this->success)) { throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); } - $xfer += $output->writeFieldBegin('success', TType::LST, 0); - { - $output->writeListBegin(TType::STRING, count($this->success)); - { - foreach ($this->success as $iter363) - { - $xfer += $output->writeString($iter363); - } - } - $output->writeListEnd(); - } + $xfer += $output->writeFieldBegin('success', TType::STRUCT, 0); + $xfer += $this->success->write($output); $xfer += $output->writeFieldEnd(); } if ($this->o1 !== null) { @@ -9092,6 +12654,16 @@ class ThriftHiveMetastore_get_tables_result { $xfer += $this->o1->write($output); $xfer += $output->writeFieldEnd(); } + if ($this->o2 !== null) { + $xfer += $output->writeFieldBegin('o2', TType::STRUCT, 2); + $xfer += $this->o2->write($output); + $xfer += $output->writeFieldEnd(); + } + if ($this->o3 !== null) { + $xfer += $output->writeFieldBegin('o3', TType::STRUCT, 3); + $xfer += $this->o3->write($output); + $xfer += $output->writeFieldEnd(); + } $xfer += $output->writeFieldStop(); $xfer += $output->writeStructEnd(); return $xfer; @@ -9099,10 +12671,12 @@ class ThriftHiveMetastore_get_tables_result { } -class ThriftHiveMetastore_get_all_tables_args { +class ThriftHiveMetastore_append_partition_by_name_args { static $_TSPEC; public $db_name = null; + public $tbl_name = null; + public $part_name = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { @@ -9111,17 +12685,31 @@ class ThriftHiveMetastore_get_all_tables_args { 'var' => 'db_name', 'type' => TType::STRING, ), + 2 => array( + 'var' => 'tbl_name', + 'type' => TType::STRING, + ), + 3 => array( + 'var' => 'part_name', + 'type' => TType::STRING, + ), ); } if (is_array($vals)) { if (isset($vals['db_name'])) { $this->db_name = $vals['db_name']; } + if (isset($vals['tbl_name'])) { + $this->tbl_name = $vals['tbl_name']; + } + if (isset($vals['part_name'])) { + $this->part_name = $vals['part_name']; + } } } public function getName() { - return 'ThriftHiveMetastore_get_all_tables_args'; + return 'ThriftHiveMetastore_append_partition_by_name_args'; } public function read($input) @@ -9146,6 +12734,20 @@ class ThriftHiveMetastore_get_all_tables_args { $xfer += $input->skip($ftype); } break; + case 2: + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->tbl_name); + } else { + $xfer += $input->skip($ftype); + } + break; + case 3: + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->part_name); + } else { + $xfer += $input->skip($ftype); + } + break; default: $xfer += $input->skip($ftype); break; @@ -9158,12 +12760,22 @@ class ThriftHiveMetastore_get_all_tables_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_all_tables_args'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_append_partition_by_name_args'); if ($this->db_name !== null) { $xfer += $output->writeFieldBegin('db_name', TType::STRING, 1); $xfer += $output->writeString($this->db_name); $xfer += $output->writeFieldEnd(); } + if ($this->tbl_name !== null) { + $xfer += $output->writeFieldBegin('tbl_name', TType::STRING, 2); + $xfer += $output->writeString($this->tbl_name); + $xfer += $output->writeFieldEnd(); + } + if ($this->part_name !== null) { + $xfer += $output->writeFieldBegin('part_name', TType::STRING, 3); + $xfer += $output->writeString($this->part_name); + $xfer += $output->writeFieldEnd(); + } $xfer += $output->writeFieldStop(); $xfer += $output->writeStructEnd(); return $xfer; @@ -9171,26 +12783,35 @@ class ThriftHiveMetastore_get_all_tables_args { } -class ThriftHiveMetastore_get_all_tables_result { +class ThriftHiveMetastore_append_partition_by_name_result { static $_TSPEC; public $success = null; public $o1 = null; + public $o2 = null; + public $o3 = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 0 => array( 'var' => 'success', - 'type' => TType::LST, - 'etype' => TType::STRING, - 'elem' => array( - 'type' => TType::STRING, - ), + 'type' => TType::STRUCT, + 'class' => '\metastore\Partition', ), 1 => array( 'var' => 'o1', 'type' => TType::STRUCT, + 'class' => '\metastore\InvalidObjectException', + ), + 2 => array( + 'var' => 'o2', + 'type' => TType::STRUCT, + 'class' => '\metastore\AlreadyExistsException', + ), + 3 => array( + 'var' => 'o3', + 'type' => TType::STRUCT, 'class' => '\metastore\MetaException', ), ); @@ -9199,14 +12820,20 @@ class ThriftHiveMetastore_get_all_tables_result { if (isset($vals['success'])) { $this->success = $vals['success']; } - if (isset($vals['o1'])) { - $this->o1 = $vals['o1']; + if (isset($vals['o1'])) { + $this->o1 = $vals['o1']; + } + if (isset($vals['o2'])) { + $this->o2 = $vals['o2']; + } + if (isset($vals['o3'])) { + $this->o3 = $vals['o3']; } } } public function getName() { - return 'ThriftHiveMetastore_get_all_tables_result'; + return 'ThriftHiveMetastore_append_partition_by_name_result'; } public function read($input) @@ -9225,30 +12852,37 @@ class ThriftHiveMetastore_get_all_tables_result { switch ($fid) { case 0: - if ($ftype == TType::LST) { - $this->success = array(); - $_size364 = 0; - $_etype367 = 0; - $xfer += $input->readListBegin($_etype367, $_size364); - for ($_i368 = 0; $_i368 < $_size364; ++$_i368) - { - $elem369 = null; - $xfer += $input->readString($elem369); - $this->success []= $elem369; - } - $xfer += $input->readListEnd(); + if ($ftype == TType::STRUCT) { + $this->success = new \metastore\Partition(); + $xfer += $this->success->read($input); } else { $xfer += $input->skip($ftype); } break; case 1: if ($ftype == TType::STRUCT) { - $this->o1 = new \metastore\MetaException(); + $this->o1 = new \metastore\InvalidObjectException(); $xfer += $this->o1->read($input); } else { $xfer += $input->skip($ftype); } break; + case 2: + if ($ftype == TType::STRUCT) { + $this->o2 = new \metastore\AlreadyExistsException(); + $xfer += $this->o2->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; + case 3: + if ($ftype == TType::STRUCT) { + $this->o3 = new \metastore\MetaException(); + $xfer += $this->o3->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; default: $xfer += $input->skip($ftype); break; @@ -9261,22 +12895,13 @@ class ThriftHiveMetastore_get_all_tables_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_all_tables_result'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_append_partition_by_name_result'); if ($this->success !== null) { - if (!is_array($this->success)) { + if (!is_object($this->success)) { throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); } - $xfer += $output->writeFieldBegin('success', TType::LST, 0); - { - $output->writeListBegin(TType::STRING, count($this->success)); - { - foreach ($this->success as $iter370) - { - $xfer += $output->writeString($iter370); - } - } - $output->writeListEnd(); - } + $xfer += $output->writeFieldBegin('success', TType::STRUCT, 0); + $xfer += $this->success->write($output); $xfer += $output->writeFieldEnd(); } if ($this->o1 !== null) { @@ -9284,6 +12909,16 @@ class ThriftHiveMetastore_get_all_tables_result { $xfer += $this->o1->write($output); $xfer += $output->writeFieldEnd(); } + if ($this->o2 !== null) { + $xfer += $output->writeFieldBegin('o2', TType::STRUCT, 2); + $xfer += $this->o2->write($output); + $xfer += $output->writeFieldEnd(); + } + if ($this->o3 !== null) { + $xfer += $output->writeFieldBegin('o3', TType::STRUCT, 3); + $xfer += $this->o3->write($output); + $xfer += $output->writeFieldEnd(); + } $xfer += $output->writeFieldStop(); $xfer += $output->writeStructEnd(); return $xfer; @@ -9291,37 +12926,54 @@ class ThriftHiveMetastore_get_all_tables_result { } -class ThriftHiveMetastore_get_table_args { +class ThriftHiveMetastore_append_partition_by_name_with_environment_context_args { static $_TSPEC; - public $dbname = null; + public $db_name = null; public $tbl_name = null; + public $part_name = null; + public $environment_context = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 1 => array( - 'var' => 'dbname', + 'var' => 'db_name', 'type' => TType::STRING, ), 2 => array( 'var' => 'tbl_name', 'type' => TType::STRING, ), + 3 => array( + 'var' => 'part_name', + 'type' => TType::STRING, + ), + 4 => array( + 'var' => 'environment_context', + 'type' => TType::STRUCT, + 'class' => '\metastore\EnvironmentContext', + ), ); } if (is_array($vals)) { - if (isset($vals['dbname'])) { - $this->dbname = $vals['dbname']; + if (isset($vals['db_name'])) { + $this->db_name = $vals['db_name']; } if (isset($vals['tbl_name'])) { $this->tbl_name = $vals['tbl_name']; } + if (isset($vals['part_name'])) { + $this->part_name = $vals['part_name']; + } + if (isset($vals['environment_context'])) { + $this->environment_context = $vals['environment_context']; + } } } public function getName() { - return 'ThriftHiveMetastore_get_table_args'; + return 'ThriftHiveMetastore_append_partition_by_name_with_environment_context_args'; } public function read($input) @@ -9341,7 +12993,7 @@ class ThriftHiveMetastore_get_table_args { { case 1: if ($ftype == TType::STRING) { - $xfer += $input->readString($this->dbname); + $xfer += $input->readString($this->db_name); } else { $xfer += $input->skip($ftype); } @@ -9353,6 +13005,21 @@ class ThriftHiveMetastore_get_table_args { $xfer += $input->skip($ftype); } break; + case 3: + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->part_name); + } else { + $xfer += $input->skip($ftype); + } + break; + case 4: + if ($ftype == TType::STRUCT) { + $this->environment_context = new \metastore\EnvironmentContext(); + $xfer += $this->environment_context->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; default: $xfer += $input->skip($ftype); break; @@ -9365,10 +13032,10 @@ class ThriftHiveMetastore_get_table_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_table_args'); - if ($this->dbname !== null) { - $xfer += $output->writeFieldBegin('dbname', TType::STRING, 1); - $xfer += $output->writeString($this->dbname); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_append_partition_by_name_with_environment_context_args'); + if ($this->db_name !== null) { + $xfer += $output->writeFieldBegin('db_name', TType::STRING, 1); + $xfer += $output->writeString($this->db_name); $xfer += $output->writeFieldEnd(); } if ($this->tbl_name !== null) { @@ -9376,6 +13043,19 @@ class ThriftHiveMetastore_get_table_args { $xfer += $output->writeString($this->tbl_name); $xfer += $output->writeFieldEnd(); } + if ($this->part_name !== null) { + $xfer += $output->writeFieldBegin('part_name', TType::STRING, 3); + $xfer += $output->writeString($this->part_name); + $xfer += $output->writeFieldEnd(); + } + if ($this->environment_context !== null) { + if (!is_object($this->environment_context)) { + throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); + } + $xfer += $output->writeFieldBegin('environment_context', TType::STRUCT, 4); + $xfer += $this->environment_context->write($output); + $xfer += $output->writeFieldEnd(); + } $xfer += $output->writeFieldStop(); $xfer += $output->writeStructEnd(); return $xfer; @@ -9383,12 +13063,13 @@ class ThriftHiveMetastore_get_table_args { } -class ThriftHiveMetastore_get_table_result { +class ThriftHiveMetastore_append_partition_by_name_with_environment_context_result { static $_TSPEC; public $success = null; public $o1 = null; public $o2 = null; + public $o3 = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { @@ -9396,17 +13077,22 @@ class ThriftHiveMetastore_get_table_result { 0 => array( 'var' => 'success', 'type' => TType::STRUCT, - 'class' => '\metastore\Table', + 'class' => '\metastore\Partition', ), 1 => array( 'var' => 'o1', 'type' => TType::STRUCT, - 'class' => '\metastore\MetaException', + 'class' => '\metastore\InvalidObjectException', ), 2 => array( 'var' => 'o2', 'type' => TType::STRUCT, - 'class' => '\metastore\NoSuchObjectException', + 'class' => '\metastore\AlreadyExistsException', + ), + 3 => array( + 'var' => 'o3', + 'type' => TType::STRUCT, + 'class' => '\metastore\MetaException', ), ); } @@ -9420,11 +13106,14 @@ class ThriftHiveMetastore_get_table_result { if (isset($vals['o2'])) { $this->o2 = $vals['o2']; } + if (isset($vals['o3'])) { + $this->o3 = $vals['o3']; + } } } public function getName() { - return 'ThriftHiveMetastore_get_table_result'; + return 'ThriftHiveMetastore_append_partition_by_name_with_environment_context_result'; } public function read($input) @@ -9444,7 +13133,7 @@ class ThriftHiveMetastore_get_table_result { { case 0: if ($ftype == TType::STRUCT) { - $this->success = new \metastore\Table(); + $this->success = new \metastore\Partition(); $xfer += $this->success->read($input); } else { $xfer += $input->skip($ftype); @@ -9452,7 +13141,7 @@ class ThriftHiveMetastore_get_table_result { break; case 1: if ($ftype == TType::STRUCT) { - $this->o1 = new \metastore\MetaException(); + $this->o1 = new \metastore\InvalidObjectException(); $xfer += $this->o1->read($input); } else { $xfer += $input->skip($ftype); @@ -9460,12 +13149,20 @@ class ThriftHiveMetastore_get_table_result { break; case 2: if ($ftype == TType::STRUCT) { - $this->o2 = new \metastore\NoSuchObjectException(); + $this->o2 = new \metastore\AlreadyExistsException(); $xfer += $this->o2->read($input); } else { $xfer += $input->skip($ftype); } break; + case 3: + if ($ftype == TType::STRUCT) { + $this->o3 = new \metastore\MetaException(); + $xfer += $this->o3->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; default: $xfer += $input->skip($ftype); break; @@ -9478,7 +13175,7 @@ class ThriftHiveMetastore_get_table_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_table_result'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_append_partition_by_name_with_environment_context_result'); if ($this->success !== null) { if (!is_object($this->success)) { throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); @@ -9497,6 +13194,11 @@ class ThriftHiveMetastore_get_table_result { $xfer += $this->o2->write($output); $xfer += $output->writeFieldEnd(); } + if ($this->o3 !== null) { + $xfer += $output->writeFieldBegin('o3', TType::STRUCT, 3); + $xfer += $this->o3->write($output); + $xfer += $output->writeFieldEnd(); + } $xfer += $output->writeFieldStop(); $xfer += $output->writeStructEnd(); return $xfer; @@ -9504,41 +13206,57 @@ class ThriftHiveMetastore_get_table_result { } -class ThriftHiveMetastore_get_table_objects_by_name_args { +class ThriftHiveMetastore_drop_partition_args { static $_TSPEC; - public $dbname = null; - public $tbl_names = null; + public $db_name = null; + public $tbl_name = null; + public $part_vals = null; + public $deleteData = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 1 => array( - 'var' => 'dbname', + 'var' => 'db_name', 'type' => TType::STRING, ), 2 => array( - 'var' => 'tbl_names', + 'var' => 'tbl_name', + 'type' => TType::STRING, + ), + 3 => array( + 'var' => 'part_vals', 'type' => TType::LST, 'etype' => TType::STRING, 'elem' => array( 'type' => TType::STRING, ), ), + 4 => array( + 'var' => 'deleteData', + 'type' => TType::BOOL, + ), ); } if (is_array($vals)) { - if (isset($vals['dbname'])) { - $this->dbname = $vals['dbname']; + if (isset($vals['db_name'])) { + $this->db_name = $vals['db_name']; } - if (isset($vals['tbl_names'])) { - $this->tbl_names = $vals['tbl_names']; + if (isset($vals['tbl_name'])) { + $this->tbl_name = $vals['tbl_name']; + } + if (isset($vals['part_vals'])) { + $this->part_vals = $vals['part_vals']; + } + if (isset($vals['deleteData'])) { + $this->deleteData = $vals['deleteData']; } } } public function getName() { - return 'ThriftHiveMetastore_get_table_objects_by_name_args'; + return 'ThriftHiveMetastore_drop_partition_args'; } public function read($input) @@ -9558,28 +13276,42 @@ class ThriftHiveMetastore_get_table_objects_by_name_args { { case 1: if ($ftype == TType::STRING) { - $xfer += $input->readString($this->dbname); + $xfer += $input->readString($this->db_name); } else { $xfer += $input->skip($ftype); } break; case 2: + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->tbl_name); + } else { + $xfer += $input->skip($ftype); + } + break; + case 3: if ($ftype == TType::LST) { - $this->tbl_names = array(); - $_size371 = 0; - $_etype374 = 0; - $xfer += $input->readListBegin($_etype374, $_size371); - for ($_i375 = 0; $_i375 < $_size371; ++$_i375) + $this->part_vals = array(); + $_size456 = 0; + $_etype459 = 0; + $xfer += $input->readListBegin($_etype459, $_size456); + for ($_i460 = 0; $_i460 < $_size456; ++$_i460) { - $elem376 = null; - $xfer += $input->readString($elem376); - $this->tbl_names []= $elem376; + $elem461 = null; + $xfer += $input->readString($elem461); + $this->part_vals []= $elem461; } $xfer += $input->readListEnd(); } else { $xfer += $input->skip($ftype); } break; + case 4: + if ($ftype == TType::BOOL) { + $xfer += $input->readBool($this->deleteData); + } else { + $xfer += $input->skip($ftype); + } + break; default: $xfer += $input->skip($ftype); break; @@ -9592,29 +13324,39 @@ class ThriftHiveMetastore_get_table_objects_by_name_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_table_objects_by_name_args'); - if ($this->dbname !== null) { - $xfer += $output->writeFieldBegin('dbname', TType::STRING, 1); - $xfer += $output->writeString($this->dbname); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_drop_partition_args'); + if ($this->db_name !== null) { + $xfer += $output->writeFieldBegin('db_name', TType::STRING, 1); + $xfer += $output->writeString($this->db_name); $xfer += $output->writeFieldEnd(); } - if ($this->tbl_names !== null) { - if (!is_array($this->tbl_names)) { + if ($this->tbl_name !== null) { + $xfer += $output->writeFieldBegin('tbl_name', TType::STRING, 2); + $xfer += $output->writeString($this->tbl_name); + $xfer += $output->writeFieldEnd(); + } + if ($this->part_vals !== null) { + if (!is_array($this->part_vals)) { throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); } - $xfer += $output->writeFieldBegin('tbl_names', TType::LST, 2); + $xfer += $output->writeFieldBegin('part_vals', TType::LST, 3); { - $output->writeListBegin(TType::STRING, count($this->tbl_names)); + $output->writeListBegin(TType::STRING, count($this->part_vals)); { - foreach ($this->tbl_names as $iter377) + foreach ($this->part_vals as $iter462) { - $xfer += $output->writeString($iter377); + $xfer += $output->writeString($iter462); } } $output->writeListEnd(); } $xfer += $output->writeFieldEnd(); } + if ($this->deleteData !== null) { + $xfer += $output->writeFieldBegin('deleteData', TType::BOOL, 4); + $xfer += $output->writeBool($this->deleteData); + $xfer += $output->writeFieldEnd(); + } $xfer += $output->writeFieldStop(); $xfer += $output->writeStructEnd(); return $xfer; @@ -9622,40 +13364,29 @@ class ThriftHiveMetastore_get_table_objects_by_name_args { } -class ThriftHiveMetastore_get_table_objects_by_name_result { +class ThriftHiveMetastore_drop_partition_result { static $_TSPEC; public $success = null; public $o1 = null; public $o2 = null; - public $o3 = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( - 0 => array( - 'var' => 'success', - 'type' => TType::LST, - 'etype' => TType::STRUCT, - 'elem' => array( - 'type' => TType::STRUCT, - 'class' => '\metastore\Table', - ), + 0 => array( + 'var' => 'success', + 'type' => TType::BOOL, ), 1 => array( 'var' => 'o1', 'type' => TType::STRUCT, - 'class' => '\metastore\MetaException', + 'class' => '\metastore\NoSuchObjectException', ), 2 => array( 'var' => 'o2', 'type' => TType::STRUCT, - 'class' => '\metastore\InvalidOperationException', - ), - 3 => array( - 'var' => 'o3', - 'type' => TType::STRUCT, - 'class' => '\metastore\UnknownDBException', + 'class' => '\metastore\MetaException', ), ); } @@ -9669,14 +13400,11 @@ class ThriftHiveMetastore_get_table_objects_by_name_result { if (isset($vals['o2'])) { $this->o2 = $vals['o2']; } - if (isset($vals['o3'])) { - $this->o3 = $vals['o3']; - } } } public function getName() { - return 'ThriftHiveMetastore_get_table_objects_by_name_result'; + return 'ThriftHiveMetastore_drop_partition_result'; } public function read($input) @@ -9695,26 +13423,15 @@ class ThriftHiveMetastore_get_table_objects_by_name_result { switch ($fid) { case 0: - if ($ftype == TType::LST) { - $this->success = array(); - $_size378 = 0; - $_etype381 = 0; - $xfer += $input->readListBegin($_etype381, $_size378); - for ($_i382 = 0; $_i382 < $_size378; ++$_i382) - { - $elem383 = null; - $elem383 = new \metastore\Table(); - $xfer += $elem383->read($input); - $this->success []= $elem383; - } - $xfer += $input->readListEnd(); + if ($ftype == TType::BOOL) { + $xfer += $input->readBool($this->success); } else { $xfer += $input->skip($ftype); } break; case 1: if ($ftype == TType::STRUCT) { - $this->o1 = new \metastore\MetaException(); + $this->o1 = new \metastore\NoSuchObjectException(); $xfer += $this->o1->read($input); } else { $xfer += $input->skip($ftype); @@ -9722,20 +13439,12 @@ class ThriftHiveMetastore_get_table_objects_by_name_result { break; case 2: if ($ftype == TType::STRUCT) { - $this->o2 = new \metastore\InvalidOperationException(); + $this->o2 = new \metastore\MetaException(); $xfer += $this->o2->read($input); } else { $xfer += $input->skip($ftype); } break; - case 3: - if ($ftype == TType::STRUCT) { - $this->o3 = new \metastore\UnknownDBException(); - $xfer += $this->o3->read($input); - } else { - $xfer += $input->skip($ftype); - } - break; default: $xfer += $input->skip($ftype); break; @@ -9748,22 +13457,10 @@ class ThriftHiveMetastore_get_table_objects_by_name_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_table_objects_by_name_result'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_drop_partition_result'); if ($this->success !== null) { - if (!is_array($this->success)) { - throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); - } - $xfer += $output->writeFieldBegin('success', TType::LST, 0); - { - $output->writeListBegin(TType::STRUCT, count($this->success)); - { - foreach ($this->success as $iter384) - { - $xfer += $iter384->write($output); - } - } - $output->writeListEnd(); - } + $xfer += $output->writeFieldBegin('success', TType::BOOL, 0); + $xfer += $output->writeBool($this->success); $xfer += $output->writeFieldEnd(); } if ($this->o1 !== null) { @@ -9776,11 +13473,6 @@ class ThriftHiveMetastore_get_table_objects_by_name_result { $xfer += $this->o2->write($output); $xfer += $output->writeFieldEnd(); } - if ($this->o3 !== null) { - $xfer += $output->writeFieldBegin('o3', TType::STRUCT, 3); - $xfer += $this->o3->write($output); - $xfer += $output->writeFieldEnd(); - } $xfer += $output->writeFieldStop(); $xfer += $output->writeStructEnd(); return $xfer; @@ -9788,45 +13480,66 @@ class ThriftHiveMetastore_get_table_objects_by_name_result { } -class ThriftHiveMetastore_get_table_names_by_filter_args { +class ThriftHiveMetastore_drop_partition_with_environment_context_args { static $_TSPEC; - public $dbname = null; - public $filter = null; - public $max_tables = -1; + public $db_name = null; + public $tbl_name = null; + public $part_vals = null; + public $deleteData = null; + public $environment_context = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 1 => array( - 'var' => 'dbname', + 'var' => 'db_name', 'type' => TType::STRING, ), 2 => array( - 'var' => 'filter', + 'var' => 'tbl_name', 'type' => TType::STRING, ), 3 => array( - 'var' => 'max_tables', - 'type' => TType::I16, + 'var' => 'part_vals', + 'type' => TType::LST, + 'etype' => TType::STRING, + 'elem' => array( + 'type' => TType::STRING, + ), + ), + 4 => array( + 'var' => 'deleteData', + 'type' => TType::BOOL, + ), + 5 => array( + 'var' => 'environment_context', + 'type' => TType::STRUCT, + 'class' => '\metastore\EnvironmentContext', ), ); } if (is_array($vals)) { - if (isset($vals['dbname'])) { - $this->dbname = $vals['dbname']; + if (isset($vals['db_name'])) { + $this->db_name = $vals['db_name']; } - if (isset($vals['filter'])) { - $this->filter = $vals['filter']; + if (isset($vals['tbl_name'])) { + $this->tbl_name = $vals['tbl_name']; } - if (isset($vals['max_tables'])) { - $this->max_tables = $vals['max_tables']; + if (isset($vals['part_vals'])) { + $this->part_vals = $vals['part_vals']; + } + if (isset($vals['deleteData'])) { + $this->deleteData = $vals['deleteData']; + } + if (isset($vals['environment_context'])) { + $this->environment_context = $vals['environment_context']; } } } public function getName() { - return 'ThriftHiveMetastore_get_table_names_by_filter_args'; + return 'ThriftHiveMetastore_drop_partition_with_environment_context_args'; } public function read($input) @@ -9846,21 +13559,46 @@ class ThriftHiveMetastore_get_table_names_by_filter_args { { case 1: if ($ftype == TType::STRING) { - $xfer += $input->readString($this->dbname); + $xfer += $input->readString($this->db_name); } else { $xfer += $input->skip($ftype); } break; case 2: if ($ftype == TType::STRING) { - $xfer += $input->readString($this->filter); + $xfer += $input->readString($this->tbl_name); } else { $xfer += $input->skip($ftype); } break; case 3: - if ($ftype == TType::I16) { - $xfer += $input->readI16($this->max_tables); + if ($ftype == TType::LST) { + $this->part_vals = array(); + $_size463 = 0; + $_etype466 = 0; + $xfer += $input->readListBegin($_etype466, $_size463); + for ($_i467 = 0; $_i467 < $_size463; ++$_i467) + { + $elem468 = null; + $xfer += $input->readString($elem468); + $this->part_vals []= $elem468; + } + $xfer += $input->readListEnd(); + } else { + $xfer += $input->skip($ftype); + } + break; + case 4: + if ($ftype == TType::BOOL) { + $xfer += $input->readBool($this->deleteData); + } else { + $xfer += $input->skip($ftype); + } + break; + case 5: + if ($ftype == TType::STRUCT) { + $this->environment_context = new \metastore\EnvironmentContext(); + $xfer += $this->environment_context->read($input); } else { $xfer += $input->skip($ftype); } @@ -9877,20 +13615,45 @@ class ThriftHiveMetastore_get_table_names_by_filter_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_table_names_by_filter_args'); - if ($this->dbname !== null) { - $xfer += $output->writeFieldBegin('dbname', TType::STRING, 1); - $xfer += $output->writeString($this->dbname); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_drop_partition_with_environment_context_args'); + if ($this->db_name !== null) { + $xfer += $output->writeFieldBegin('db_name', TType::STRING, 1); + $xfer += $output->writeString($this->db_name); $xfer += $output->writeFieldEnd(); } - if ($this->filter !== null) { - $xfer += $output->writeFieldBegin('filter', TType::STRING, 2); - $xfer += $output->writeString($this->filter); + if ($this->tbl_name !== null) { + $xfer += $output->writeFieldBegin('tbl_name', TType::STRING, 2); + $xfer += $output->writeString($this->tbl_name); $xfer += $output->writeFieldEnd(); } - if ($this->max_tables !== null) { - $xfer += $output->writeFieldBegin('max_tables', TType::I16, 3); - $xfer += $output->writeI16($this->max_tables); + if ($this->part_vals !== null) { + if (!is_array($this->part_vals)) { + throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); + } + $xfer += $output->writeFieldBegin('part_vals', TType::LST, 3); + { + $output->writeListBegin(TType::STRING, count($this->part_vals)); + { + foreach ($this->part_vals as $iter469) + { + $xfer += $output->writeString($iter469); + } + } + $output->writeListEnd(); + } + $xfer += $output->writeFieldEnd(); + } + if ($this->deleteData !== null) { + $xfer += $output->writeFieldBegin('deleteData', TType::BOOL, 4); + $xfer += $output->writeBool($this->deleteData); + $xfer += $output->writeFieldEnd(); + } + if ($this->environment_context !== null) { + if (!is_object($this->environment_context)) { + throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); + } + $xfer += $output->writeFieldBegin('environment_context', TType::STRUCT, 5); + $xfer += $this->environment_context->write($output); $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -9900,39 +13663,29 @@ class ThriftHiveMetastore_get_table_names_by_filter_args { } -class ThriftHiveMetastore_get_table_names_by_filter_result { +class ThriftHiveMetastore_drop_partition_with_environment_context_result { static $_TSPEC; public $success = null; public $o1 = null; public $o2 = null; - public $o3 = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 0 => array( 'var' => 'success', - 'type' => TType::LST, - 'etype' => TType::STRING, - 'elem' => array( - 'type' => TType::STRING, - ), + 'type' => TType::BOOL, ), 1 => array( 'var' => 'o1', 'type' => TType::STRUCT, - 'class' => '\metastore\MetaException', + 'class' => '\metastore\NoSuchObjectException', ), 2 => array( 'var' => 'o2', 'type' => TType::STRUCT, - 'class' => '\metastore\InvalidOperationException', - ), - 3 => array( - 'var' => 'o3', - 'type' => TType::STRUCT, - 'class' => '\metastore\UnknownDBException', + 'class' => '\metastore\MetaException', ), ); } @@ -9946,14 +13699,11 @@ class ThriftHiveMetastore_get_table_names_by_filter_result { if (isset($vals['o2'])) { $this->o2 = $vals['o2']; } - if (isset($vals['o3'])) { - $this->o3 = $vals['o3']; - } } } public function getName() { - return 'ThriftHiveMetastore_get_table_names_by_filter_result'; + return 'ThriftHiveMetastore_drop_partition_with_environment_context_result'; } public function read($input) @@ -9972,25 +13722,15 @@ class ThriftHiveMetastore_get_table_names_by_filter_result { switch ($fid) { case 0: - if ($ftype == TType::LST) { - $this->success = array(); - $_size385 = 0; - $_etype388 = 0; - $xfer += $input->readListBegin($_etype388, $_size385); - for ($_i389 = 0; $_i389 < $_size385; ++$_i389) - { - $elem390 = null; - $xfer += $input->readString($elem390); - $this->success []= $elem390; - } - $xfer += $input->readListEnd(); + if ($ftype == TType::BOOL) { + $xfer += $input->readBool($this->success); } else { $xfer += $input->skip($ftype); } break; case 1: if ($ftype == TType::STRUCT) { - $this->o1 = new \metastore\MetaException(); + $this->o1 = new \metastore\NoSuchObjectException(); $xfer += $this->o1->read($input); } else { $xfer += $input->skip($ftype); @@ -9998,20 +13738,12 @@ class ThriftHiveMetastore_get_table_names_by_filter_result { break; case 2: if ($ftype == TType::STRUCT) { - $this->o2 = new \metastore\InvalidOperationException(); + $this->o2 = new \metastore\MetaException(); $xfer += $this->o2->read($input); } else { $xfer += $input->skip($ftype); } break; - case 3: - if ($ftype == TType::STRUCT) { - $this->o3 = new \metastore\UnknownDBException(); - $xfer += $this->o3->read($input); - } else { - $xfer += $input->skip($ftype); - } - break; default: $xfer += $input->skip($ftype); break; @@ -10024,22 +13756,10 @@ class ThriftHiveMetastore_get_table_names_by_filter_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_table_names_by_filter_result'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_drop_partition_with_environment_context_result'); if ($this->success !== null) { - if (!is_array($this->success)) { - throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); - } - $xfer += $output->writeFieldBegin('success', TType::LST, 0); - { - $output->writeListBegin(TType::STRING, count($this->success)); - { - foreach ($this->success as $iter391) - { - $xfer += $output->writeString($iter391); - } - } - $output->writeListEnd(); - } + $xfer += $output->writeFieldBegin('success', TType::BOOL, 0); + $xfer += $output->writeBool($this->success); $xfer += $output->writeFieldEnd(); } if ($this->o1 !== null) { @@ -10052,11 +13772,6 @@ class ThriftHiveMetastore_get_table_names_by_filter_result { $xfer += $this->o2->write($output); $xfer += $output->writeFieldEnd(); } - if ($this->o3 !== null) { - $xfer += $output->writeFieldBegin('o3', TType::STRUCT, 3); - $xfer += $this->o3->write($output); - $xfer += $output->writeFieldEnd(); - } $xfer += $output->writeFieldStop(); $xfer += $output->writeStructEnd(); return $xfer; @@ -10064,18 +13779,19 @@ class ThriftHiveMetastore_get_table_names_by_filter_result { } -class ThriftHiveMetastore_alter_table_args { +class ThriftHiveMetastore_drop_partition_by_name_args { static $_TSPEC; - public $dbname = null; + public $db_name = null; public $tbl_name = null; - public $new_tbl = null; + public $part_name = null; + public $deleteData = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 1 => array( - 'var' => 'dbname', + 'var' => 'db_name', 'type' => TType::STRING, ), 2 => array( @@ -10083,27 +13799,33 @@ class ThriftHiveMetastore_alter_table_args { 'type' => TType::STRING, ), 3 => array( - 'var' => 'new_tbl', - 'type' => TType::STRUCT, - 'class' => '\metastore\Table', + 'var' => 'part_name', + 'type' => TType::STRING, + ), + 4 => array( + 'var' => 'deleteData', + 'type' => TType::BOOL, ), ); } if (is_array($vals)) { - if (isset($vals['dbname'])) { - $this->dbname = $vals['dbname']; + if (isset($vals['db_name'])) { + $this->db_name = $vals['db_name']; } if (isset($vals['tbl_name'])) { $this->tbl_name = $vals['tbl_name']; } - if (isset($vals['new_tbl'])) { - $this->new_tbl = $vals['new_tbl']; + if (isset($vals['part_name'])) { + $this->part_name = $vals['part_name']; + } + if (isset($vals['deleteData'])) { + $this->deleteData = $vals['deleteData']; } } } public function getName() { - return 'ThriftHiveMetastore_alter_table_args'; + return 'ThriftHiveMetastore_drop_partition_by_name_args'; } public function read($input) @@ -10123,7 +13845,7 @@ class ThriftHiveMetastore_alter_table_args { { case 1: if ($ftype == TType::STRING) { - $xfer += $input->readString($this->dbname); + $xfer += $input->readString($this->db_name); } else { $xfer += $input->skip($ftype); } @@ -10136,9 +13858,15 @@ class ThriftHiveMetastore_alter_table_args { } break; case 3: - if ($ftype == TType::STRUCT) { - $this->new_tbl = new \metastore\Table(); - $xfer += $this->new_tbl->read($input); + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->part_name); + } else { + $xfer += $input->skip($ftype); + } + break; + case 4: + if ($ftype == TType::BOOL) { + $xfer += $input->readBool($this->deleteData); } else { $xfer += $input->skip($ftype); } @@ -10155,10 +13883,10 @@ class ThriftHiveMetastore_alter_table_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_alter_table_args'); - if ($this->dbname !== null) { - $xfer += $output->writeFieldBegin('dbname', TType::STRING, 1); - $xfer += $output->writeString($this->dbname); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_drop_partition_by_name_args'); + if ($this->db_name !== null) { + $xfer += $output->writeFieldBegin('db_name', TType::STRING, 1); + $xfer += $output->writeString($this->db_name); $xfer += $output->writeFieldEnd(); } if ($this->tbl_name !== null) { @@ -10166,12 +13894,14 @@ class ThriftHiveMetastore_alter_table_args { $xfer += $output->writeString($this->tbl_name); $xfer += $output->writeFieldEnd(); } - if ($this->new_tbl !== null) { - if (!is_object($this->new_tbl)) { - throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); - } - $xfer += $output->writeFieldBegin('new_tbl', TType::STRUCT, 3); - $xfer += $this->new_tbl->write($output); + if ($this->part_name !== null) { + $xfer += $output->writeFieldBegin('part_name', TType::STRING, 3); + $xfer += $output->writeString($this->part_name); + $xfer += $output->writeFieldEnd(); + } + if ($this->deleteData !== null) { + $xfer += $output->writeFieldBegin('deleteData', TType::BOOL, 4); + $xfer += $output->writeBool($this->deleteData); $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -10181,19 +13911,24 @@ class ThriftHiveMetastore_alter_table_args { } -class ThriftHiveMetastore_alter_table_result { +class ThriftHiveMetastore_drop_partition_by_name_result { static $_TSPEC; + public $success = null; public $o1 = null; public $o2 = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( + 0 => array( + 'var' => 'success', + 'type' => TType::BOOL, + ), 1 => array( 'var' => 'o1', 'type' => TType::STRUCT, - 'class' => '\metastore\InvalidOperationException', + 'class' => '\metastore\NoSuchObjectException', ), 2 => array( 'var' => 'o2', @@ -10203,6 +13938,9 @@ class ThriftHiveMetastore_alter_table_result { ); } if (is_array($vals)) { + if (isset($vals['success'])) { + $this->success = $vals['success']; + } if (isset($vals['o1'])) { $this->o1 = $vals['o1']; } @@ -10213,7 +13951,7 @@ class ThriftHiveMetastore_alter_table_result { } public function getName() { - return 'ThriftHiveMetastore_alter_table_result'; + return 'ThriftHiveMetastore_drop_partition_by_name_result'; } public function read($input) @@ -10231,9 +13969,16 @@ class ThriftHiveMetastore_alter_table_result { } switch ($fid) { + case 0: + if ($ftype == TType::BOOL) { + $xfer += $input->readBool($this->success); + } else { + $xfer += $input->skip($ftype); + } + break; case 1: if ($ftype == TType::STRUCT) { - $this->o1 = new \metastore\InvalidOperationException(); + $this->o1 = new \metastore\NoSuchObjectException(); $xfer += $this->o1->read($input); } else { $xfer += $input->skip($ftype); @@ -10259,7 +14004,12 @@ class ThriftHiveMetastore_alter_table_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_alter_table_result'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_drop_partition_by_name_result'); + if ($this->success !== null) { + $xfer += $output->writeFieldBegin('success', TType::BOOL, 0); + $xfer += $output->writeBool($this->success); + $xfer += $output->writeFieldEnd(); + } if ($this->o1 !== null) { $xfer += $output->writeFieldBegin('o1', TType::STRUCT, 1); $xfer += $this->o1->write($output); @@ -10277,19 +14027,20 @@ class ThriftHiveMetastore_alter_table_result { } -class ThriftHiveMetastore_alter_table_with_environment_context_args { +class ThriftHiveMetastore_drop_partition_by_name_with_environment_context_args { static $_TSPEC; - public $dbname = null; + public $db_name = null; public $tbl_name = null; - public $new_tbl = null; + public $part_name = null; + public $deleteData = null; public $environment_context = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 1 => array( - 'var' => 'dbname', + 'var' => 'db_name', 'type' => TType::STRING, ), 2 => array( @@ -10297,11 +14048,14 @@ class ThriftHiveMetastore_alter_table_with_environment_context_args { 'type' => TType::STRING, ), 3 => array( - 'var' => 'new_tbl', - 'type' => TType::STRUCT, - 'class' => '\metastore\Table', + 'var' => 'part_name', + 'type' => TType::STRING, ), 4 => array( + 'var' => 'deleteData', + 'type' => TType::BOOL, + ), + 5 => array( 'var' => 'environment_context', 'type' => TType::STRUCT, 'class' => '\metastore\EnvironmentContext', @@ -10309,14 +14063,17 @@ class ThriftHiveMetastore_alter_table_with_environment_context_args { ); } if (is_array($vals)) { - if (isset($vals['dbname'])) { - $this->dbname = $vals['dbname']; + if (isset($vals['db_name'])) { + $this->db_name = $vals['db_name']; } if (isset($vals['tbl_name'])) { $this->tbl_name = $vals['tbl_name']; } - if (isset($vals['new_tbl'])) { - $this->new_tbl = $vals['new_tbl']; + if (isset($vals['part_name'])) { + $this->part_name = $vals['part_name']; + } + if (isset($vals['deleteData'])) { + $this->deleteData = $vals['deleteData']; } if (isset($vals['environment_context'])) { $this->environment_context = $vals['environment_context']; @@ -10325,7 +14082,7 @@ class ThriftHiveMetastore_alter_table_with_environment_context_args { } public function getName() { - return 'ThriftHiveMetastore_alter_table_with_environment_context_args'; + return 'ThriftHiveMetastore_drop_partition_by_name_with_environment_context_args'; } public function read($input) @@ -10345,7 +14102,7 @@ class ThriftHiveMetastore_alter_table_with_environment_context_args { { case 1: if ($ftype == TType::STRING) { - $xfer += $input->readString($this->dbname); + $xfer += $input->readString($this->db_name); } else { $xfer += $input->skip($ftype); } @@ -10358,14 +14115,20 @@ class ThriftHiveMetastore_alter_table_with_environment_context_args { } break; case 3: - if ($ftype == TType::STRUCT) { - $this->new_tbl = new \metastore\Table(); - $xfer += $this->new_tbl->read($input); + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->part_name); } else { $xfer += $input->skip($ftype); } break; case 4: + if ($ftype == TType::BOOL) { + $xfer += $input->readBool($this->deleteData); + } else { + $xfer += $input->skip($ftype); + } + break; + case 5: if ($ftype == TType::STRUCT) { $this->environment_context = new \metastore\EnvironmentContext(); $xfer += $this->environment_context->read($input); @@ -10385,10 +14148,10 @@ class ThriftHiveMetastore_alter_table_with_environment_context_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_alter_table_with_environment_context_args'); - if ($this->dbname !== null) { - $xfer += $output->writeFieldBegin('dbname', TType::STRING, 1); - $xfer += $output->writeString($this->dbname); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_drop_partition_by_name_with_environment_context_args'); + if ($this->db_name !== null) { + $xfer += $output->writeFieldBegin('db_name', TType::STRING, 1); + $xfer += $output->writeString($this->db_name); $xfer += $output->writeFieldEnd(); } if ($this->tbl_name !== null) { @@ -10396,19 +14159,21 @@ class ThriftHiveMetastore_alter_table_with_environment_context_args { $xfer += $output->writeString($this->tbl_name); $xfer += $output->writeFieldEnd(); } - if ($this->new_tbl !== null) { - if (!is_object($this->new_tbl)) { - throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); - } - $xfer += $output->writeFieldBegin('new_tbl', TType::STRUCT, 3); - $xfer += $this->new_tbl->write($output); + if ($this->part_name !== null) { + $xfer += $output->writeFieldBegin('part_name', TType::STRING, 3); + $xfer += $output->writeString($this->part_name); + $xfer += $output->writeFieldEnd(); + } + if ($this->deleteData !== null) { + $xfer += $output->writeFieldBegin('deleteData', TType::BOOL, 4); + $xfer += $output->writeBool($this->deleteData); $xfer += $output->writeFieldEnd(); } if ($this->environment_context !== null) { if (!is_object($this->environment_context)) { throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); } - $xfer += $output->writeFieldBegin('environment_context', TType::STRUCT, 4); + $xfer += $output->writeFieldBegin('environment_context', TType::STRUCT, 5); $xfer += $this->environment_context->write($output); $xfer += $output->writeFieldEnd(); } @@ -10419,19 +14184,24 @@ class ThriftHiveMetastore_alter_table_with_environment_context_args { } -class ThriftHiveMetastore_alter_table_with_environment_context_result { +class ThriftHiveMetastore_drop_partition_by_name_with_environment_context_result { static $_TSPEC; + public $success = null; public $o1 = null; public $o2 = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( + 0 => array( + 'var' => 'success', + 'type' => TType::BOOL, + ), 1 => array( 'var' => 'o1', 'type' => TType::STRUCT, - 'class' => '\metastore\InvalidOperationException', + 'class' => '\metastore\NoSuchObjectException', ), 2 => array( 'var' => 'o2', @@ -10441,6 +14211,9 @@ class ThriftHiveMetastore_alter_table_with_environment_context_result { ); } if (is_array($vals)) { + if (isset($vals['success'])) { + $this->success = $vals['success']; + } if (isset($vals['o1'])) { $this->o1 = $vals['o1']; } @@ -10451,7 +14224,7 @@ class ThriftHiveMetastore_alter_table_with_environment_context_result { } public function getName() { - return 'ThriftHiveMetastore_alter_table_with_environment_context_result'; + return 'ThriftHiveMetastore_drop_partition_by_name_with_environment_context_result'; } public function read($input) @@ -10469,9 +14242,16 @@ class ThriftHiveMetastore_alter_table_with_environment_context_result { } switch ($fid) { + case 0: + if ($ftype == TType::BOOL) { + $xfer += $input->readBool($this->success); + } else { + $xfer += $input->skip($ftype); + } + break; case 1: if ($ftype == TType::STRUCT) { - $this->o1 = new \metastore\InvalidOperationException(); + $this->o1 = new \metastore\NoSuchObjectException(); $xfer += $this->o1->read($input); } else { $xfer += $input->skip($ftype); @@ -10497,7 +14277,12 @@ class ThriftHiveMetastore_alter_table_with_environment_context_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_alter_table_with_environment_context_result'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_drop_partition_by_name_with_environment_context_result'); + if ($this->success !== null) { + $xfer += $output->writeFieldBegin('success', TType::BOOL, 0); + $xfer += $output->writeBool($this->success); + $xfer += $output->writeFieldEnd(); + } if ($this->o1 !== null) { $xfer += $output->writeFieldBegin('o1', TType::STRUCT, 1); $xfer += $this->o1->write($output); @@ -10515,30 +14300,30 @@ class ThriftHiveMetastore_alter_table_with_environment_context_result { } -class ThriftHiveMetastore_add_partition_args { +class ThriftHiveMetastore_drop_partitions_req_args { static $_TSPEC; - public $new_part = null; + public $req = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 1 => array( - 'var' => 'new_part', + 'var' => 'req', 'type' => TType::STRUCT, - 'class' => '\metastore\Partition', + 'class' => '\metastore\DropPartitionsRequest', ), ); } if (is_array($vals)) { - if (isset($vals['new_part'])) { - $this->new_part = $vals['new_part']; + if (isset($vals['req'])) { + $this->req = $vals['req']; } } } public function getName() { - return 'ThriftHiveMetastore_add_partition_args'; + return 'ThriftHiveMetastore_drop_partitions_req_args'; } public function read($input) @@ -10558,8 +14343,8 @@ class ThriftHiveMetastore_add_partition_args { { case 1: if ($ftype == TType::STRUCT) { - $this->new_part = new \metastore\Partition(); - $xfer += $this->new_part->read($input); + $this->req = new \metastore\DropPartitionsRequest(); + $xfer += $this->req->read($input); } else { $xfer += $input->skip($ftype); } @@ -10576,13 +14361,13 @@ class ThriftHiveMetastore_add_partition_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_add_partition_args'); - if ($this->new_part !== null) { - if (!is_object($this->new_part)) { + $xfer += $output->writeStructBegin('ThriftHiveMetastore_drop_partitions_req_args'); + if ($this->req !== null) { + if (!is_object($this->req)) { throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); } - $xfer += $output->writeFieldBegin('new_part', TType::STRUCT, 1); - $xfer += $this->new_part->write($output); + $xfer += $output->writeFieldBegin('req', TType::STRUCT, 1); + $xfer += $this->req->write($output); $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -10592,13 +14377,12 @@ class ThriftHiveMetastore_add_partition_args { } -class ThriftHiveMetastore_add_partition_result { +class ThriftHiveMetastore_drop_partitions_req_result { static $_TSPEC; public $success = null; public $o1 = null; public $o2 = null; - public $o3 = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { @@ -10606,21 +14390,16 @@ class ThriftHiveMetastore_add_partition_result { 0 => array( 'var' => 'success', 'type' => TType::STRUCT, - 'class' => '\metastore\Partition', + 'class' => '\metastore\DropPartitionsResult', ), 1 => array( 'var' => 'o1', 'type' => TType::STRUCT, - 'class' => '\metastore\InvalidObjectException', + 'class' => '\metastore\NoSuchObjectException', ), 2 => array( 'var' => 'o2', 'type' => TType::STRUCT, - 'class' => '\metastore\AlreadyExistsException', - ), - 3 => array( - 'var' => 'o3', - 'type' => TType::STRUCT, 'class' => '\metastore\MetaException', ), ); @@ -10635,14 +14414,11 @@ class ThriftHiveMetastore_add_partition_result { if (isset($vals['o2'])) { $this->o2 = $vals['o2']; } - if (isset($vals['o3'])) { - $this->o3 = $vals['o3']; - } } } public function getName() { - return 'ThriftHiveMetastore_add_partition_result'; + return 'ThriftHiveMetastore_drop_partitions_req_result'; } public function read($input) @@ -10662,7 +14438,7 @@ class ThriftHiveMetastore_add_partition_result { { case 0: if ($ftype == TType::STRUCT) { - $this->success = new \metastore\Partition(); + $this->success = new \metastore\DropPartitionsResult(); $xfer += $this->success->read($input); } else { $xfer += $input->skip($ftype); @@ -10670,7 +14446,7 @@ class ThriftHiveMetastore_add_partition_result { break; case 1: if ($ftype == TType::STRUCT) { - $this->o1 = new \metastore\InvalidObjectException(); + $this->o1 = new \metastore\NoSuchObjectException(); $xfer += $this->o1->read($input); } else { $xfer += $input->skip($ftype); @@ -10678,20 +14454,12 @@ class ThriftHiveMetastore_add_partition_result { break; case 2: if ($ftype == TType::STRUCT) { - $this->o2 = new \metastore\AlreadyExistsException(); + $this->o2 = new \metastore\MetaException(); $xfer += $this->o2->read($input); } else { $xfer += $input->skip($ftype); } break; - case 3: - if ($ftype == TType::STRUCT) { - $this->o3 = new \metastore\MetaException(); - $xfer += $this->o3->read($input); - } else { - $xfer += $input->skip($ftype); - } - break; default: $xfer += $input->skip($ftype); break; @@ -10704,7 +14472,7 @@ class ThriftHiveMetastore_add_partition_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_add_partition_result'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_drop_partitions_req_result'); if ($this->success !== null) { if (!is_object($this->success)) { throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); @@ -10723,11 +14491,6 @@ class ThriftHiveMetastore_add_partition_result { $xfer += $this->o2->write($output); $xfer += $output->writeFieldEnd(); } - if ($this->o3 !== null) { - $xfer += $output->writeFieldBegin('o3', TType::STRUCT, 3); - $xfer += $this->o3->write($output); - $xfer += $output->writeFieldEnd(); - } $xfer += $output->writeFieldStop(); $xfer += $output->writeStructEnd(); return $xfer; @@ -10735,39 +14498,49 @@ class ThriftHiveMetastore_add_partition_result { } -class ThriftHiveMetastore_add_partition_with_environment_context_args { +class ThriftHiveMetastore_get_partition_args { static $_TSPEC; - public $new_part = null; - public $environment_context = null; + public $db_name = null; + public $tbl_name = null; + public $part_vals = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 1 => array( - 'var' => 'new_part', - 'type' => TType::STRUCT, - 'class' => '\metastore\Partition', + 'var' => 'db_name', + 'type' => TType::STRING, ), 2 => array( - 'var' => 'environment_context', - 'type' => TType::STRUCT, - 'class' => '\metastore\EnvironmentContext', + 'var' => 'tbl_name', + 'type' => TType::STRING, + ), + 3 => array( + 'var' => 'part_vals', + 'type' => TType::LST, + 'etype' => TType::STRING, + 'elem' => array( + 'type' => TType::STRING, + ), ), ); } if (is_array($vals)) { - if (isset($vals['new_part'])) { - $this->new_part = $vals['new_part']; + if (isset($vals['db_name'])) { + $this->db_name = $vals['db_name']; } - if (isset($vals['environment_context'])) { - $this->environment_context = $vals['environment_context']; + if (isset($vals['tbl_name'])) { + $this->tbl_name = $vals['tbl_name']; + } + if (isset($vals['part_vals'])) { + $this->part_vals = $vals['part_vals']; } } } public function getName() { - return 'ThriftHiveMetastore_add_partition_with_environment_context_args'; + return 'ThriftHiveMetastore_get_partition_args'; } public function read($input) @@ -10786,17 +14559,32 @@ class ThriftHiveMetastore_add_partition_with_environment_context_args { switch ($fid) { case 1: - if ($ftype == TType::STRUCT) { - $this->new_part = new \metastore\Partition(); - $xfer += $this->new_part->read($input); + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->db_name); } else { $xfer += $input->skip($ftype); } break; case 2: - if ($ftype == TType::STRUCT) { - $this->environment_context = new \metastore\EnvironmentContext(); - $xfer += $this->environment_context->read($input); + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->tbl_name); + } else { + $xfer += $input->skip($ftype); + } + break; + case 3: + if ($ftype == TType::LST) { + $this->part_vals = array(); + $_size470 = 0; + $_etype473 = 0; + $xfer += $input->readListBegin($_etype473, $_size470); + for ($_i474 = 0; $_i474 < $_size470; ++$_i474) + { + $elem475 = null; + $xfer += $input->readString($elem475); + $this->part_vals []= $elem475; + } + $xfer += $input->readListEnd(); } else { $xfer += $input->skip($ftype); } @@ -10813,21 +14601,32 @@ class ThriftHiveMetastore_add_partition_with_environment_context_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_add_partition_with_environment_context_args'); - if ($this->new_part !== null) { - if (!is_object($this->new_part)) { - throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); - } - $xfer += $output->writeFieldBegin('new_part', TType::STRUCT, 1); - $xfer += $this->new_part->write($output); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_partition_args'); + if ($this->db_name !== null) { + $xfer += $output->writeFieldBegin('db_name', TType::STRING, 1); + $xfer += $output->writeString($this->db_name); $xfer += $output->writeFieldEnd(); } - if ($this->environment_context !== null) { - if (!is_object($this->environment_context)) { + if ($this->tbl_name !== null) { + $xfer += $output->writeFieldBegin('tbl_name', TType::STRING, 2); + $xfer += $output->writeString($this->tbl_name); + $xfer += $output->writeFieldEnd(); + } + if ($this->part_vals !== null) { + if (!is_array($this->part_vals)) { throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); } - $xfer += $output->writeFieldBegin('environment_context', TType::STRUCT, 2); - $xfer += $this->environment_context->write($output); + $xfer += $output->writeFieldBegin('part_vals', TType::LST, 3); + { + $output->writeListBegin(TType::STRING, count($this->part_vals)); + { + foreach ($this->part_vals as $iter476) + { + $xfer += $output->writeString($iter476); + } + } + $output->writeListEnd(); + } $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -10837,13 +14636,12 @@ class ThriftHiveMetastore_add_partition_with_environment_context_args { } -class ThriftHiveMetastore_add_partition_with_environment_context_result { +class ThriftHiveMetastore_get_partition_result { static $_TSPEC; public $success = null; public $o1 = null; public $o2 = null; - public $o3 = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { @@ -10856,17 +14654,12 @@ class ThriftHiveMetastore_add_partition_with_environment_context_result { 1 => array( 'var' => 'o1', 'type' => TType::STRUCT, - 'class' => '\metastore\InvalidObjectException', + 'class' => '\metastore\MetaException', ), 2 => array( 'var' => 'o2', 'type' => TType::STRUCT, - 'class' => '\metastore\AlreadyExistsException', - ), - 3 => array( - 'var' => 'o3', - 'type' => TType::STRUCT, - 'class' => '\metastore\MetaException', + 'class' => '\metastore\NoSuchObjectException', ), ); } @@ -10880,14 +14673,11 @@ class ThriftHiveMetastore_add_partition_with_environment_context_result { if (isset($vals['o2'])) { $this->o2 = $vals['o2']; } - if (isset($vals['o3'])) { - $this->o3 = $vals['o3']; - } } } public function getName() { - return 'ThriftHiveMetastore_add_partition_with_environment_context_result'; + return 'ThriftHiveMetastore_get_partition_result'; } public function read($input) @@ -10915,7 +14705,7 @@ class ThriftHiveMetastore_add_partition_with_environment_context_result { break; case 1: if ($ftype == TType::STRUCT) { - $this->o1 = new \metastore\InvalidObjectException(); + $this->o1 = new \metastore\MetaException(); $xfer += $this->o1->read($input); } else { $xfer += $input->skip($ftype); @@ -10923,20 +14713,12 @@ class ThriftHiveMetastore_add_partition_with_environment_context_result { break; case 2: if ($ftype == TType::STRUCT) { - $this->o2 = new \metastore\AlreadyExistsException(); + $this->o2 = new \metastore\NoSuchObjectException(); $xfer += $this->o2->read($input); } else { $xfer += $input->skip($ftype); } break; - case 3: - if ($ftype == TType::STRUCT) { - $this->o3 = new \metastore\MetaException(); - $xfer += $this->o3->read($input); - } else { - $xfer += $input->skip($ftype); - } - break; default: $xfer += $input->skip($ftype); break; @@ -10949,7 +14731,7 @@ class ThriftHiveMetastore_add_partition_with_environment_context_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_add_partition_with_environment_context_result'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_partition_result'); if ($this->success !== null) { if (!is_object($this->success)) { throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); @@ -10968,11 +14750,6 @@ class ThriftHiveMetastore_add_partition_with_environment_context_result { $xfer += $this->o2->write($output); $xfer += $output->writeFieldEnd(); } - if ($this->o3 !== null) { - $xfer += $output->writeFieldBegin('o3', TType::STRUCT, 3); - $xfer += $this->o3->write($output); - $xfer += $output->writeFieldEnd(); - } $xfer += $output->writeFieldStop(); $xfer += $output->writeStructEnd(); return $xfer; @@ -10980,34 +14757,69 @@ class ThriftHiveMetastore_add_partition_with_environment_context_result { } -class ThriftHiveMetastore_add_partitions_args { +class ThriftHiveMetastore_exchange_partition_args { static $_TSPEC; - public $new_parts = null; + public $partitionSpecs = null; + public $source_db = null; + public $source_table_name = null; + public $dest_db = null; + public $dest_table_name = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 1 => array( - 'var' => 'new_parts', - 'type' => TType::LST, - 'etype' => TType::STRUCT, - 'elem' => array( - 'type' => TType::STRUCT, - 'class' => '\metastore\Partition', + 'var' => 'partitionSpecs', + 'type' => TType::MAP, + 'ktype' => TType::STRING, + 'vtype' => TType::STRING, + 'key' => array( + 'type' => TType::STRING, + ), + 'val' => array( + 'type' => TType::STRING, ), ), + 2 => array( + 'var' => 'source_db', + 'type' => TType::STRING, + ), + 3 => array( + 'var' => 'source_table_name', + 'type' => TType::STRING, + ), + 4 => array( + 'var' => 'dest_db', + 'type' => TType::STRING, + ), + 5 => array( + 'var' => 'dest_table_name', + 'type' => TType::STRING, + ), ); } if (is_array($vals)) { - if (isset($vals['new_parts'])) { - $this->new_parts = $vals['new_parts']; + if (isset($vals['partitionSpecs'])) { + $this->partitionSpecs = $vals['partitionSpecs']; + } + if (isset($vals['source_db'])) { + $this->source_db = $vals['source_db']; + } + if (isset($vals['source_table_name'])) { + $this->source_table_name = $vals['source_table_name']; + } + if (isset($vals['dest_db'])) { + $this->dest_db = $vals['dest_db']; + } + if (isset($vals['dest_table_name'])) { + $this->dest_table_name = $vals['dest_table_name']; } } } public function getName() { - return 'ThriftHiveMetastore_add_partitions_args'; + return 'ThriftHiveMetastore_exchange_partition_args'; } public function read($input) @@ -11026,19 +14838,49 @@ class ThriftHiveMetastore_add_partitions_args { switch ($fid) { case 1: - if ($ftype == TType::LST) { - $this->new_parts = array(); - $_size392 = 0; - $_etype395 = 0; - $xfer += $input->readListBegin($_etype395, $_size392); - for ($_i396 = 0; $_i396 < $_size392; ++$_i396) + if ($ftype == TType::MAP) { + $this->partitionSpecs = array(); + $_size477 = 0; + $_ktype478 = 0; + $_vtype479 = 0; + $xfer += $input->readMapBegin($_ktype478, $_vtype479, $_size477); + for ($_i481 = 0; $_i481 < $_size477; ++$_i481) { - $elem397 = null; - $elem397 = new \metastore\Partition(); - $xfer += $elem397->read($input); - $this->new_parts []= $elem397; + $key482 = ''; + $val483 = ''; + $xfer += $input->readString($key482); + $xfer += $input->readString($val483); + $this->partitionSpecs[$key482] = $val483; } - $xfer += $input->readListEnd(); + $xfer += $input->readMapEnd(); + } else { + $xfer += $input->skip($ftype); + } + break; + case 2: + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->source_db); + } else { + $xfer += $input->skip($ftype); + } + break; + case 3: + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->source_table_name); + } else { + $xfer += $input->skip($ftype); + } + break; + case 4: + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->dest_db); + } else { + $xfer += $input->skip($ftype); + } + break; + case 5: + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->dest_table_name); } else { $xfer += $input->skip($ftype); } @@ -11055,24 +14897,45 @@ class ThriftHiveMetastore_add_partitions_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_add_partitions_args'); - if ($this->new_parts !== null) { - if (!is_array($this->new_parts)) { + $xfer += $output->writeStructBegin('ThriftHiveMetastore_exchange_partition_args'); + if ($this->partitionSpecs !== null) { + if (!is_array($this->partitionSpecs)) { throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); } - $xfer += $output->writeFieldBegin('new_parts', TType::LST, 1); + $xfer += $output->writeFieldBegin('partitionSpecs', TType::MAP, 1); { - $output->writeListBegin(TType::STRUCT, count($this->new_parts)); + $output->writeMapBegin(TType::STRING, TType::STRING, count($this->partitionSpecs)); { - foreach ($this->new_parts as $iter398) + foreach ($this->partitionSpecs as $kiter484 => $viter485) { - $xfer += $iter398->write($output); + $xfer += $output->writeString($kiter484); + $xfer += $output->writeString($viter485); } } - $output->writeListEnd(); + $output->writeMapEnd(); } $xfer += $output->writeFieldEnd(); } + if ($this->source_db !== null) { + $xfer += $output->writeFieldBegin('source_db', TType::STRING, 2); + $xfer += $output->writeString($this->source_db); + $xfer += $output->writeFieldEnd(); + } + if ($this->source_table_name !== null) { + $xfer += $output->writeFieldBegin('source_table_name', TType::STRING, 3); + $xfer += $output->writeString($this->source_table_name); + $xfer += $output->writeFieldEnd(); + } + if ($this->dest_db !== null) { + $xfer += $output->writeFieldBegin('dest_db', TType::STRING, 4); + $xfer += $output->writeString($this->dest_db); + $xfer += $output->writeFieldEnd(); + } + if ($this->dest_table_name !== null) { + $xfer += $output->writeFieldBegin('dest_table_name', TType::STRING, 5); + $xfer += $output->writeString($this->dest_table_name); + $xfer += $output->writeFieldEnd(); + } $xfer += $output->writeFieldStop(); $xfer += $output->writeStructEnd(); return $xfer; @@ -11080,35 +14943,42 @@ class ThriftHiveMetastore_add_partitions_args { } -class ThriftHiveMetastore_add_partitions_result { +class ThriftHiveMetastore_exchange_partition_result { static $_TSPEC; public $success = null; public $o1 = null; public $o2 = null; public $o3 = null; + public $o4 = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 0 => array( 'var' => 'success', - 'type' => TType::I32, + 'type' => TType::STRUCT, + 'class' => '\metastore\Partition', ), 1 => array( 'var' => 'o1', 'type' => TType::STRUCT, - 'class' => '\metastore\InvalidObjectException', + 'class' => '\metastore\MetaException', ), 2 => array( 'var' => 'o2', 'type' => TType::STRUCT, - 'class' => '\metastore\AlreadyExistsException', + 'class' => '\metastore\NoSuchObjectException', ), 3 => array( 'var' => 'o3', 'type' => TType::STRUCT, - 'class' => '\metastore\MetaException', + 'class' => '\metastore\InvalidObjectException', + ), + 4 => array( + 'var' => 'o4', + 'type' => TType::STRUCT, + 'class' => '\metastore\InvalidInputException', ), ); } @@ -11125,11 +14995,14 @@ class ThriftHiveMetastore_add_partitions_result { if (isset($vals['o3'])) { $this->o3 = $vals['o3']; } + if (isset($vals['o4'])) { + $this->o4 = $vals['o4']; + } } } public function getName() { - return 'ThriftHiveMetastore_add_partitions_result'; + return 'ThriftHiveMetastore_exchange_partition_result'; } public function read($input) @@ -11148,15 +15021,16 @@ class ThriftHiveMetastore_add_partitions_result { switch ($fid) { case 0: - if ($ftype == TType::I32) { - $xfer += $input->readI32($this->success); + if ($ftype == TType::STRUCT) { + $this->success = new \metastore\Partition(); + $xfer += $this->success->read($input); } else { $xfer += $input->skip($ftype); } break; case 1: if ($ftype == TType::STRUCT) { - $this->o1 = new \metastore\InvalidObjectException(); + $this->o1 = new \metastore\MetaException(); $xfer += $this->o1->read($input); } else { $xfer += $input->skip($ftype); @@ -11164,7 +15038,7 @@ class ThriftHiveMetastore_add_partitions_result { break; case 2: if ($ftype == TType::STRUCT) { - $this->o2 = new \metastore\AlreadyExistsException(); + $this->o2 = new \metastore\NoSuchObjectException(); $xfer += $this->o2->read($input); } else { $xfer += $input->skip($ftype); @@ -11172,12 +15046,20 @@ class ThriftHiveMetastore_add_partitions_result { break; case 3: if ($ftype == TType::STRUCT) { - $this->o3 = new \metastore\MetaException(); + $this->o3 = new \metastore\InvalidObjectException(); $xfer += $this->o3->read($input); } else { $xfer += $input->skip($ftype); } break; + case 4: + if ($ftype == TType::STRUCT) { + $this->o4 = new \metastore\InvalidInputException(); + $xfer += $this->o4->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; default: $xfer += $input->skip($ftype); break; @@ -11190,10 +15072,13 @@ class ThriftHiveMetastore_add_partitions_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_add_partitions_result'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_exchange_partition_result'); if ($this->success !== null) { - $xfer += $output->writeFieldBegin('success', TType::I32, 0); - $xfer += $output->writeI32($this->success); + if (!is_object($this->success)) { + throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); + } + $xfer += $output->writeFieldBegin('success', TType::STRUCT, 0); + $xfer += $this->success->write($output); $xfer += $output->writeFieldEnd(); } if ($this->o1 !== null) { @@ -11211,6 +15096,11 @@ class ThriftHiveMetastore_add_partitions_result { $xfer += $this->o3->write($output); $xfer += $output->writeFieldEnd(); } + if ($this->o4 !== null) { + $xfer += $output->writeFieldBegin('o4', TType::STRUCT, 4); + $xfer += $this->o4->write($output); + $xfer += $output->writeFieldEnd(); + } $xfer += $output->writeFieldStop(); $xfer += $output->writeStructEnd(); return $xfer; @@ -11218,12 +15108,14 @@ class ThriftHiveMetastore_add_partitions_result { } -class ThriftHiveMetastore_append_partition_args { +class ThriftHiveMetastore_get_partition_with_auth_args { static $_TSPEC; public $db_name = null; public $tbl_name = null; public $part_vals = null; + public $user_name = null; + public $group_names = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { @@ -11244,6 +15136,18 @@ class ThriftHiveMetastore_append_partition_args { 'type' => TType::STRING, ), ), + 4 => array( + 'var' => 'user_name', + 'type' => TType::STRING, + ), + 5 => array( + 'var' => 'group_names', + 'type' => TType::LST, + 'etype' => TType::STRING, + 'elem' => array( + 'type' => TType::STRING, + ), + ), ); } if (is_array($vals)) { @@ -11256,11 +15160,17 @@ class ThriftHiveMetastore_append_partition_args { if (isset($vals['part_vals'])) { $this->part_vals = $vals['part_vals']; } + if (isset($vals['user_name'])) { + $this->user_name = $vals['user_name']; + } + if (isset($vals['group_names'])) { + $this->group_names = $vals['group_names']; + } } } public function getName() { - return 'ThriftHiveMetastore_append_partition_args'; + return 'ThriftHiveMetastore_get_partition_with_auth_args'; } public function read($input) @@ -11295,14 +15205,38 @@ class ThriftHiveMetastore_append_partition_args { case 3: if ($ftype == TType::LST) { $this->part_vals = array(); - $_size399 = 0; - $_etype402 = 0; - $xfer += $input->readListBegin($_etype402, $_size399); - for ($_i403 = 0; $_i403 < $_size399; ++$_i403) + $_size486 = 0; + $_etype489 = 0; + $xfer += $input->readListBegin($_etype489, $_size486); + for ($_i490 = 0; $_i490 < $_size486; ++$_i490) + { + $elem491 = null; + $xfer += $input->readString($elem491); + $this->part_vals []= $elem491; + } + $xfer += $input->readListEnd(); + } else { + $xfer += $input->skip($ftype); + } + break; + case 4: + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->user_name); + } else { + $xfer += $input->skip($ftype); + } + break; + case 5: + if ($ftype == TType::LST) { + $this->group_names = array(); + $_size492 = 0; + $_etype495 = 0; + $xfer += $input->readListBegin($_etype495, $_size492); + for ($_i496 = 0; $_i496 < $_size492; ++$_i496) { - $elem404 = null; - $xfer += $input->readString($elem404); - $this->part_vals []= $elem404; + $elem497 = null; + $xfer += $input->readString($elem497); + $this->group_names []= $elem497; } $xfer += $input->readListEnd(); } else { @@ -11321,7 +15255,7 @@ class ThriftHiveMetastore_append_partition_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_append_partition_args'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_partition_with_auth_args'); if ($this->db_name !== null) { $xfer += $output->writeFieldBegin('db_name', TType::STRING, 1); $xfer += $output->writeString($this->db_name); @@ -11340,9 +15274,31 @@ class ThriftHiveMetastore_append_partition_args { { $output->writeListBegin(TType::STRING, count($this->part_vals)); { - foreach ($this->part_vals as $iter405) + foreach ($this->part_vals as $iter498) + { + $xfer += $output->writeString($iter498); + } + } + $output->writeListEnd(); + } + $xfer += $output->writeFieldEnd(); + } + if ($this->user_name !== null) { + $xfer += $output->writeFieldBegin('user_name', TType::STRING, 4); + $xfer += $output->writeString($this->user_name); + $xfer += $output->writeFieldEnd(); + } + if ($this->group_names !== null) { + if (!is_array($this->group_names)) { + throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); + } + $xfer += $output->writeFieldBegin('group_names', TType::LST, 5); + { + $output->writeListBegin(TType::STRING, count($this->group_names)); + { + foreach ($this->group_names as $iter499) { - $xfer += $output->writeString($iter405); + $xfer += $output->writeString($iter499); } } $output->writeListEnd(); @@ -11356,13 +15312,12 @@ class ThriftHiveMetastore_append_partition_args { } -class ThriftHiveMetastore_append_partition_result { +class ThriftHiveMetastore_get_partition_with_auth_result { static $_TSPEC; public $success = null; public $o1 = null; public $o2 = null; - public $o3 = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { @@ -11375,17 +15330,12 @@ class ThriftHiveMetastore_append_partition_result { 1 => array( 'var' => 'o1', 'type' => TType::STRUCT, - 'class' => '\metastore\InvalidObjectException', + 'class' => '\metastore\MetaException', ), 2 => array( 'var' => 'o2', 'type' => TType::STRUCT, - 'class' => '\metastore\AlreadyExistsException', - ), - 3 => array( - 'var' => 'o3', - 'type' => TType::STRUCT, - 'class' => '\metastore\MetaException', + 'class' => '\metastore\NoSuchObjectException', ), ); } @@ -11399,14 +15349,11 @@ class ThriftHiveMetastore_append_partition_result { if (isset($vals['o2'])) { $this->o2 = $vals['o2']; } - if (isset($vals['o3'])) { - $this->o3 = $vals['o3']; - } } } public function getName() { - return 'ThriftHiveMetastore_append_partition_result'; + return 'ThriftHiveMetastore_get_partition_with_auth_result'; } public function read($input) @@ -11434,7 +15381,7 @@ class ThriftHiveMetastore_append_partition_result { break; case 1: if ($ftype == TType::STRUCT) { - $this->o1 = new \metastore\InvalidObjectException(); + $this->o1 = new \metastore\MetaException(); $xfer += $this->o1->read($input); } else { $xfer += $input->skip($ftype); @@ -11442,20 +15389,12 @@ class ThriftHiveMetastore_append_partition_result { break; case 2: if ($ftype == TType::STRUCT) { - $this->o2 = new \metastore\AlreadyExistsException(); + $this->o2 = new \metastore\NoSuchObjectException(); $xfer += $this->o2->read($input); } else { $xfer += $input->skip($ftype); } break; - case 3: - if ($ftype == TType::STRUCT) { - $this->o3 = new \metastore\MetaException(); - $xfer += $this->o3->read($input); - } else { - $xfer += $input->skip($ftype); - } - break; default: $xfer += $input->skip($ftype); break; @@ -11468,7 +15407,7 @@ class ThriftHiveMetastore_append_partition_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_append_partition_result'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_partition_with_auth_result'); if ($this->success !== null) { if (!is_object($this->success)) { throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); @@ -11487,11 +15426,6 @@ class ThriftHiveMetastore_append_partition_result { $xfer += $this->o2->write($output); $xfer += $output->writeFieldEnd(); } - if ($this->o3 !== null) { - $xfer += $output->writeFieldBegin('o3', TType::STRUCT, 3); - $xfer += $this->o3->write($output); - $xfer += $output->writeFieldEnd(); - } $xfer += $output->writeFieldStop(); $xfer += $output->writeStructEnd(); return $xfer; @@ -11499,30 +15433,45 @@ class ThriftHiveMetastore_append_partition_result { } -class ThriftHiveMetastore_add_partitions_req_args { +class ThriftHiveMetastore_get_partition_by_name_args { static $_TSPEC; - public $request = null; + public $db_name = null; + public $tbl_name = null; + public $part_name = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 1 => array( - 'var' => 'request', - 'type' => TType::STRUCT, - 'class' => '\metastore\AddPartitionsRequest', + 'var' => 'db_name', + 'type' => TType::STRING, + ), + 2 => array( + 'var' => 'tbl_name', + 'type' => TType::STRING, + ), + 3 => array( + 'var' => 'part_name', + 'type' => TType::STRING, ), ); } if (is_array($vals)) { - if (isset($vals['request'])) { - $this->request = $vals['request']; + if (isset($vals['db_name'])) { + $this->db_name = $vals['db_name']; + } + if (isset($vals['tbl_name'])) { + $this->tbl_name = $vals['tbl_name']; + } + if (isset($vals['part_name'])) { + $this->part_name = $vals['part_name']; } } } public function getName() { - return 'ThriftHiveMetastore_add_partitions_req_args'; + return 'ThriftHiveMetastore_get_partition_by_name_args'; } public function read($input) @@ -11541,9 +15490,22 @@ class ThriftHiveMetastore_add_partitions_req_args { switch ($fid) { case 1: - if ($ftype == TType::STRUCT) { - $this->request = new \metastore\AddPartitionsRequest(); - $xfer += $this->request->read($input); + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->db_name); + } else { + $xfer += $input->skip($ftype); + } + break; + case 2: + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->tbl_name); + } else { + $xfer += $input->skip($ftype); + } + break; + case 3: + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->part_name); } else { $xfer += $input->skip($ftype); } @@ -11560,13 +15522,20 @@ class ThriftHiveMetastore_add_partitions_req_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_add_partitions_req_args'); - if ($this->request !== null) { - if (!is_object($this->request)) { - throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); - } - $xfer += $output->writeFieldBegin('request', TType::STRUCT, 1); - $xfer += $this->request->write($output); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_partition_by_name_args'); + if ($this->db_name !== null) { + $xfer += $output->writeFieldBegin('db_name', TType::STRING, 1); + $xfer += $output->writeString($this->db_name); + $xfer += $output->writeFieldEnd(); + } + if ($this->tbl_name !== null) { + $xfer += $output->writeFieldBegin('tbl_name', TType::STRING, 2); + $xfer += $output->writeString($this->tbl_name); + $xfer += $output->writeFieldEnd(); + } + if ($this->part_name !== null) { + $xfer += $output->writeFieldBegin('part_name', TType::STRING, 3); + $xfer += $output->writeString($this->part_name); $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -11576,13 +15545,12 @@ class ThriftHiveMetastore_add_partitions_req_args { } -class ThriftHiveMetastore_add_partitions_req_result { +class ThriftHiveMetastore_get_partition_by_name_result { static $_TSPEC; public $success = null; public $o1 = null; public $o2 = null; - public $o3 = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { @@ -11590,22 +15558,17 @@ class ThriftHiveMetastore_add_partitions_req_result { 0 => array( 'var' => 'success', 'type' => TType::STRUCT, - 'class' => '\metastore\AddPartitionsResult', + 'class' => '\metastore\Partition', ), 1 => array( 'var' => 'o1', 'type' => TType::STRUCT, - 'class' => '\metastore\InvalidObjectException', + 'class' => '\metastore\MetaException', ), 2 => array( 'var' => 'o2', 'type' => TType::STRUCT, - 'class' => '\metastore\AlreadyExistsException', - ), - 3 => array( - 'var' => 'o3', - 'type' => TType::STRUCT, - 'class' => '\metastore\MetaException', + 'class' => '\metastore\NoSuchObjectException', ), ); } @@ -11619,14 +15582,11 @@ class ThriftHiveMetastore_add_partitions_req_result { if (isset($vals['o2'])) { $this->o2 = $vals['o2']; } - if (isset($vals['o3'])) { - $this->o3 = $vals['o3']; - } } } public function getName() { - return 'ThriftHiveMetastore_add_partitions_req_result'; + return 'ThriftHiveMetastore_get_partition_by_name_result'; } public function read($input) @@ -11646,7 +15606,7 @@ class ThriftHiveMetastore_add_partitions_req_result { { case 0: if ($ftype == TType::STRUCT) { - $this->success = new \metastore\AddPartitionsResult(); + $this->success = new \metastore\Partition(); $xfer += $this->success->read($input); } else { $xfer += $input->skip($ftype); @@ -11654,7 +15614,7 @@ class ThriftHiveMetastore_add_partitions_req_result { break; case 1: if ($ftype == TType::STRUCT) { - $this->o1 = new \metastore\InvalidObjectException(); + $this->o1 = new \metastore\MetaException(); $xfer += $this->o1->read($input); } else { $xfer += $input->skip($ftype); @@ -11662,20 +15622,12 @@ class ThriftHiveMetastore_add_partitions_req_result { break; case 2: if ($ftype == TType::STRUCT) { - $this->o2 = new \metastore\AlreadyExistsException(); + $this->o2 = new \metastore\NoSuchObjectException(); $xfer += $this->o2->read($input); } else { $xfer += $input->skip($ftype); } break; - case 3: - if ($ftype == TType::STRUCT) { - $this->o3 = new \metastore\MetaException(); - $xfer += $this->o3->read($input); - } else { - $xfer += $input->skip($ftype); - } - break; default: $xfer += $input->skip($ftype); break; @@ -11688,7 +15640,7 @@ class ThriftHiveMetastore_add_partitions_req_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_add_partitions_req_result'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_partition_by_name_result'); if ($this->success !== null) { if (!is_object($this->success)) { throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); @@ -11707,11 +15659,6 @@ class ThriftHiveMetastore_add_partitions_req_result { $xfer += $this->o2->write($output); $xfer += $output->writeFieldEnd(); } - if ($this->o3 !== null) { - $xfer += $output->writeFieldBegin('o3', TType::STRUCT, 3); - $xfer += $this->o3->write($output); - $xfer += $output->writeFieldEnd(); - } $xfer += $output->writeFieldStop(); $xfer += $output->writeStructEnd(); return $xfer; @@ -11719,13 +15666,12 @@ class ThriftHiveMetastore_add_partitions_req_result { } -class ThriftHiveMetastore_append_partition_with_environment_context_args { +class ThriftHiveMetastore_get_partitions_args { static $_TSPEC; public $db_name = null; public $tbl_name = null; - public $part_vals = null; - public $environment_context = null; + public $max_parts = -1; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { @@ -11739,17 +15685,8 @@ class ThriftHiveMetastore_append_partition_with_environment_context_args { 'type' => TType::STRING, ), 3 => array( - 'var' => 'part_vals', - 'type' => TType::LST, - 'etype' => TType::STRING, - 'elem' => array( - 'type' => TType::STRING, - ), - ), - 4 => array( - 'var' => 'environment_context', - 'type' => TType::STRUCT, - 'class' => '\metastore\EnvironmentContext', + 'var' => 'max_parts', + 'type' => TType::I16, ), ); } @@ -11760,17 +15697,14 @@ class ThriftHiveMetastore_append_partition_with_environment_context_args { if (isset($vals['tbl_name'])) { $this->tbl_name = $vals['tbl_name']; } - if (isset($vals['part_vals'])) { - $this->part_vals = $vals['part_vals']; - } - if (isset($vals['environment_context'])) { - $this->environment_context = $vals['environment_context']; + if (isset($vals['max_parts'])) { + $this->max_parts = $vals['max_parts']; } } } public function getName() { - return 'ThriftHiveMetastore_append_partition_with_environment_context_args'; + return 'ThriftHiveMetastore_get_partitions_args'; } public function read($input) @@ -11803,26 +15737,8 @@ class ThriftHiveMetastore_append_partition_with_environment_context_args { } break; case 3: - if ($ftype == TType::LST) { - $this->part_vals = array(); - $_size406 = 0; - $_etype409 = 0; - $xfer += $input->readListBegin($_etype409, $_size406); - for ($_i410 = 0; $_i410 < $_size406; ++$_i410) - { - $elem411 = null; - $xfer += $input->readString($elem411); - $this->part_vals []= $elem411; - } - $xfer += $input->readListEnd(); - } else { - $xfer += $input->skip($ftype); - } - break; - case 4: - if ($ftype == TType::STRUCT) { - $this->environment_context = new \metastore\EnvironmentContext(); - $xfer += $this->environment_context->read($input); + if ($ftype == TType::I16) { + $xfer += $input->readI16($this->max_parts); } else { $xfer += $input->skip($ftype); } @@ -11839,7 +15755,7 @@ class ThriftHiveMetastore_append_partition_with_environment_context_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_append_partition_with_environment_context_args'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_partitions_args'); if ($this->db_name !== null) { $xfer += $output->writeFieldBegin('db_name', TType::STRING, 1); $xfer += $output->writeString($this->db_name); @@ -11850,29 +15766,9 @@ class ThriftHiveMetastore_append_partition_with_environment_context_args { $xfer += $output->writeString($this->tbl_name); $xfer += $output->writeFieldEnd(); } - if ($this->part_vals !== null) { - if (!is_array($this->part_vals)) { - throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); - } - $xfer += $output->writeFieldBegin('part_vals', TType::LST, 3); - { - $output->writeListBegin(TType::STRING, count($this->part_vals)); - { - foreach ($this->part_vals as $iter412) - { - $xfer += $output->writeString($iter412); - } - } - $output->writeListEnd(); - } - $xfer += $output->writeFieldEnd(); - } - if ($this->environment_context !== null) { - if (!is_object($this->environment_context)) { - throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); - } - $xfer += $output->writeFieldBegin('environment_context', TType::STRUCT, 4); - $xfer += $this->environment_context->write($output); + if ($this->max_parts !== null) { + $xfer += $output->writeFieldBegin('max_parts', TType::I16, 3); + $xfer += $output->writeI16($this->max_parts); $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -11882,35 +15778,33 @@ class ThriftHiveMetastore_append_partition_with_environment_context_args { } -class ThriftHiveMetastore_append_partition_with_environment_context_result { +class ThriftHiveMetastore_get_partitions_result { static $_TSPEC; public $success = null; public $o1 = null; public $o2 = null; - public $o3 = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 0 => array( 'var' => 'success', - 'type' => TType::STRUCT, - 'class' => '\metastore\Partition', + 'type' => TType::LST, + 'etype' => TType::STRUCT, + 'elem' => array( + 'type' => TType::STRUCT, + 'class' => '\metastore\Partition', + ), ), 1 => array( 'var' => 'o1', 'type' => TType::STRUCT, - 'class' => '\metastore\InvalidObjectException', + 'class' => '\metastore\NoSuchObjectException', ), 2 => array( 'var' => 'o2', 'type' => TType::STRUCT, - 'class' => '\metastore\AlreadyExistsException', - ), - 3 => array( - 'var' => 'o3', - 'type' => TType::STRUCT, 'class' => '\metastore\MetaException', ), ); @@ -11925,14 +15819,11 @@ class ThriftHiveMetastore_append_partition_with_environment_context_result { if (isset($vals['o2'])) { $this->o2 = $vals['o2']; } - if (isset($vals['o3'])) { - $this->o3 = $vals['o3']; - } } } public function getName() { - return 'ThriftHiveMetastore_append_partition_with_environment_context_result'; + return 'ThriftHiveMetastore_get_partitions_result'; } public function read($input) @@ -11951,16 +15842,26 @@ class ThriftHiveMetastore_append_partition_with_environment_context_result { switch ($fid) { case 0: - if ($ftype == TType::STRUCT) { - $this->success = new \metastore\Partition(); - $xfer += $this->success->read($input); + if ($ftype == TType::LST) { + $this->success = array(); + $_size500 = 0; + $_etype503 = 0; + $xfer += $input->readListBegin($_etype503, $_size500); + for ($_i504 = 0; $_i504 < $_size500; ++$_i504) + { + $elem505 = null; + $elem505 = new \metastore\Partition(); + $xfer += $elem505->read($input); + $this->success []= $elem505; + } + $xfer += $input->readListEnd(); } else { $xfer += $input->skip($ftype); } break; case 1: if ($ftype == TType::STRUCT) { - $this->o1 = new \metastore\InvalidObjectException(); + $this->o1 = new \metastore\NoSuchObjectException(); $xfer += $this->o1->read($input); } else { $xfer += $input->skip($ftype); @@ -11968,20 +15869,12 @@ class ThriftHiveMetastore_append_partition_with_environment_context_result { break; case 2: if ($ftype == TType::STRUCT) { - $this->o2 = new \metastore\AlreadyExistsException(); + $this->o2 = new \metastore\MetaException(); $xfer += $this->o2->read($input); } else { $xfer += $input->skip($ftype); } break; - case 3: - if ($ftype == TType::STRUCT) { - $this->o3 = new \metastore\MetaException(); - $xfer += $this->o3->read($input); - } else { - $xfer += $input->skip($ftype); - } - break; default: $xfer += $input->skip($ftype); break; @@ -11994,13 +15887,22 @@ class ThriftHiveMetastore_append_partition_with_environment_context_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_append_partition_with_environment_context_result'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_partitions_result'); if ($this->success !== null) { - if (!is_object($this->success)) { + if (!is_array($this->success)) { throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); } - $xfer += $output->writeFieldBegin('success', TType::STRUCT, 0); - $xfer += $this->success->write($output); + $xfer += $output->writeFieldBegin('success', TType::LST, 0); + { + $output->writeListBegin(TType::STRUCT, count($this->success)); + { + foreach ($this->success as $iter506) + { + $xfer += $iter506->write($output); + } + } + $output->writeListEnd(); + } $xfer += $output->writeFieldEnd(); } if ($this->o1 !== null) { @@ -12013,11 +15915,6 @@ class ThriftHiveMetastore_append_partition_with_environment_context_result { $xfer += $this->o2->write($output); $xfer += $output->writeFieldEnd(); } - if ($this->o3 !== null) { - $xfer += $output->writeFieldBegin('o3', TType::STRUCT, 3); - $xfer += $this->o3->write($output); - $xfer += $output->writeFieldEnd(); - } $xfer += $output->writeFieldStop(); $xfer += $output->writeStructEnd(); return $xfer; @@ -12025,12 +15922,14 @@ class ThriftHiveMetastore_append_partition_with_environment_context_result { } -class ThriftHiveMetastore_append_partition_by_name_args { +class ThriftHiveMetastore_get_partitions_with_auth_args { static $_TSPEC; public $db_name = null; public $tbl_name = null; - public $part_name = null; + public $max_parts = -1; + public $user_name = null; + public $group_names = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { @@ -12044,9 +15943,21 @@ class ThriftHiveMetastore_append_partition_by_name_args { 'type' => TType::STRING, ), 3 => array( - 'var' => 'part_name', + 'var' => 'max_parts', + 'type' => TType::I16, + ), + 4 => array( + 'var' => 'user_name', 'type' => TType::STRING, ), + 5 => array( + 'var' => 'group_names', + 'type' => TType::LST, + 'etype' => TType::STRING, + 'elem' => array( + 'type' => TType::STRING, + ), + ), ); } if (is_array($vals)) { @@ -12056,14 +15967,20 @@ class ThriftHiveMetastore_append_partition_by_name_args { if (isset($vals['tbl_name'])) { $this->tbl_name = $vals['tbl_name']; } - if (isset($vals['part_name'])) { - $this->part_name = $vals['part_name']; + if (isset($vals['max_parts'])) { + $this->max_parts = $vals['max_parts']; + } + if (isset($vals['user_name'])) { + $this->user_name = $vals['user_name']; + } + if (isset($vals['group_names'])) { + $this->group_names = $vals['group_names']; } } } public function getName() { - return 'ThriftHiveMetastore_append_partition_by_name_args'; + return 'ThriftHiveMetastore_get_partitions_with_auth_args'; } public function read($input) @@ -12096,8 +16013,32 @@ class ThriftHiveMetastore_append_partition_by_name_args { } break; case 3: + if ($ftype == TType::I16) { + $xfer += $input->readI16($this->max_parts); + } else { + $xfer += $input->skip($ftype); + } + break; + case 4: if ($ftype == TType::STRING) { - $xfer += $input->readString($this->part_name); + $xfer += $input->readString($this->user_name); + } else { + $xfer += $input->skip($ftype); + } + break; + case 5: + if ($ftype == TType::LST) { + $this->group_names = array(); + $_size507 = 0; + $_etype510 = 0; + $xfer += $input->readListBegin($_etype510, $_size507); + for ($_i511 = 0; $_i511 < $_size507; ++$_i511) + { + $elem512 = null; + $xfer += $input->readString($elem512); + $this->group_names []= $elem512; + } + $xfer += $input->readListEnd(); } else { $xfer += $input->skip($ftype); } @@ -12114,7 +16055,7 @@ class ThriftHiveMetastore_append_partition_by_name_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_append_partition_by_name_args'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_partitions_with_auth_args'); if ($this->db_name !== null) { $xfer += $output->writeFieldBegin('db_name', TType::STRING, 1); $xfer += $output->writeString($this->db_name); @@ -12125,9 +16066,31 @@ class ThriftHiveMetastore_append_partition_by_name_args { $xfer += $output->writeString($this->tbl_name); $xfer += $output->writeFieldEnd(); } - if ($this->part_name !== null) { - $xfer += $output->writeFieldBegin('part_name', TType::STRING, 3); - $xfer += $output->writeString($this->part_name); + if ($this->max_parts !== null) { + $xfer += $output->writeFieldBegin('max_parts', TType::I16, 3); + $xfer += $output->writeI16($this->max_parts); + $xfer += $output->writeFieldEnd(); + } + if ($this->user_name !== null) { + $xfer += $output->writeFieldBegin('user_name', TType::STRING, 4); + $xfer += $output->writeString($this->user_name); + $xfer += $output->writeFieldEnd(); + } + if ($this->group_names !== null) { + if (!is_array($this->group_names)) { + throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); + } + $xfer += $output->writeFieldBegin('group_names', TType::LST, 5); + { + $output->writeListBegin(TType::STRING, count($this->group_names)); + { + foreach ($this->group_names as $iter513) + { + $xfer += $output->writeString($iter513); + } + } + $output->writeListEnd(); + } $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -12137,35 +16100,33 @@ class ThriftHiveMetastore_append_partition_by_name_args { } -class ThriftHiveMetastore_append_partition_by_name_result { +class ThriftHiveMetastore_get_partitions_with_auth_result { static $_TSPEC; public $success = null; public $o1 = null; public $o2 = null; - public $o3 = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 0 => array( 'var' => 'success', - 'type' => TType::STRUCT, - 'class' => '\metastore\Partition', + 'type' => TType::LST, + 'etype' => TType::STRUCT, + 'elem' => array( + 'type' => TType::STRUCT, + 'class' => '\metastore\Partition', + ), ), 1 => array( 'var' => 'o1', 'type' => TType::STRUCT, - 'class' => '\metastore\InvalidObjectException', + 'class' => '\metastore\NoSuchObjectException', ), 2 => array( 'var' => 'o2', 'type' => TType::STRUCT, - 'class' => '\metastore\AlreadyExistsException', - ), - 3 => array( - 'var' => 'o3', - 'type' => TType::STRUCT, 'class' => '\metastore\MetaException', ), ); @@ -12180,14 +16141,11 @@ class ThriftHiveMetastore_append_partition_by_name_result { if (isset($vals['o2'])) { $this->o2 = $vals['o2']; } - if (isset($vals['o3'])) { - $this->o3 = $vals['o3']; - } } } public function getName() { - return 'ThriftHiveMetastore_append_partition_by_name_result'; + return 'ThriftHiveMetastore_get_partitions_with_auth_result'; } public function read($input) @@ -12206,16 +16164,26 @@ class ThriftHiveMetastore_append_partition_by_name_result { switch ($fid) { case 0: - if ($ftype == TType::STRUCT) { - $this->success = new \metastore\Partition(); - $xfer += $this->success->read($input); + if ($ftype == TType::LST) { + $this->success = array(); + $_size514 = 0; + $_etype517 = 0; + $xfer += $input->readListBegin($_etype517, $_size514); + for ($_i518 = 0; $_i518 < $_size514; ++$_i518) + { + $elem519 = null; + $elem519 = new \metastore\Partition(); + $xfer += $elem519->read($input); + $this->success []= $elem519; + } + $xfer += $input->readListEnd(); } else { $xfer += $input->skip($ftype); } break; case 1: if ($ftype == TType::STRUCT) { - $this->o1 = new \metastore\InvalidObjectException(); + $this->o1 = new \metastore\NoSuchObjectException(); $xfer += $this->o1->read($input); } else { $xfer += $input->skip($ftype); @@ -12223,20 +16191,12 @@ class ThriftHiveMetastore_append_partition_by_name_result { break; case 2: if ($ftype == TType::STRUCT) { - $this->o2 = new \metastore\AlreadyExistsException(); + $this->o2 = new \metastore\MetaException(); $xfer += $this->o2->read($input); } else { $xfer += $input->skip($ftype); } break; - case 3: - if ($ftype == TType::STRUCT) { - $this->o3 = new \metastore\MetaException(); - $xfer += $this->o3->read($input); - } else { - $xfer += $input->skip($ftype); - } - break; default: $xfer += $input->skip($ftype); break; @@ -12249,13 +16209,22 @@ class ThriftHiveMetastore_append_partition_by_name_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_append_partition_by_name_result'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_partitions_with_auth_result'); if ($this->success !== null) { - if (!is_object($this->success)) { + if (!is_array($this->success)) { throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); } - $xfer += $output->writeFieldBegin('success', TType::STRUCT, 0); - $xfer += $this->success->write($output); + $xfer += $output->writeFieldBegin('success', TType::LST, 0); + { + $output->writeListBegin(TType::STRUCT, count($this->success)); + { + foreach ($this->success as $iter520) + { + $xfer += $iter520->write($output); + } + } + $output->writeListEnd(); + } $xfer += $output->writeFieldEnd(); } if ($this->o1 !== null) { @@ -12268,11 +16237,6 @@ class ThriftHiveMetastore_append_partition_by_name_result { $xfer += $this->o2->write($output); $xfer += $output->writeFieldEnd(); } - if ($this->o3 !== null) { - $xfer += $output->writeFieldBegin('o3', TType::STRUCT, 3); - $xfer += $this->o3->write($output); - $xfer += $output->writeFieldEnd(); - } $xfer += $output->writeFieldStop(); $xfer += $output->writeStructEnd(); return $xfer; @@ -12280,13 +16244,12 @@ class ThriftHiveMetastore_append_partition_by_name_result { } -class ThriftHiveMetastore_append_partition_by_name_with_environment_context_args { +class ThriftHiveMetastore_get_partition_names_args { static $_TSPEC; public $db_name = null; public $tbl_name = null; - public $part_name = null; - public $environment_context = null; + public $max_parts = -1; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { @@ -12300,13 +16263,8 @@ class ThriftHiveMetastore_append_partition_by_name_with_environment_context_args 'type' => TType::STRING, ), 3 => array( - 'var' => 'part_name', - 'type' => TType::STRING, - ), - 4 => array( - 'var' => 'environment_context', - 'type' => TType::STRUCT, - 'class' => '\metastore\EnvironmentContext', + 'var' => 'max_parts', + 'type' => TType::I16, ), ); } @@ -12317,17 +16275,14 @@ class ThriftHiveMetastore_append_partition_by_name_with_environment_context_args if (isset($vals['tbl_name'])) { $this->tbl_name = $vals['tbl_name']; } - if (isset($vals['part_name'])) { - $this->part_name = $vals['part_name']; - } - if (isset($vals['environment_context'])) { - $this->environment_context = $vals['environment_context']; + if (isset($vals['max_parts'])) { + $this->max_parts = $vals['max_parts']; } } } public function getName() { - return 'ThriftHiveMetastore_append_partition_by_name_with_environment_context_args'; + return 'ThriftHiveMetastore_get_partition_names_args'; } public function read($input) @@ -12360,16 +16315,8 @@ class ThriftHiveMetastore_append_partition_by_name_with_environment_context_args } break; case 3: - if ($ftype == TType::STRING) { - $xfer += $input->readString($this->part_name); - } else { - $xfer += $input->skip($ftype); - } - break; - case 4: - if ($ftype == TType::STRUCT) { - $this->environment_context = new \metastore\EnvironmentContext(); - $xfer += $this->environment_context->read($input); + if ($ftype == TType::I16) { + $xfer += $input->readI16($this->max_parts); } else { $xfer += $input->skip($ftype); } @@ -12386,7 +16333,7 @@ class ThriftHiveMetastore_append_partition_by_name_with_environment_context_args public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_append_partition_by_name_with_environment_context_args'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_partition_names_args'); if ($this->db_name !== null) { $xfer += $output->writeFieldBegin('db_name', TType::STRING, 1); $xfer += $output->writeString($this->db_name); @@ -12397,17 +16344,9 @@ class ThriftHiveMetastore_append_partition_by_name_with_environment_context_args $xfer += $output->writeString($this->tbl_name); $xfer += $output->writeFieldEnd(); } - if ($this->part_name !== null) { - $xfer += $output->writeFieldBegin('part_name', TType::STRING, 3); - $xfer += $output->writeString($this->part_name); - $xfer += $output->writeFieldEnd(); - } - if ($this->environment_context !== null) { - if (!is_object($this->environment_context)) { - throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); - } - $xfer += $output->writeFieldBegin('environment_context', TType::STRUCT, 4); - $xfer += $this->environment_context->write($output); + if ($this->max_parts !== null) { + $xfer += $output->writeFieldBegin('max_parts', TType::I16, 3); + $xfer += $output->writeI16($this->max_parts); $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -12417,35 +16356,26 @@ class ThriftHiveMetastore_append_partition_by_name_with_environment_context_args } -class ThriftHiveMetastore_append_partition_by_name_with_environment_context_result { +class ThriftHiveMetastore_get_partition_names_result { static $_TSPEC; public $success = null; - public $o1 = null; public $o2 = null; - public $o3 = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 0 => array( 'var' => 'success', - 'type' => TType::STRUCT, - 'class' => '\metastore\Partition', + 'type' => TType::LST, + 'etype' => TType::STRING, + 'elem' => array( + 'type' => TType::STRING, + ), ), 1 => array( - 'var' => 'o1', - 'type' => TType::STRUCT, - 'class' => '\metastore\InvalidObjectException', - ), - 2 => array( 'var' => 'o2', 'type' => TType::STRUCT, - 'class' => '\metastore\AlreadyExistsException', - ), - 3 => array( - 'var' => 'o3', - 'type' => TType::STRUCT, 'class' => '\metastore\MetaException', ), ); @@ -12454,20 +16384,14 @@ class ThriftHiveMetastore_append_partition_by_name_with_environment_context_resu if (isset($vals['success'])) { $this->success = $vals['success']; } - if (isset($vals['o1'])) { - $this->o1 = $vals['o1']; - } if (isset($vals['o2'])) { $this->o2 = $vals['o2']; } - if (isset($vals['o3'])) { - $this->o3 = $vals['o3']; - } } } public function getName() { - return 'ThriftHiveMetastore_append_partition_by_name_with_environment_context_result'; + return 'ThriftHiveMetastore_get_partition_names_result'; } public function read($input) @@ -12486,37 +16410,30 @@ class ThriftHiveMetastore_append_partition_by_name_with_environment_context_resu switch ($fid) { case 0: - if ($ftype == TType::STRUCT) { - $this->success = new \metastore\Partition(); - $xfer += $this->success->read($input); + if ($ftype == TType::LST) { + $this->success = array(); + $_size521 = 0; + $_etype524 = 0; + $xfer += $input->readListBegin($_etype524, $_size521); + for ($_i525 = 0; $_i525 < $_size521; ++$_i525) + { + $elem526 = null; + $xfer += $input->readString($elem526); + $this->success []= $elem526; + } + $xfer += $input->readListEnd(); } else { $xfer += $input->skip($ftype); } break; case 1: if ($ftype == TType::STRUCT) { - $this->o1 = new \metastore\InvalidObjectException(); - $xfer += $this->o1->read($input); - } else { - $xfer += $input->skip($ftype); - } - break; - case 2: - if ($ftype == TType::STRUCT) { - $this->o2 = new \metastore\AlreadyExistsException(); + $this->o2 = new \metastore\MetaException(); $xfer += $this->o2->read($input); } else { $xfer += $input->skip($ftype); } break; - case 3: - if ($ftype == TType::STRUCT) { - $this->o3 = new \metastore\MetaException(); - $xfer += $this->o3->read($input); - } else { - $xfer += $input->skip($ftype); - } - break; default: $xfer += $input->skip($ftype); break; @@ -12529,30 +16446,29 @@ class ThriftHiveMetastore_append_partition_by_name_with_environment_context_resu public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_append_partition_by_name_with_environment_context_result'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_partition_names_result'); if ($this->success !== null) { - if (!is_object($this->success)) { + if (!is_array($this->success)) { throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); } - $xfer += $output->writeFieldBegin('success', TType::STRUCT, 0); - $xfer += $this->success->write($output); - $xfer += $output->writeFieldEnd(); - } - if ($this->o1 !== null) { - $xfer += $output->writeFieldBegin('o1', TType::STRUCT, 1); - $xfer += $this->o1->write($output); + $xfer += $output->writeFieldBegin('success', TType::LST, 0); + { + $output->writeListBegin(TType::STRING, count($this->success)); + { + foreach ($this->success as $iter527) + { + $xfer += $output->writeString($iter527); + } + } + $output->writeListEnd(); + } $xfer += $output->writeFieldEnd(); } if ($this->o2 !== null) { - $xfer += $output->writeFieldBegin('o2', TType::STRUCT, 2); + $xfer += $output->writeFieldBegin('o2', TType::STRUCT, 1); $xfer += $this->o2->write($output); $xfer += $output->writeFieldEnd(); } - if ($this->o3 !== null) { - $xfer += $output->writeFieldBegin('o3', TType::STRUCT, 3); - $xfer += $this->o3->write($output); - $xfer += $output->writeFieldEnd(); - } $xfer += $output->writeFieldStop(); $xfer += $output->writeStructEnd(); return $xfer; @@ -12560,13 +16476,13 @@ class ThriftHiveMetastore_append_partition_by_name_with_environment_context_resu } -class ThriftHiveMetastore_drop_partition_args { +class ThriftHiveMetastore_get_partitions_ps_args { static $_TSPEC; public $db_name = null; public $tbl_name = null; public $part_vals = null; - public $deleteData = null; + public $max_parts = -1; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { @@ -12588,8 +16504,8 @@ class ThriftHiveMetastore_drop_partition_args { ), ), 4 => array( - 'var' => 'deleteData', - 'type' => TType::BOOL, + 'var' => 'max_parts', + 'type' => TType::I16, ), ); } @@ -12603,14 +16519,14 @@ class ThriftHiveMetastore_drop_partition_args { if (isset($vals['part_vals'])) { $this->part_vals = $vals['part_vals']; } - if (isset($vals['deleteData'])) { - $this->deleteData = $vals['deleteData']; + if (isset($vals['max_parts'])) { + $this->max_parts = $vals['max_parts']; } } } public function getName() { - return 'ThriftHiveMetastore_drop_partition_args'; + return 'ThriftHiveMetastore_get_partitions_ps_args'; } public function read($input) @@ -12645,14 +16561,14 @@ class ThriftHiveMetastore_drop_partition_args { case 3: if ($ftype == TType::LST) { $this->part_vals = array(); - $_size413 = 0; - $_etype416 = 0; - $xfer += $input->readListBegin($_etype416, $_size413); - for ($_i417 = 0; $_i417 < $_size413; ++$_i417) + $_size528 = 0; + $_etype531 = 0; + $xfer += $input->readListBegin($_etype531, $_size528); + for ($_i532 = 0; $_i532 < $_size528; ++$_i532) { - $elem418 = null; - $xfer += $input->readString($elem418); - $this->part_vals []= $elem418; + $elem533 = null; + $xfer += $input->readString($elem533); + $this->part_vals []= $elem533; } $xfer += $input->readListEnd(); } else { @@ -12660,8 +16576,8 @@ class ThriftHiveMetastore_drop_partition_args { } break; case 4: - if ($ftype == TType::BOOL) { - $xfer += $input->readBool($this->deleteData); + if ($ftype == TType::I16) { + $xfer += $input->readI16($this->max_parts); } else { $xfer += $input->skip($ftype); } @@ -12678,7 +16594,7 @@ class ThriftHiveMetastore_drop_partition_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_drop_partition_args'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_partitions_ps_args'); if ($this->db_name !== null) { $xfer += $output->writeFieldBegin('db_name', TType::STRING, 1); $xfer += $output->writeString($this->db_name); @@ -12697,18 +16613,18 @@ class ThriftHiveMetastore_drop_partition_args { { $output->writeListBegin(TType::STRING, count($this->part_vals)); { - foreach ($this->part_vals as $iter419) + foreach ($this->part_vals as $iter534) { - $xfer += $output->writeString($iter419); + $xfer += $output->writeString($iter534); } } $output->writeListEnd(); } $xfer += $output->writeFieldEnd(); } - if ($this->deleteData !== null) { - $xfer += $output->writeFieldBegin('deleteData', TType::BOOL, 4); - $xfer += $output->writeBool($this->deleteData); + if ($this->max_parts !== null) { + $xfer += $output->writeFieldBegin('max_parts', TType::I16, 4); + $xfer += $output->writeI16($this->max_parts); $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -12718,7 +16634,7 @@ class ThriftHiveMetastore_drop_partition_args { } -class ThriftHiveMetastore_drop_partition_result { +class ThriftHiveMetastore_get_partitions_ps_result { static $_TSPEC; public $success = null; @@ -12730,17 +16646,22 @@ class ThriftHiveMetastore_drop_partition_result { self::$_TSPEC = array( 0 => array( 'var' => 'success', - 'type' => TType::BOOL, + 'type' => TType::LST, + 'etype' => TType::STRUCT, + 'elem' => array( + 'type' => TType::STRUCT, + 'class' => '\metastore\Partition', + ), ), 1 => array( 'var' => 'o1', 'type' => TType::STRUCT, - 'class' => '\metastore\NoSuchObjectException', + 'class' => '\metastore\MetaException', ), 2 => array( 'var' => 'o2', 'type' => TType::STRUCT, - 'class' => '\metastore\MetaException', + 'class' => '\metastore\NoSuchObjectException', ), ); } @@ -12758,7 +16679,7 @@ class ThriftHiveMetastore_drop_partition_result { } public function getName() { - return 'ThriftHiveMetastore_drop_partition_result'; + return 'ThriftHiveMetastore_get_partitions_ps_result'; } public function read($input) @@ -12777,15 +16698,26 @@ class ThriftHiveMetastore_drop_partition_result { switch ($fid) { case 0: - if ($ftype == TType::BOOL) { - $xfer += $input->readBool($this->success); + if ($ftype == TType::LST) { + $this->success = array(); + $_size535 = 0; + $_etype538 = 0; + $xfer += $input->readListBegin($_etype538, $_size535); + for ($_i539 = 0; $_i539 < $_size535; ++$_i539) + { + $elem540 = null; + $elem540 = new \metastore\Partition(); + $xfer += $elem540->read($input); + $this->success []= $elem540; + } + $xfer += $input->readListEnd(); } else { $xfer += $input->skip($ftype); } break; case 1: if ($ftype == TType::STRUCT) { - $this->o1 = new \metastore\NoSuchObjectException(); + $this->o1 = new \metastore\MetaException(); $xfer += $this->o1->read($input); } else { $xfer += $input->skip($ftype); @@ -12793,7 +16725,7 @@ class ThriftHiveMetastore_drop_partition_result { break; case 2: if ($ftype == TType::STRUCT) { - $this->o2 = new \metastore\MetaException(); + $this->o2 = new \metastore\NoSuchObjectException(); $xfer += $this->o2->read($input); } else { $xfer += $input->skip($ftype); @@ -12811,10 +16743,22 @@ class ThriftHiveMetastore_drop_partition_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_drop_partition_result'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_partitions_ps_result'); if ($this->success !== null) { - $xfer += $output->writeFieldBegin('success', TType::BOOL, 0); - $xfer += $output->writeBool($this->success); + if (!is_array($this->success)) { + throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); + } + $xfer += $output->writeFieldBegin('success', TType::LST, 0); + { + $output->writeListBegin(TType::STRUCT, count($this->success)); + { + foreach ($this->success as $iter541) + { + $xfer += $iter541->write($output); + } + } + $output->writeListEnd(); + } $xfer += $output->writeFieldEnd(); } if ($this->o1 !== null) { @@ -12834,14 +16778,15 @@ class ThriftHiveMetastore_drop_partition_result { } -class ThriftHiveMetastore_drop_partition_with_environment_context_args { +class ThriftHiveMetastore_get_partitions_ps_with_auth_args { static $_TSPEC; public $db_name = null; public $tbl_name = null; public $part_vals = null; - public $deleteData = null; - public $environment_context = null; + public $max_parts = -1; + public $user_name = null; + public $group_names = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { @@ -12863,13 +16808,20 @@ class ThriftHiveMetastore_drop_partition_with_environment_context_args { ), ), 4 => array( - 'var' => 'deleteData', - 'type' => TType::BOOL, + 'var' => 'max_parts', + 'type' => TType::I16, ), 5 => array( - 'var' => 'environment_context', - 'type' => TType::STRUCT, - 'class' => '\metastore\EnvironmentContext', + 'var' => 'user_name', + 'type' => TType::STRING, + ), + 6 => array( + 'var' => 'group_names', + 'type' => TType::LST, + 'etype' => TType::STRING, + 'elem' => array( + 'type' => TType::STRING, + ), ), ); } @@ -12883,17 +16835,20 @@ class ThriftHiveMetastore_drop_partition_with_environment_context_args { if (isset($vals['part_vals'])) { $this->part_vals = $vals['part_vals']; } - if (isset($vals['deleteData'])) { - $this->deleteData = $vals['deleteData']; + if (isset($vals['max_parts'])) { + $this->max_parts = $vals['max_parts']; } - if (isset($vals['environment_context'])) { - $this->environment_context = $vals['environment_context']; + if (isset($vals['user_name'])) { + $this->user_name = $vals['user_name']; + } + if (isset($vals['group_names'])) { + $this->group_names = $vals['group_names']; } } } public function getName() { - return 'ThriftHiveMetastore_drop_partition_with_environment_context_args'; + return 'ThriftHiveMetastore_get_partitions_ps_with_auth_args'; } public function read($input) @@ -12928,31 +16883,47 @@ class ThriftHiveMetastore_drop_partition_with_environment_context_args { case 3: if ($ftype == TType::LST) { $this->part_vals = array(); - $_size420 = 0; - $_etype423 = 0; - $xfer += $input->readListBegin($_etype423, $_size420); - for ($_i424 = 0; $_i424 < $_size420; ++$_i424) + $_size542 = 0; + $_etype545 = 0; + $xfer += $input->readListBegin($_etype545, $_size542); + for ($_i546 = 0; $_i546 < $_size542; ++$_i546) { - $elem425 = null; - $xfer += $input->readString($elem425); - $this->part_vals []= $elem425; + $elem547 = null; + $xfer += $input->readString($elem547); + $this->part_vals []= $elem547; } $xfer += $input->readListEnd(); } else { $xfer += $input->skip($ftype); } break; - case 4: - if ($ftype == TType::BOOL) { - $xfer += $input->readBool($this->deleteData); + case 4: + if ($ftype == TType::I16) { + $xfer += $input->readI16($this->max_parts); + } else { + $xfer += $input->skip($ftype); + } + break; + case 5: + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->user_name); } else { $xfer += $input->skip($ftype); } break; - case 5: - if ($ftype == TType::STRUCT) { - $this->environment_context = new \metastore\EnvironmentContext(); - $xfer += $this->environment_context->read($input); + case 6: + if ($ftype == TType::LST) { + $this->group_names = array(); + $_size548 = 0; + $_etype551 = 0; + $xfer += $input->readListBegin($_etype551, $_size548); + for ($_i552 = 0; $_i552 < $_size548; ++$_i552) + { + $elem553 = null; + $xfer += $input->readString($elem553); + $this->group_names []= $elem553; + } + $xfer += $input->readListEnd(); } else { $xfer += $input->skip($ftype); } @@ -12969,7 +16940,7 @@ class ThriftHiveMetastore_drop_partition_with_environment_context_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_drop_partition_with_environment_context_args'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_partitions_ps_with_auth_args'); if ($this->db_name !== null) { $xfer += $output->writeFieldBegin('db_name', TType::STRING, 1); $xfer += $output->writeString($this->db_name); @@ -12988,26 +16959,40 @@ class ThriftHiveMetastore_drop_partition_with_environment_context_args { { $output->writeListBegin(TType::STRING, count($this->part_vals)); { - foreach ($this->part_vals as $iter426) + foreach ($this->part_vals as $iter554) { - $xfer += $output->writeString($iter426); + $xfer += $output->writeString($iter554); } } $output->writeListEnd(); } $xfer += $output->writeFieldEnd(); } - if ($this->deleteData !== null) { - $xfer += $output->writeFieldBegin('deleteData', TType::BOOL, 4); - $xfer += $output->writeBool($this->deleteData); + if ($this->max_parts !== null) { + $xfer += $output->writeFieldBegin('max_parts', TType::I16, 4); + $xfer += $output->writeI16($this->max_parts); $xfer += $output->writeFieldEnd(); } - if ($this->environment_context !== null) { - if (!is_object($this->environment_context)) { + if ($this->user_name !== null) { + $xfer += $output->writeFieldBegin('user_name', TType::STRING, 5); + $xfer += $output->writeString($this->user_name); + $xfer += $output->writeFieldEnd(); + } + if ($this->group_names !== null) { + if (!is_array($this->group_names)) { throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); } - $xfer += $output->writeFieldBegin('environment_context', TType::STRUCT, 5); - $xfer += $this->environment_context->write($output); + $xfer += $output->writeFieldBegin('group_names', TType::LST, 6); + { + $output->writeListBegin(TType::STRING, count($this->group_names)); + { + foreach ($this->group_names as $iter555) + { + $xfer += $output->writeString($iter555); + } + } + $output->writeListEnd(); + } $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -13017,7 +17002,7 @@ class ThriftHiveMetastore_drop_partition_with_environment_context_args { } -class ThriftHiveMetastore_drop_partition_with_environment_context_result { +class ThriftHiveMetastore_get_partitions_ps_with_auth_result { static $_TSPEC; public $success = null; @@ -13029,7 +17014,12 @@ class ThriftHiveMetastore_drop_partition_with_environment_context_result { self::$_TSPEC = array( 0 => array( 'var' => 'success', - 'type' => TType::BOOL, + 'type' => TType::LST, + 'etype' => TType::STRUCT, + 'elem' => array( + 'type' => TType::STRUCT, + 'class' => '\metastore\Partition', + ), ), 1 => array( 'var' => 'o1', @@ -13057,7 +17047,7 @@ class ThriftHiveMetastore_drop_partition_with_environment_context_result { } public function getName() { - return 'ThriftHiveMetastore_drop_partition_with_environment_context_result'; + return 'ThriftHiveMetastore_get_partitions_ps_with_auth_result'; } public function read($input) @@ -13076,8 +17066,19 @@ class ThriftHiveMetastore_drop_partition_with_environment_context_result { switch ($fid) { case 0: - if ($ftype == TType::BOOL) { - $xfer += $input->readBool($this->success); + if ($ftype == TType::LST) { + $this->success = array(); + $_size556 = 0; + $_etype559 = 0; + $xfer += $input->readListBegin($_etype559, $_size556); + for ($_i560 = 0; $_i560 < $_size556; ++$_i560) + { + $elem561 = null; + $elem561 = new \metastore\Partition(); + $xfer += $elem561->read($input); + $this->success []= $elem561; + } + $xfer += $input->readListEnd(); } else { $xfer += $input->skip($ftype); } @@ -13110,10 +17111,22 @@ class ThriftHiveMetastore_drop_partition_with_environment_context_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_drop_partition_with_environment_context_result'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_partitions_ps_with_auth_result'); if ($this->success !== null) { - $xfer += $output->writeFieldBegin('success', TType::BOOL, 0); - $xfer += $output->writeBool($this->success); + if (!is_array($this->success)) { + throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); + } + $xfer += $output->writeFieldBegin('success', TType::LST, 0); + { + $output->writeListBegin(TType::STRUCT, count($this->success)); + { + foreach ($this->success as $iter562) + { + $xfer += $iter562->write($output); + } + } + $output->writeListEnd(); + } $xfer += $output->writeFieldEnd(); } if ($this->o1 !== null) { @@ -13133,13 +17146,13 @@ class ThriftHiveMetastore_drop_partition_with_environment_context_result { } -class ThriftHiveMetastore_drop_partition_by_name_args { +class ThriftHiveMetastore_get_partition_names_ps_args { static $_TSPEC; public $db_name = null; public $tbl_name = null; - public $part_name = null; - public $deleteData = null; + public $part_vals = null; + public $max_parts = -1; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { @@ -13153,12 +17166,16 @@ class ThriftHiveMetastore_drop_partition_by_name_args { 'type' => TType::STRING, ), 3 => array( - 'var' => 'part_name', - 'type' => TType::STRING, + 'var' => 'part_vals', + 'type' => TType::LST, + 'etype' => TType::STRING, + 'elem' => array( + 'type' => TType::STRING, + ), ), 4 => array( - 'var' => 'deleteData', - 'type' => TType::BOOL, + 'var' => 'max_parts', + 'type' => TType::I16, ), ); } @@ -13169,17 +17186,17 @@ class ThriftHiveMetastore_drop_partition_by_name_args { if (isset($vals['tbl_name'])) { $this->tbl_name = $vals['tbl_name']; } - if (isset($vals['part_name'])) { - $this->part_name = $vals['part_name']; + if (isset($vals['part_vals'])) { + $this->part_vals = $vals['part_vals']; } - if (isset($vals['deleteData'])) { - $this->deleteData = $vals['deleteData']; + if (isset($vals['max_parts'])) { + $this->max_parts = $vals['max_parts']; } } } public function getName() { - return 'ThriftHiveMetastore_drop_partition_by_name_args'; + return 'ThriftHiveMetastore_get_partition_names_ps_args'; } public function read($input) @@ -13212,15 +17229,25 @@ class ThriftHiveMetastore_drop_partition_by_name_args { } break; case 3: - if ($ftype == TType::STRING) { - $xfer += $input->readString($this->part_name); + if ($ftype == TType::LST) { + $this->part_vals = array(); + $_size563 = 0; + $_etype566 = 0; + $xfer += $input->readListBegin($_etype566, $_size563); + for ($_i567 = 0; $_i567 < $_size563; ++$_i567) + { + $elem568 = null; + $xfer += $input->readString($elem568); + $this->part_vals []= $elem568; + } + $xfer += $input->readListEnd(); } else { $xfer += $input->skip($ftype); } break; case 4: - if ($ftype == TType::BOOL) { - $xfer += $input->readBool($this->deleteData); + if ($ftype == TType::I16) { + $xfer += $input->readI16($this->max_parts); } else { $xfer += $input->skip($ftype); } @@ -13237,7 +17264,7 @@ class ThriftHiveMetastore_drop_partition_by_name_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_drop_partition_by_name_args'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_partition_names_ps_args'); if ($this->db_name !== null) { $xfer += $output->writeFieldBegin('db_name', TType::STRING, 1); $xfer += $output->writeString($this->db_name); @@ -13248,14 +17275,26 @@ class ThriftHiveMetastore_drop_partition_by_name_args { $xfer += $output->writeString($this->tbl_name); $xfer += $output->writeFieldEnd(); } - if ($this->part_name !== null) { - $xfer += $output->writeFieldBegin('part_name', TType::STRING, 3); - $xfer += $output->writeString($this->part_name); + if ($this->part_vals !== null) { + if (!is_array($this->part_vals)) { + throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); + } + $xfer += $output->writeFieldBegin('part_vals', TType::LST, 3); + { + $output->writeListBegin(TType::STRING, count($this->part_vals)); + { + foreach ($this->part_vals as $iter569) + { + $xfer += $output->writeString($iter569); + } + } + $output->writeListEnd(); + } $xfer += $output->writeFieldEnd(); } - if ($this->deleteData !== null) { - $xfer += $output->writeFieldBegin('deleteData', TType::BOOL, 4); - $xfer += $output->writeBool($this->deleteData); + if ($this->max_parts !== null) { + $xfer += $output->writeFieldBegin('max_parts', TType::I16, 4); + $xfer += $output->writeI16($this->max_parts); $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -13265,7 +17304,7 @@ class ThriftHiveMetastore_drop_partition_by_name_args { } -class ThriftHiveMetastore_drop_partition_by_name_result { +class ThriftHiveMetastore_get_partition_names_ps_result { static $_TSPEC; public $success = null; @@ -13277,17 +17316,21 @@ class ThriftHiveMetastore_drop_partition_by_name_result { self::$_TSPEC = array( 0 => array( 'var' => 'success', - 'type' => TType::BOOL, + 'type' => TType::LST, + 'etype' => TType::STRING, + 'elem' => array( + 'type' => TType::STRING, + ), ), 1 => array( 'var' => 'o1', 'type' => TType::STRUCT, - 'class' => '\metastore\NoSuchObjectException', + 'class' => '\metastore\MetaException', ), 2 => array( 'var' => 'o2', 'type' => TType::STRUCT, - 'class' => '\metastore\MetaException', + 'class' => '\metastore\NoSuchObjectException', ), ); } @@ -13305,7 +17348,7 @@ class ThriftHiveMetastore_drop_partition_by_name_result { } public function getName() { - return 'ThriftHiveMetastore_drop_partition_by_name_result'; + return 'ThriftHiveMetastore_get_partition_names_ps_result'; } public function read($input) @@ -13324,15 +17367,25 @@ class ThriftHiveMetastore_drop_partition_by_name_result { switch ($fid) { case 0: - if ($ftype == TType::BOOL) { - $xfer += $input->readBool($this->success); + if ($ftype == TType::LST) { + $this->success = array(); + $_size570 = 0; + $_etype573 = 0; + $xfer += $input->readListBegin($_etype573, $_size570); + for ($_i574 = 0; $_i574 < $_size570; ++$_i574) + { + $elem575 = null; + $xfer += $input->readString($elem575); + $this->success []= $elem575; + } + $xfer += $input->readListEnd(); } else { $xfer += $input->skip($ftype); } break; case 1: if ($ftype == TType::STRUCT) { - $this->o1 = new \metastore\NoSuchObjectException(); + $this->o1 = new \metastore\MetaException(); $xfer += $this->o1->read($input); } else { $xfer += $input->skip($ftype); @@ -13340,7 +17393,7 @@ class ThriftHiveMetastore_drop_partition_by_name_result { break; case 2: if ($ftype == TType::STRUCT) { - $this->o2 = new \metastore\MetaException(); + $this->o2 = new \metastore\NoSuchObjectException(); $xfer += $this->o2->read($input); } else { $xfer += $input->skip($ftype); @@ -13358,10 +17411,22 @@ class ThriftHiveMetastore_drop_partition_by_name_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_drop_partition_by_name_result'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_partition_names_ps_result'); if ($this->success !== null) { - $xfer += $output->writeFieldBegin('success', TType::BOOL, 0); - $xfer += $output->writeBool($this->success); + if (!is_array($this->success)) { + throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); + } + $xfer += $output->writeFieldBegin('success', TType::LST, 0); + { + $output->writeListBegin(TType::STRING, count($this->success)); + { + foreach ($this->success as $iter576) + { + $xfer += $output->writeString($iter576); + } + } + $output->writeListEnd(); + } $xfer += $output->writeFieldEnd(); } if ($this->o1 !== null) { @@ -13381,14 +17446,13 @@ class ThriftHiveMetastore_drop_partition_by_name_result { } -class ThriftHiveMetastore_drop_partition_by_name_with_environment_context_args { +class ThriftHiveMetastore_get_partitions_by_filter_args { static $_TSPEC; public $db_name = null; public $tbl_name = null; - public $part_name = null; - public $deleteData = null; - public $environment_context = null; + public $filter = null; + public $max_parts = -1; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { @@ -13402,17 +17466,12 @@ class ThriftHiveMetastore_drop_partition_by_name_with_environment_context_args { 'type' => TType::STRING, ), 3 => array( - 'var' => 'part_name', + 'var' => 'filter', 'type' => TType::STRING, ), 4 => array( - 'var' => 'deleteData', - 'type' => TType::BOOL, - ), - 5 => array( - 'var' => 'environment_context', - 'type' => TType::STRUCT, - 'class' => '\metastore\EnvironmentContext', + 'var' => 'max_parts', + 'type' => TType::I16, ), ); } @@ -13423,20 +17482,17 @@ class ThriftHiveMetastore_drop_partition_by_name_with_environment_context_args { if (isset($vals['tbl_name'])) { $this->tbl_name = $vals['tbl_name']; } - if (isset($vals['part_name'])) { - $this->part_name = $vals['part_name']; - } - if (isset($vals['deleteData'])) { - $this->deleteData = $vals['deleteData']; + if (isset($vals['filter'])) { + $this->filter = $vals['filter']; } - if (isset($vals['environment_context'])) { - $this->environment_context = $vals['environment_context']; + if (isset($vals['max_parts'])) { + $this->max_parts = $vals['max_parts']; } } } public function getName() { - return 'ThriftHiveMetastore_drop_partition_by_name_with_environment_context_args'; + return 'ThriftHiveMetastore_get_partitions_by_filter_args'; } public function read($input) @@ -13470,22 +17526,14 @@ class ThriftHiveMetastore_drop_partition_by_name_with_environment_context_args { break; case 3: if ($ftype == TType::STRING) { - $xfer += $input->readString($this->part_name); + $xfer += $input->readString($this->filter); } else { $xfer += $input->skip($ftype); } break; case 4: - if ($ftype == TType::BOOL) { - $xfer += $input->readBool($this->deleteData); - } else { - $xfer += $input->skip($ftype); - } - break; - case 5: - if ($ftype == TType::STRUCT) { - $this->environment_context = new \metastore\EnvironmentContext(); - $xfer += $this->environment_context->read($input); + if ($ftype == TType::I16) { + $xfer += $input->readI16($this->max_parts); } else { $xfer += $input->skip($ftype); } @@ -13502,7 +17550,7 @@ class ThriftHiveMetastore_drop_partition_by_name_with_environment_context_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_drop_partition_by_name_with_environment_context_args'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_partitions_by_filter_args'); if ($this->db_name !== null) { $xfer += $output->writeFieldBegin('db_name', TType::STRING, 1); $xfer += $output->writeString($this->db_name); @@ -13513,22 +17561,14 @@ class ThriftHiveMetastore_drop_partition_by_name_with_environment_context_args { $xfer += $output->writeString($this->tbl_name); $xfer += $output->writeFieldEnd(); } - if ($this->part_name !== null) { - $xfer += $output->writeFieldBegin('part_name', TType::STRING, 3); - $xfer += $output->writeString($this->part_name); - $xfer += $output->writeFieldEnd(); - } - if ($this->deleteData !== null) { - $xfer += $output->writeFieldBegin('deleteData', TType::BOOL, 4); - $xfer += $output->writeBool($this->deleteData); + if ($this->filter !== null) { + $xfer += $output->writeFieldBegin('filter', TType::STRING, 3); + $xfer += $output->writeString($this->filter); $xfer += $output->writeFieldEnd(); } - if ($this->environment_context !== null) { - if (!is_object($this->environment_context)) { - throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); - } - $xfer += $output->writeFieldBegin('environment_context', TType::STRUCT, 5); - $xfer += $this->environment_context->write($output); + if ($this->max_parts !== null) { + $xfer += $output->writeFieldBegin('max_parts', TType::I16, 4); + $xfer += $output->writeI16($this->max_parts); $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -13538,7 +17578,7 @@ class ThriftHiveMetastore_drop_partition_by_name_with_environment_context_args { } -class ThriftHiveMetastore_drop_partition_by_name_with_environment_context_result { +class ThriftHiveMetastore_get_partitions_by_filter_result { static $_TSPEC; public $success = null; @@ -13550,17 +17590,22 @@ class ThriftHiveMetastore_drop_partition_by_name_with_environment_context_result self::$_TSPEC = array( 0 => array( 'var' => 'success', - 'type' => TType::BOOL, + 'type' => TType::LST, + 'etype' => TType::STRUCT, + 'elem' => array( + 'type' => TType::STRUCT, + 'class' => '\metastore\Partition', + ), ), 1 => array( 'var' => 'o1', 'type' => TType::STRUCT, - 'class' => '\metastore\NoSuchObjectException', + 'class' => '\metastore\MetaException', ), 2 => array( 'var' => 'o2', 'type' => TType::STRUCT, - 'class' => '\metastore\MetaException', + 'class' => '\metastore\NoSuchObjectException', ), ); } @@ -13578,7 +17623,7 @@ class ThriftHiveMetastore_drop_partition_by_name_with_environment_context_result } public function getName() { - return 'ThriftHiveMetastore_drop_partition_by_name_with_environment_context_result'; + return 'ThriftHiveMetastore_get_partitions_by_filter_result'; } public function read($input) @@ -13597,15 +17642,26 @@ class ThriftHiveMetastore_drop_partition_by_name_with_environment_context_result switch ($fid) { case 0: - if ($ftype == TType::BOOL) { - $xfer += $input->readBool($this->success); + if ($ftype == TType::LST) { + $this->success = array(); + $_size577 = 0; + $_etype580 = 0; + $xfer += $input->readListBegin($_etype580, $_size577); + for ($_i581 = 0; $_i581 < $_size577; ++$_i581) + { + $elem582 = null; + $elem582 = new \metastore\Partition(); + $xfer += $elem582->read($input); + $this->success []= $elem582; + } + $xfer += $input->readListEnd(); } else { $xfer += $input->skip($ftype); } break; case 1: if ($ftype == TType::STRUCT) { - $this->o1 = new \metastore\NoSuchObjectException(); + $this->o1 = new \metastore\MetaException(); $xfer += $this->o1->read($input); } else { $xfer += $input->skip($ftype); @@ -13613,7 +17669,7 @@ class ThriftHiveMetastore_drop_partition_by_name_with_environment_context_result break; case 2: if ($ftype == TType::STRUCT) { - $this->o2 = new \metastore\MetaException(); + $this->o2 = new \metastore\NoSuchObjectException(); $xfer += $this->o2->read($input); } else { $xfer += $input->skip($ftype); @@ -13631,10 +17687,22 @@ class ThriftHiveMetastore_drop_partition_by_name_with_environment_context_result public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_drop_partition_by_name_with_environment_context_result'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_partitions_by_filter_result'); if ($this->success !== null) { - $xfer += $output->writeFieldBegin('success', TType::BOOL, 0); - $xfer += $output->writeBool($this->success); + if (!is_array($this->success)) { + throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); + } + $xfer += $output->writeFieldBegin('success', TType::LST, 0); + { + $output->writeListBegin(TType::STRUCT, count($this->success)); + { + foreach ($this->success as $iter583) + { + $xfer += $iter583->write($output); + } + } + $output->writeListEnd(); + } $xfer += $output->writeFieldEnd(); } if ($this->o1 !== null) { @@ -13654,7 +17722,7 @@ class ThriftHiveMetastore_drop_partition_by_name_with_environment_context_result } -class ThriftHiveMetastore_drop_partitions_req_args { +class ThriftHiveMetastore_get_partitions_by_expr_args { static $_TSPEC; public $req = null; @@ -13665,7 +17733,7 @@ class ThriftHiveMetastore_drop_partitions_req_args { 1 => array( 'var' => 'req', 'type' => TType::STRUCT, - 'class' => '\metastore\DropPartitionsRequest', + 'class' => '\metastore\PartitionsByExprRequest', ), ); } @@ -13677,7 +17745,7 @@ class ThriftHiveMetastore_drop_partitions_req_args { } public function getName() { - return 'ThriftHiveMetastore_drop_partitions_req_args'; + return 'ThriftHiveMetastore_get_partitions_by_expr_args'; } public function read($input) @@ -13697,7 +17765,7 @@ class ThriftHiveMetastore_drop_partitions_req_args { { case 1: if ($ftype == TType::STRUCT) { - $this->req = new \metastore\DropPartitionsRequest(); + $this->req = new \metastore\PartitionsByExprRequest(); $xfer += $this->req->read($input); } else { $xfer += $input->skip($ftype); @@ -13715,7 +17783,7 @@ class ThriftHiveMetastore_drop_partitions_req_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_drop_partitions_req_args'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_partitions_by_expr_args'); if ($this->req !== null) { if (!is_object($this->req)) { throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); @@ -13731,7 +17799,7 @@ class ThriftHiveMetastore_drop_partitions_req_args { } -class ThriftHiveMetastore_drop_partitions_req_result { +class ThriftHiveMetastore_get_partitions_by_expr_result { static $_TSPEC; public $success = null; @@ -13744,17 +17812,17 @@ class ThriftHiveMetastore_drop_partitions_req_result { 0 => array( 'var' => 'success', 'type' => TType::STRUCT, - 'class' => '\metastore\DropPartitionsResult', + 'class' => '\metastore\PartitionsByExprResult', ), 1 => array( 'var' => 'o1', 'type' => TType::STRUCT, - 'class' => '\metastore\NoSuchObjectException', + 'class' => '\metastore\MetaException', ), 2 => array( 'var' => 'o2', 'type' => TType::STRUCT, - 'class' => '\metastore\MetaException', + 'class' => '\metastore\NoSuchObjectException', ), ); } @@ -13772,7 +17840,7 @@ class ThriftHiveMetastore_drop_partitions_req_result { } public function getName() { - return 'ThriftHiveMetastore_drop_partitions_req_result'; + return 'ThriftHiveMetastore_get_partitions_by_expr_result'; } public function read($input) @@ -13792,7 +17860,7 @@ class ThriftHiveMetastore_drop_partitions_req_result { { case 0: if ($ftype == TType::STRUCT) { - $this->success = new \metastore\DropPartitionsResult(); + $this->success = new \metastore\PartitionsByExprResult(); $xfer += $this->success->read($input); } else { $xfer += $input->skip($ftype); @@ -13800,7 +17868,7 @@ class ThriftHiveMetastore_drop_partitions_req_result { break; case 1: if ($ftype == TType::STRUCT) { - $this->o1 = new \metastore\NoSuchObjectException(); + $this->o1 = new \metastore\MetaException(); $xfer += $this->o1->read($input); } else { $xfer += $input->skip($ftype); @@ -13808,7 +17876,7 @@ class ThriftHiveMetastore_drop_partitions_req_result { break; case 2: if ($ftype == TType::STRUCT) { - $this->o2 = new \metastore\MetaException(); + $this->o2 = new \metastore\NoSuchObjectException(); $xfer += $this->o2->read($input); } else { $xfer += $input->skip($ftype); @@ -13826,7 +17894,7 @@ class ThriftHiveMetastore_drop_partitions_req_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_drop_partitions_req_result'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_partitions_by_expr_result'); if ($this->success !== null) { if (!is_object($this->success)) { throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); @@ -13852,12 +17920,12 @@ class ThriftHiveMetastore_drop_partitions_req_result { } -class ThriftHiveMetastore_get_partition_args { +class ThriftHiveMetastore_get_partitions_by_names_args { static $_TSPEC; public $db_name = null; public $tbl_name = null; - public $part_vals = null; + public $names = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { @@ -13871,7 +17939,7 @@ class ThriftHiveMetastore_get_partition_args { 'type' => TType::STRING, ), 3 => array( - 'var' => 'part_vals', + 'var' => 'names', 'type' => TType::LST, 'etype' => TType::STRING, 'elem' => array( @@ -13887,14 +17955,14 @@ class ThriftHiveMetastore_get_partition_args { if (isset($vals['tbl_name'])) { $this->tbl_name = $vals['tbl_name']; } - if (isset($vals['part_vals'])) { - $this->part_vals = $vals['part_vals']; + if (isset($vals['names'])) { + $this->names = $vals['names']; } } } public function getName() { - return 'ThriftHiveMetastore_get_partition_args'; + return 'ThriftHiveMetastore_get_partitions_by_names_args'; } public function read($input) @@ -13928,15 +17996,15 @@ class ThriftHiveMetastore_get_partition_args { break; case 3: if ($ftype == TType::LST) { - $this->part_vals = array(); - $_size427 = 0; - $_etype430 = 0; - $xfer += $input->readListBegin($_etype430, $_size427); - for ($_i431 = 0; $_i431 < $_size427; ++$_i431) + $this->names = array(); + $_size584 = 0; + $_etype587 = 0; + $xfer += $input->readListBegin($_etype587, $_size584); + for ($_i588 = 0; $_i588 < $_size584; ++$_i588) { - $elem432 = null; - $xfer += $input->readString($elem432); - $this->part_vals []= $elem432; + $elem589 = null; + $xfer += $input->readString($elem589); + $this->names []= $elem589; } $xfer += $input->readListEnd(); } else { @@ -13955,7 +18023,7 @@ class ThriftHiveMetastore_get_partition_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_partition_args'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_partitions_by_names_args'); if ($this->db_name !== null) { $xfer += $output->writeFieldBegin('db_name', TType::STRING, 1); $xfer += $output->writeString($this->db_name); @@ -13966,17 +18034,17 @@ class ThriftHiveMetastore_get_partition_args { $xfer += $output->writeString($this->tbl_name); $xfer += $output->writeFieldEnd(); } - if ($this->part_vals !== null) { - if (!is_array($this->part_vals)) { + if ($this->names !== null) { + if (!is_array($this->names)) { throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); } - $xfer += $output->writeFieldBegin('part_vals', TType::LST, 3); + $xfer += $output->writeFieldBegin('names', TType::LST, 3); { - $output->writeListBegin(TType::STRING, count($this->part_vals)); + $output->writeListBegin(TType::STRING, count($this->names)); { - foreach ($this->part_vals as $iter433) + foreach ($this->names as $iter590) { - $xfer += $output->writeString($iter433); + $xfer += $output->writeString($iter590); } } $output->writeListEnd(); @@ -13990,7 +18058,7 @@ class ThriftHiveMetastore_get_partition_args { } -class ThriftHiveMetastore_get_partition_result { +class ThriftHiveMetastore_get_partitions_by_names_result { static $_TSPEC; public $success = null; @@ -14002,8 +18070,12 @@ class ThriftHiveMetastore_get_partition_result { self::$_TSPEC = array( 0 => array( 'var' => 'success', - 'type' => TType::STRUCT, - 'class' => '\metastore\Partition', + 'type' => TType::LST, + 'etype' => TType::STRUCT, + 'elem' => array( + 'type' => TType::STRUCT, + 'class' => '\metastore\Partition', + ), ), 1 => array( 'var' => 'o1', @@ -14031,7 +18103,7 @@ class ThriftHiveMetastore_get_partition_result { } public function getName() { - return 'ThriftHiveMetastore_get_partition_result'; + return 'ThriftHiveMetastore_get_partitions_by_names_result'; } public function read($input) @@ -14050,9 +18122,19 @@ class ThriftHiveMetastore_get_partition_result { switch ($fid) { case 0: - if ($ftype == TType::STRUCT) { - $this->success = new \metastore\Partition(); - $xfer += $this->success->read($input); + if ($ftype == TType::LST) { + $this->success = array(); + $_size591 = 0; + $_etype594 = 0; + $xfer += $input->readListBegin($_etype594, $_size591); + for ($_i595 = 0; $_i595 < $_size591; ++$_i595) + { + $elem596 = null; + $elem596 = new \metastore\Partition(); + $xfer += $elem596->read($input); + $this->success []= $elem596; + } + $xfer += $input->readListEnd(); } else { $xfer += $input->skip($ftype); } @@ -14085,13 +18167,22 @@ class ThriftHiveMetastore_get_partition_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_partition_result'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_partitions_by_names_result'); if ($this->success !== null) { - if (!is_object($this->success)) { + if (!is_array($this->success)) { throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); } - $xfer += $output->writeFieldBegin('success', TType::STRUCT, 0); - $xfer += $this->success->write($output); + $xfer += $output->writeFieldBegin('success', TType::LST, 0); + { + $output->writeListBegin(TType::STRUCT, count($this->success)); + { + foreach ($this->success as $iter597) + { + $xfer += $iter597->write($output); + } + } + $output->writeListEnd(); + } $xfer += $output->writeFieldEnd(); } if ($this->o1 !== null) { @@ -14111,69 +18202,46 @@ class ThriftHiveMetastore_get_partition_result { } -class ThriftHiveMetastore_exchange_partition_args { +class ThriftHiveMetastore_alter_partition_args { static $_TSPEC; - public $partitionSpecs = null; - public $source_db = null; - public $source_table_name = null; - public $dest_db = null; - public $dest_table_name = null; + public $db_name = null; + public $tbl_name = null; + public $new_part = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 1 => array( - 'var' => 'partitionSpecs', - 'type' => TType::MAP, - 'ktype' => TType::STRING, - 'vtype' => TType::STRING, - 'key' => array( - 'type' => TType::STRING, - ), - 'val' => array( - 'type' => TType::STRING, - ), + 'var' => 'db_name', + 'type' => TType::STRING, ), 2 => array( - 'var' => 'source_db', + 'var' => 'tbl_name', 'type' => TType::STRING, ), 3 => array( - 'var' => 'source_table_name', - 'type' => TType::STRING, - ), - 4 => array( - 'var' => 'dest_db', - 'type' => TType::STRING, - ), - 5 => array( - 'var' => 'dest_table_name', - 'type' => TType::STRING, + 'var' => 'new_part', + 'type' => TType::STRUCT, + 'class' => '\metastore\Partition', ), ); } if (is_array($vals)) { - if (isset($vals['partitionSpecs'])) { - $this->partitionSpecs = $vals['partitionSpecs']; - } - if (isset($vals['source_db'])) { - $this->source_db = $vals['source_db']; - } - if (isset($vals['source_table_name'])) { - $this->source_table_name = $vals['source_table_name']; + if (isset($vals['db_name'])) { + $this->db_name = $vals['db_name']; } - if (isset($vals['dest_db'])) { - $this->dest_db = $vals['dest_db']; + if (isset($vals['tbl_name'])) { + $this->tbl_name = $vals['tbl_name']; } - if (isset($vals['dest_table_name'])) { - $this->dest_table_name = $vals['dest_table_name']; + if (isset($vals['new_part'])) { + $this->new_part = $vals['new_part']; } } } public function getName() { - return 'ThriftHiveMetastore_exchange_partition_args'; + return 'ThriftHiveMetastore_alter_partition_args'; } public function read($input) @@ -14192,49 +18260,23 @@ class ThriftHiveMetastore_exchange_partition_args { switch ($fid) { case 1: - if ($ftype == TType::MAP) { - $this->partitionSpecs = array(); - $_size434 = 0; - $_ktype435 = 0; - $_vtype436 = 0; - $xfer += $input->readMapBegin($_ktype435, $_vtype436, $_size434); - for ($_i438 = 0; $_i438 < $_size434; ++$_i438) - { - $key439 = ''; - $val440 = ''; - $xfer += $input->readString($key439); - $xfer += $input->readString($val440); - $this->partitionSpecs[$key439] = $val440; - } - $xfer += $input->readMapEnd(); + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->db_name); } else { $xfer += $input->skip($ftype); } break; case 2: if ($ftype == TType::STRING) { - $xfer += $input->readString($this->source_db); + $xfer += $input->readString($this->tbl_name); } else { $xfer += $input->skip($ftype); } break; case 3: - if ($ftype == TType::STRING) { - $xfer += $input->readString($this->source_table_name); - } else { - $xfer += $input->skip($ftype); - } - break; - case 4: - if ($ftype == TType::STRING) { - $xfer += $input->readString($this->dest_db); - } else { - $xfer += $input->skip($ftype); - } - break; - case 5: - if ($ftype == TType::STRING) { - $xfer += $input->readString($this->dest_table_name); + if ($ftype == TType::STRUCT) { + $this->new_part = new \metastore\Partition(); + $xfer += $this->new_part->read($input); } else { $xfer += $input->skip($ftype); } @@ -14251,43 +18293,23 @@ class ThriftHiveMetastore_exchange_partition_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_exchange_partition_args'); - if ($this->partitionSpecs !== null) { - if (!is_array($this->partitionSpecs)) { - throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); - } - $xfer += $output->writeFieldBegin('partitionSpecs', TType::MAP, 1); - { - $output->writeMapBegin(TType::STRING, TType::STRING, count($this->partitionSpecs)); - { - foreach ($this->partitionSpecs as $kiter441 => $viter442) - { - $xfer += $output->writeString($kiter441); - $xfer += $output->writeString($viter442); - } - } - $output->writeMapEnd(); - } - $xfer += $output->writeFieldEnd(); - } - if ($this->source_db !== null) { - $xfer += $output->writeFieldBegin('source_db', TType::STRING, 2); - $xfer += $output->writeString($this->source_db); - $xfer += $output->writeFieldEnd(); - } - if ($this->source_table_name !== null) { - $xfer += $output->writeFieldBegin('source_table_name', TType::STRING, 3); - $xfer += $output->writeString($this->source_table_name); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_alter_partition_args'); + if ($this->db_name !== null) { + $xfer += $output->writeFieldBegin('db_name', TType::STRING, 1); + $xfer += $output->writeString($this->db_name); $xfer += $output->writeFieldEnd(); } - if ($this->dest_db !== null) { - $xfer += $output->writeFieldBegin('dest_db', TType::STRING, 4); - $xfer += $output->writeString($this->dest_db); + if ($this->tbl_name !== null) { + $xfer += $output->writeFieldBegin('tbl_name', TType::STRING, 2); + $xfer += $output->writeString($this->tbl_name); $xfer += $output->writeFieldEnd(); } - if ($this->dest_table_name !== null) { - $xfer += $output->writeFieldBegin('dest_table_name', TType::STRING, 5); - $xfer += $output->writeString($this->dest_table_name); + if ($this->new_part !== null) { + if (!is_object($this->new_part)) { + throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); + } + $xfer += $output->writeFieldBegin('new_part', TType::STRUCT, 3); + $xfer += $this->new_part->write($output); $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -14297,66 +18319,39 @@ class ThriftHiveMetastore_exchange_partition_args { } -class ThriftHiveMetastore_exchange_partition_result { +class ThriftHiveMetastore_alter_partition_result { static $_TSPEC; - public $success = null; public $o1 = null; public $o2 = null; - public $o3 = null; - public $o4 = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( - 0 => array( - 'var' => 'success', - 'type' => TType::STRUCT, - 'class' => '\metastore\Partition', - ), 1 => array( 'var' => 'o1', 'type' => TType::STRUCT, - 'class' => '\metastore\MetaException', + 'class' => '\metastore\InvalidOperationException', ), 2 => array( 'var' => 'o2', 'type' => TType::STRUCT, - 'class' => '\metastore\NoSuchObjectException', - ), - 3 => array( - 'var' => 'o3', - 'type' => TType::STRUCT, - 'class' => '\metastore\InvalidObjectException', - ), - 4 => array( - 'var' => 'o4', - 'type' => TType::STRUCT, - 'class' => '\metastore\InvalidInputException', + 'class' => '\metastore\MetaException', ), ); } if (is_array($vals)) { - if (isset($vals['success'])) { - $this->success = $vals['success']; - } if (isset($vals['o1'])) { $this->o1 = $vals['o1']; } if (isset($vals['o2'])) { $this->o2 = $vals['o2']; } - if (isset($vals['o3'])) { - $this->o3 = $vals['o3']; - } - if (isset($vals['o4'])) { - $this->o4 = $vals['o4']; - } } } public function getName() { - return 'ThriftHiveMetastore_exchange_partition_result'; + return 'ThriftHiveMetastore_alter_partition_result'; } public function read($input) @@ -14374,17 +18369,9 @@ class ThriftHiveMetastore_exchange_partition_result { } switch ($fid) { - case 0: - if ($ftype == TType::STRUCT) { - $this->success = new \metastore\Partition(); - $xfer += $this->success->read($input); - } else { - $xfer += $input->skip($ftype); - } - break; case 1: if ($ftype == TType::STRUCT) { - $this->o1 = new \metastore\MetaException(); + $this->o1 = new \metastore\InvalidOperationException(); $xfer += $this->o1->read($input); } else { $xfer += $input->skip($ftype); @@ -14392,28 +18379,12 @@ class ThriftHiveMetastore_exchange_partition_result { break; case 2: if ($ftype == TType::STRUCT) { - $this->o2 = new \metastore\NoSuchObjectException(); + $this->o2 = new \metastore\MetaException(); $xfer += $this->o2->read($input); } else { $xfer += $input->skip($ftype); } break; - case 3: - if ($ftype == TType::STRUCT) { - $this->o3 = new \metastore\InvalidObjectException(); - $xfer += $this->o3->read($input); - } else { - $xfer += $input->skip($ftype); - } - break; - case 4: - if ($ftype == TType::STRUCT) { - $this->o4 = new \metastore\InvalidInputException(); - $xfer += $this->o4->read($input); - } else { - $xfer += $input->skip($ftype); - } - break; default: $xfer += $input->skip($ftype); break; @@ -14426,15 +18397,7 @@ class ThriftHiveMetastore_exchange_partition_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_exchange_partition_result'); - if ($this->success !== null) { - if (!is_object($this->success)) { - throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); - } - $xfer += $output->writeFieldBegin('success', TType::STRUCT, 0); - $xfer += $this->success->write($output); - $xfer += $output->writeFieldEnd(); - } + $xfer += $output->writeStructBegin('ThriftHiveMetastore_alter_partition_result'); if ($this->o1 !== null) { $xfer += $output->writeFieldBegin('o1', TType::STRUCT, 1); $xfer += $this->o1->write($output); @@ -14445,16 +18408,6 @@ class ThriftHiveMetastore_exchange_partition_result { $xfer += $this->o2->write($output); $xfer += $output->writeFieldEnd(); } - if ($this->o3 !== null) { - $xfer += $output->writeFieldBegin('o3', TType::STRUCT, 3); - $xfer += $this->o3->write($output); - $xfer += $output->writeFieldEnd(); - } - if ($this->o4 !== null) { - $xfer += $output->writeFieldBegin('o4', TType::STRUCT, 4); - $xfer += $this->o4->write($output); - $xfer += $output->writeFieldEnd(); - } $xfer += $output->writeFieldStop(); $xfer += $output->writeStructEnd(); return $xfer; @@ -14462,14 +18415,12 @@ class ThriftHiveMetastore_exchange_partition_result { } -class ThriftHiveMetastore_get_partition_with_auth_args { +class ThriftHiveMetastore_alter_partitions_args { static $_TSPEC; public $db_name = null; public $tbl_name = null; - public $part_vals = null; - public $user_name = null; - public $group_names = null; + public $new_parts = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { @@ -14483,23 +18434,12 @@ class ThriftHiveMetastore_get_partition_with_auth_args { 'type' => TType::STRING, ), 3 => array( - 'var' => 'part_vals', - 'type' => TType::LST, - 'etype' => TType::STRING, - 'elem' => array( - 'type' => TType::STRING, - ), - ), - 4 => array( - 'var' => 'user_name', - 'type' => TType::STRING, - ), - 5 => array( - 'var' => 'group_names', + 'var' => 'new_parts', 'type' => TType::LST, - 'etype' => TType::STRING, + 'etype' => TType::STRUCT, 'elem' => array( - 'type' => TType::STRING, + 'type' => TType::STRUCT, + 'class' => '\metastore\Partition', ), ), ); @@ -14511,20 +18451,14 @@ class ThriftHiveMetastore_get_partition_with_auth_args { if (isset($vals['tbl_name'])) { $this->tbl_name = $vals['tbl_name']; } - if (isset($vals['part_vals'])) { - $this->part_vals = $vals['part_vals']; - } - if (isset($vals['user_name'])) { - $this->user_name = $vals['user_name']; - } - if (isset($vals['group_names'])) { - $this->group_names = $vals['group_names']; + if (isset($vals['new_parts'])) { + $this->new_parts = $vals['new_parts']; } } } public function getName() { - return 'ThriftHiveMetastore_get_partition_with_auth_args'; + return 'ThriftHiveMetastore_alter_partitions_args'; } public function read($input) @@ -14558,39 +18492,16 @@ class ThriftHiveMetastore_get_partition_with_auth_args { break; case 3: if ($ftype == TType::LST) { - $this->part_vals = array(); - $_size443 = 0; - $_etype446 = 0; - $xfer += $input->readListBegin($_etype446, $_size443); - for ($_i447 = 0; $_i447 < $_size443; ++$_i447) - { - $elem448 = null; - $xfer += $input->readString($elem448); - $this->part_vals []= $elem448; - } - $xfer += $input->readListEnd(); - } else { - $xfer += $input->skip($ftype); - } - break; - case 4: - if ($ftype == TType::STRING) { - $xfer += $input->readString($this->user_name); - } else { - $xfer += $input->skip($ftype); - } - break; - case 5: - if ($ftype == TType::LST) { - $this->group_names = array(); - $_size449 = 0; - $_etype452 = 0; - $xfer += $input->readListBegin($_etype452, $_size449); - for ($_i453 = 0; $_i453 < $_size449; ++$_i453) + $this->new_parts = array(); + $_size598 = 0; + $_etype601 = 0; + $xfer += $input->readListBegin($_etype601, $_size598); + for ($_i602 = 0; $_i602 < $_size598; ++$_i602) { - $elem454 = null; - $xfer += $input->readString($elem454); - $this->group_names []= $elem454; + $elem603 = null; + $elem603 = new \metastore\Partition(); + $xfer += $elem603->read($input); + $this->new_parts []= $elem603; } $xfer += $input->readListEnd(); } else { @@ -14609,7 +18520,7 @@ class ThriftHiveMetastore_get_partition_with_auth_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_partition_with_auth_args'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_alter_partitions_args'); if ($this->db_name !== null) { $xfer += $output->writeFieldBegin('db_name', TType::STRING, 1); $xfer += $output->writeString($this->db_name); @@ -14620,39 +18531,17 @@ class ThriftHiveMetastore_get_partition_with_auth_args { $xfer += $output->writeString($this->tbl_name); $xfer += $output->writeFieldEnd(); } - if ($this->part_vals !== null) { - if (!is_array($this->part_vals)) { - throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); - } - $xfer += $output->writeFieldBegin('part_vals', TType::LST, 3); - { - $output->writeListBegin(TType::STRING, count($this->part_vals)); - { - foreach ($this->part_vals as $iter455) - { - $xfer += $output->writeString($iter455); - } - } - $output->writeListEnd(); - } - $xfer += $output->writeFieldEnd(); - } - if ($this->user_name !== null) { - $xfer += $output->writeFieldBegin('user_name', TType::STRING, 4); - $xfer += $output->writeString($this->user_name); - $xfer += $output->writeFieldEnd(); - } - if ($this->group_names !== null) { - if (!is_array($this->group_names)) { + if ($this->new_parts !== null) { + if (!is_array($this->new_parts)) { throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); } - $xfer += $output->writeFieldBegin('group_names', TType::LST, 5); + $xfer += $output->writeFieldBegin('new_parts', TType::LST, 3); { - $output->writeListBegin(TType::STRING, count($this->group_names)); + $output->writeListBegin(TType::STRUCT, count($this->new_parts)); { - foreach ($this->group_names as $iter456) + foreach ($this->new_parts as $iter604) { - $xfer += $output->writeString($iter456); + $xfer += $iter604->write($output); } } $output->writeListEnd(); @@ -14666,37 +18555,28 @@ class ThriftHiveMetastore_get_partition_with_auth_args { } -class ThriftHiveMetastore_get_partition_with_auth_result { +class ThriftHiveMetastore_alter_partitions_result { static $_TSPEC; - public $success = null; public $o1 = null; public $o2 = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( - 0 => array( - 'var' => 'success', - 'type' => TType::STRUCT, - 'class' => '\metastore\Partition', - ), 1 => array( 'var' => 'o1', 'type' => TType::STRUCT, - 'class' => '\metastore\MetaException', + 'class' => '\metastore\InvalidOperationException', ), 2 => array( 'var' => 'o2', 'type' => TType::STRUCT, - 'class' => '\metastore\NoSuchObjectException', + 'class' => '\metastore\MetaException', ), ); } if (is_array($vals)) { - if (isset($vals['success'])) { - $this->success = $vals['success']; - } if (isset($vals['o1'])) { $this->o1 = $vals['o1']; } @@ -14707,7 +18587,7 @@ class ThriftHiveMetastore_get_partition_with_auth_result { } public function getName() { - return 'ThriftHiveMetastore_get_partition_with_auth_result'; + return 'ThriftHiveMetastore_alter_partitions_result'; } public function read($input) @@ -14725,17 +18605,9 @@ class ThriftHiveMetastore_get_partition_with_auth_result { } switch ($fid) { - case 0: - if ($ftype == TType::STRUCT) { - $this->success = new \metastore\Partition(); - $xfer += $this->success->read($input); - } else { - $xfer += $input->skip($ftype); - } - break; case 1: if ($ftype == TType::STRUCT) { - $this->o1 = new \metastore\MetaException(); + $this->o1 = new \metastore\InvalidOperationException(); $xfer += $this->o1->read($input); } else { $xfer += $input->skip($ftype); @@ -14743,7 +18615,7 @@ class ThriftHiveMetastore_get_partition_with_auth_result { break; case 2: if ($ftype == TType::STRUCT) { - $this->o2 = new \metastore\NoSuchObjectException(); + $this->o2 = new \metastore\MetaException(); $xfer += $this->o2->read($input); } else { $xfer += $input->skip($ftype); @@ -14761,15 +18633,7 @@ class ThriftHiveMetastore_get_partition_with_auth_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_partition_with_auth_result'); - if ($this->success !== null) { - if (!is_object($this->success)) { - throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); - } - $xfer += $output->writeFieldBegin('success', TType::STRUCT, 0); - $xfer += $this->success->write($output); - $xfer += $output->writeFieldEnd(); - } + $xfer += $output->writeStructBegin('ThriftHiveMetastore_alter_partitions_result'); if ($this->o1 !== null) { $xfer += $output->writeFieldBegin('o1', TType::STRUCT, 1); $xfer += $this->o1->write($output); @@ -14787,12 +18651,13 @@ class ThriftHiveMetastore_get_partition_with_auth_result { } -class ThriftHiveMetastore_get_partition_by_name_args { +class ThriftHiveMetastore_alter_partition_with_environment_context_args { static $_TSPEC; public $db_name = null; public $tbl_name = null; - public $part_name = null; + public $new_part = null; + public $environment_context = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { @@ -14806,8 +18671,14 @@ class ThriftHiveMetastore_get_partition_by_name_args { 'type' => TType::STRING, ), 3 => array( - 'var' => 'part_name', - 'type' => TType::STRING, + 'var' => 'new_part', + 'type' => TType::STRUCT, + 'class' => '\metastore\Partition', + ), + 4 => array( + 'var' => 'environment_context', + 'type' => TType::STRUCT, + 'class' => '\metastore\EnvironmentContext', ), ); } @@ -14818,14 +18689,17 @@ class ThriftHiveMetastore_get_partition_by_name_args { if (isset($vals['tbl_name'])) { $this->tbl_name = $vals['tbl_name']; } - if (isset($vals['part_name'])) { - $this->part_name = $vals['part_name']; + if (isset($vals['new_part'])) { + $this->new_part = $vals['new_part']; + } + if (isset($vals['environment_context'])) { + $this->environment_context = $vals['environment_context']; } } } public function getName() { - return 'ThriftHiveMetastore_get_partition_by_name_args'; + return 'ThriftHiveMetastore_alter_partition_with_environment_context_args'; } public function read($input) @@ -14858,8 +18732,17 @@ class ThriftHiveMetastore_get_partition_by_name_args { } break; case 3: - if ($ftype == TType::STRING) { - $xfer += $input->readString($this->part_name); + if ($ftype == TType::STRUCT) { + $this->new_part = new \metastore\Partition(); + $xfer += $this->new_part->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; + case 4: + if ($ftype == TType::STRUCT) { + $this->environment_context = new \metastore\EnvironmentContext(); + $xfer += $this->environment_context->read($input); } else { $xfer += $input->skip($ftype); } @@ -14876,7 +18759,7 @@ class ThriftHiveMetastore_get_partition_by_name_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_partition_by_name_args'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_alter_partition_with_environment_context_args'); if ($this->db_name !== null) { $xfer += $output->writeFieldBegin('db_name', TType::STRING, 1); $xfer += $output->writeString($this->db_name); @@ -14887,9 +18770,20 @@ class ThriftHiveMetastore_get_partition_by_name_args { $xfer += $output->writeString($this->tbl_name); $xfer += $output->writeFieldEnd(); } - if ($this->part_name !== null) { - $xfer += $output->writeFieldBegin('part_name', TType::STRING, 3); - $xfer += $output->writeString($this->part_name); + if ($this->new_part !== null) { + if (!is_object($this->new_part)) { + throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); + } + $xfer += $output->writeFieldBegin('new_part', TType::STRUCT, 3); + $xfer += $this->new_part->write($output); + $xfer += $output->writeFieldEnd(); + } + if ($this->environment_context !== null) { + if (!is_object($this->environment_context)) { + throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); + } + $xfer += $output->writeFieldBegin('environment_context', TType::STRUCT, 4); + $xfer += $this->environment_context->write($output); $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -14899,37 +18793,28 @@ class ThriftHiveMetastore_get_partition_by_name_args { } -class ThriftHiveMetastore_get_partition_by_name_result { +class ThriftHiveMetastore_alter_partition_with_environment_context_result { static $_TSPEC; - public $success = null; public $o1 = null; public $o2 = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( - 0 => array( - 'var' => 'success', - 'type' => TType::STRUCT, - 'class' => '\metastore\Partition', - ), 1 => array( 'var' => 'o1', 'type' => TType::STRUCT, - 'class' => '\metastore\MetaException', + 'class' => '\metastore\InvalidOperationException', ), 2 => array( 'var' => 'o2', 'type' => TType::STRUCT, - 'class' => '\metastore\NoSuchObjectException', + 'class' => '\metastore\MetaException', ), ); } if (is_array($vals)) { - if (isset($vals['success'])) { - $this->success = $vals['success']; - } if (isset($vals['o1'])) { $this->o1 = $vals['o1']; } @@ -14940,7 +18825,7 @@ class ThriftHiveMetastore_get_partition_by_name_result { } public function getName() { - return 'ThriftHiveMetastore_get_partition_by_name_result'; + return 'ThriftHiveMetastore_alter_partition_with_environment_context_result'; } public function read($input) @@ -14958,17 +18843,9 @@ class ThriftHiveMetastore_get_partition_by_name_result { } switch ($fid) { - case 0: - if ($ftype == TType::STRUCT) { - $this->success = new \metastore\Partition(); - $xfer += $this->success->read($input); - } else { - $xfer += $input->skip($ftype); - } - break; case 1: if ($ftype == TType::STRUCT) { - $this->o1 = new \metastore\MetaException(); + $this->o1 = new \metastore\InvalidOperationException(); $xfer += $this->o1->read($input); } else { $xfer += $input->skip($ftype); @@ -14976,7 +18853,7 @@ class ThriftHiveMetastore_get_partition_by_name_result { break; case 2: if ($ftype == TType::STRUCT) { - $this->o2 = new \metastore\NoSuchObjectException(); + $this->o2 = new \metastore\MetaException(); $xfer += $this->o2->read($input); } else { $xfer += $input->skip($ftype); @@ -14994,15 +18871,7 @@ class ThriftHiveMetastore_get_partition_by_name_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_partition_by_name_result'); - if ($this->success !== null) { - if (!is_object($this->success)) { - throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); - } - $xfer += $output->writeFieldBegin('success', TType::STRUCT, 0); - $xfer += $this->success->write($output); - $xfer += $output->writeFieldEnd(); - } + $xfer += $output->writeStructBegin('ThriftHiveMetastore_alter_partition_with_environment_context_result'); if ($this->o1 !== null) { $xfer += $output->writeFieldBegin('o1', TType::STRUCT, 1); $xfer += $this->o1->write($output); @@ -15020,12 +18889,13 @@ class ThriftHiveMetastore_get_partition_by_name_result { } -class ThriftHiveMetastore_get_partitions_args { +class ThriftHiveMetastore_rename_partition_args { static $_TSPEC; public $db_name = null; public $tbl_name = null; - public $max_parts = -1; + public $part_vals = null; + public $new_part = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { @@ -15039,8 +18909,17 @@ class ThriftHiveMetastore_get_partitions_args { 'type' => TType::STRING, ), 3 => array( - 'var' => 'max_parts', - 'type' => TType::I16, + 'var' => 'part_vals', + 'type' => TType::LST, + 'etype' => TType::STRING, + 'elem' => array( + 'type' => TType::STRING, + ), + ), + 4 => array( + 'var' => 'new_part', + 'type' => TType::STRUCT, + 'class' => '\metastore\Partition', ), ); } @@ -15051,14 +18930,17 @@ class ThriftHiveMetastore_get_partitions_args { if (isset($vals['tbl_name'])) { $this->tbl_name = $vals['tbl_name']; } - if (isset($vals['max_parts'])) { - $this->max_parts = $vals['max_parts']; + if (isset($vals['part_vals'])) { + $this->part_vals = $vals['part_vals']; + } + if (isset($vals['new_part'])) { + $this->new_part = $vals['new_part']; } } } public function getName() { - return 'ThriftHiveMetastore_get_partitions_args'; + return 'ThriftHiveMetastore_rename_partition_args'; } public function read($input) @@ -15091,8 +18973,26 @@ class ThriftHiveMetastore_get_partitions_args { } break; case 3: - if ($ftype == TType::I16) { - $xfer += $input->readI16($this->max_parts); + if ($ftype == TType::LST) { + $this->part_vals = array(); + $_size605 = 0; + $_etype608 = 0; + $xfer += $input->readListBegin($_etype608, $_size605); + for ($_i609 = 0; $_i609 < $_size605; ++$_i609) + { + $elem610 = null; + $xfer += $input->readString($elem610); + $this->part_vals []= $elem610; + } + $xfer += $input->readListEnd(); + } else { + $xfer += $input->skip($ftype); + } + break; + case 4: + if ($ftype == TType::STRUCT) { + $this->new_part = new \metastore\Partition(); + $xfer += $this->new_part->read($input); } else { $xfer += $input->skip($ftype); } @@ -15109,7 +19009,7 @@ class ThriftHiveMetastore_get_partitions_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_partitions_args'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_rename_partition_args'); if ($this->db_name !== null) { $xfer += $output->writeFieldBegin('db_name', TType::STRING, 1); $xfer += $output->writeString($this->db_name); @@ -15120,9 +19020,29 @@ class ThriftHiveMetastore_get_partitions_args { $xfer += $output->writeString($this->tbl_name); $xfer += $output->writeFieldEnd(); } - if ($this->max_parts !== null) { - $xfer += $output->writeFieldBegin('max_parts', TType::I16, 3); - $xfer += $output->writeI16($this->max_parts); + if ($this->part_vals !== null) { + if (!is_array($this->part_vals)) { + throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); + } + $xfer += $output->writeFieldBegin('part_vals', TType::LST, 3); + { + $output->writeListBegin(TType::STRING, count($this->part_vals)); + { + foreach ($this->part_vals as $iter611) + { + $xfer += $output->writeString($iter611); + } + } + $output->writeListEnd(); + } + $xfer += $output->writeFieldEnd(); + } + if ($this->new_part !== null) { + if (!is_object($this->new_part)) { + throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); + } + $xfer += $output->writeFieldBegin('new_part', TType::STRUCT, 4); + $xfer += $this->new_part->write($output); $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -15132,29 +19052,19 @@ class ThriftHiveMetastore_get_partitions_args { } -class ThriftHiveMetastore_get_partitions_result { +class ThriftHiveMetastore_rename_partition_result { static $_TSPEC; - public $success = null; public $o1 = null; public $o2 = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( - 0 => array( - 'var' => 'success', - 'type' => TType::LST, - 'etype' => TType::STRUCT, - 'elem' => array( - 'type' => TType::STRUCT, - 'class' => '\metastore\Partition', - ), - ), 1 => array( 'var' => 'o1', 'type' => TType::STRUCT, - 'class' => '\metastore\NoSuchObjectException', + 'class' => '\metastore\InvalidOperationException', ), 2 => array( 'var' => 'o2', @@ -15164,9 +19074,6 @@ class ThriftHiveMetastore_get_partitions_result { ); } if (is_array($vals)) { - if (isset($vals['success'])) { - $this->success = $vals['success']; - } if (isset($vals['o1'])) { $this->o1 = $vals['o1']; } @@ -15177,7 +19084,7 @@ class ThriftHiveMetastore_get_partitions_result { } public function getName() { - return 'ThriftHiveMetastore_get_partitions_result'; + return 'ThriftHiveMetastore_rename_partition_result'; } public function read($input) @@ -15195,27 +19102,9 @@ class ThriftHiveMetastore_get_partitions_result { } switch ($fid) { - case 0: - if ($ftype == TType::LST) { - $this->success = array(); - $_size457 = 0; - $_etype460 = 0; - $xfer += $input->readListBegin($_etype460, $_size457); - for ($_i461 = 0; $_i461 < $_size457; ++$_i461) - { - $elem462 = null; - $elem462 = new \metastore\Partition(); - $xfer += $elem462->read($input); - $this->success []= $elem462; - } - $xfer += $input->readListEnd(); - } else { - $xfer += $input->skip($ftype); - } - break; case 1: if ($ftype == TType::STRUCT) { - $this->o1 = new \metastore\NoSuchObjectException(); + $this->o1 = new \metastore\InvalidOperationException(); $xfer += $this->o1->read($input); } else { $xfer += $input->skip($ftype); @@ -15241,24 +19130,7 @@ class ThriftHiveMetastore_get_partitions_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_partitions_result'); - if ($this->success !== null) { - if (!is_array($this->success)) { - throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); - } - $xfer += $output->writeFieldBegin('success', TType::LST, 0); - { - $output->writeListBegin(TType::STRUCT, count($this->success)); - { - foreach ($this->success as $iter463) - { - $xfer += $iter463->write($output); - } - } - $output->writeListEnd(); - } - $xfer += $output->writeFieldEnd(); - } + $xfer += $output->writeStructBegin('ThriftHiveMetastore_rename_partition_result'); if ($this->o1 !== null) { $xfer += $output->writeFieldBegin('o1', TType::STRUCT, 1); $xfer += $this->o1->write($output); @@ -15276,65 +19148,41 @@ class ThriftHiveMetastore_get_partitions_result { } -class ThriftHiveMetastore_get_partitions_with_auth_args { +class ThriftHiveMetastore_partition_name_has_valid_characters_args { static $_TSPEC; - public $db_name = null; - public $tbl_name = null; - public $max_parts = -1; - public $user_name = null; - public $group_names = null; + public $part_vals = null; + public $throw_exception = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 1 => array( - 'var' => 'db_name', - 'type' => TType::STRING, - ), - 2 => array( - 'var' => 'tbl_name', - 'type' => TType::STRING, - ), - 3 => array( - 'var' => 'max_parts', - 'type' => TType::I16, - ), - 4 => array( - 'var' => 'user_name', - 'type' => TType::STRING, - ), - 5 => array( - 'var' => 'group_names', + 'var' => 'part_vals', 'type' => TType::LST, 'etype' => TType::STRING, 'elem' => array( 'type' => TType::STRING, ), ), + 2 => array( + 'var' => 'throw_exception', + 'type' => TType::BOOL, + ), ); } if (is_array($vals)) { - if (isset($vals['db_name'])) { - $this->db_name = $vals['db_name']; - } - if (isset($vals['tbl_name'])) { - $this->tbl_name = $vals['tbl_name']; - } - if (isset($vals['max_parts'])) { - $this->max_parts = $vals['max_parts']; - } - if (isset($vals['user_name'])) { - $this->user_name = $vals['user_name']; + if (isset($vals['part_vals'])) { + $this->part_vals = $vals['part_vals']; } - if (isset($vals['group_names'])) { - $this->group_names = $vals['group_names']; + if (isset($vals['throw_exception'])) { + $this->throw_exception = $vals['throw_exception']; } } } public function getName() { - return 'ThriftHiveMetastore_get_partitions_with_auth_args'; + return 'ThriftHiveMetastore_partition_name_has_valid_characters_args'; } public function read($input) @@ -15353,50 +19201,29 @@ class ThriftHiveMetastore_get_partitions_with_auth_args { switch ($fid) { case 1: - if ($ftype == TType::STRING) { - $xfer += $input->readString($this->db_name); - } else { - $xfer += $input->skip($ftype); - } - break; - case 2: - if ($ftype == TType::STRING) { - $xfer += $input->readString($this->tbl_name); - } else { - $xfer += $input->skip($ftype); - } - break; - case 3: - if ($ftype == TType::I16) { - $xfer += $input->readI16($this->max_parts); - } else { - $xfer += $input->skip($ftype); - } - break; - case 4: - if ($ftype == TType::STRING) { - $xfer += $input->readString($this->user_name); - } else { - $xfer += $input->skip($ftype); - } - break; - case 5: if ($ftype == TType::LST) { - $this->group_names = array(); - $_size464 = 0; - $_etype467 = 0; - $xfer += $input->readListBegin($_etype467, $_size464); - for ($_i468 = 0; $_i468 < $_size464; ++$_i468) + $this->part_vals = array(); + $_size612 = 0; + $_etype615 = 0; + $xfer += $input->readListBegin($_etype615, $_size612); + for ($_i616 = 0; $_i616 < $_size612; ++$_i616) { - $elem469 = null; - $xfer += $input->readString($elem469); - $this->group_names []= $elem469; + $elem617 = null; + $xfer += $input->readString($elem617); + $this->part_vals []= $elem617; } $xfer += $input->readListEnd(); } else { $xfer += $input->skip($ftype); } break; + case 2: + if ($ftype == TType::BOOL) { + $xfer += $input->readBool($this->throw_exception); + } else { + $xfer += $input->skip($ftype); + } + break; default: $xfer += $input->skip($ftype); break; @@ -15409,44 +19236,29 @@ class ThriftHiveMetastore_get_partitions_with_auth_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_partitions_with_auth_args'); - if ($this->db_name !== null) { - $xfer += $output->writeFieldBegin('db_name', TType::STRING, 1); - $xfer += $output->writeString($this->db_name); - $xfer += $output->writeFieldEnd(); - } - if ($this->tbl_name !== null) { - $xfer += $output->writeFieldBegin('tbl_name', TType::STRING, 2); - $xfer += $output->writeString($this->tbl_name); - $xfer += $output->writeFieldEnd(); - } - if ($this->max_parts !== null) { - $xfer += $output->writeFieldBegin('max_parts', TType::I16, 3); - $xfer += $output->writeI16($this->max_parts); - $xfer += $output->writeFieldEnd(); - } - if ($this->user_name !== null) { - $xfer += $output->writeFieldBegin('user_name', TType::STRING, 4); - $xfer += $output->writeString($this->user_name); - $xfer += $output->writeFieldEnd(); - } - if ($this->group_names !== null) { - if (!is_array($this->group_names)) { + $xfer += $output->writeStructBegin('ThriftHiveMetastore_partition_name_has_valid_characters_args'); + if ($this->part_vals !== null) { + if (!is_array($this->part_vals)) { throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); } - $xfer += $output->writeFieldBegin('group_names', TType::LST, 5); + $xfer += $output->writeFieldBegin('part_vals', TType::LST, 1); { - $output->writeListBegin(TType::STRING, count($this->group_names)); + $output->writeListBegin(TType::STRING, count($this->part_vals)); { - foreach ($this->group_names as $iter470) + foreach ($this->part_vals as $iter618) { - $xfer += $output->writeString($iter470); + $xfer += $output->writeString($iter618); } } $output->writeListEnd(); } $xfer += $output->writeFieldEnd(); } + if ($this->throw_exception !== null) { + $xfer += $output->writeFieldBegin('throw_exception', TType::BOOL, 2); + $xfer += $output->writeBool($this->throw_exception); + $xfer += $output->writeFieldEnd(); + } $xfer += $output->writeFieldStop(); $xfer += $output->writeStructEnd(); return $xfer; @@ -15454,33 +19266,22 @@ class ThriftHiveMetastore_get_partitions_with_auth_args { } -class ThriftHiveMetastore_get_partitions_with_auth_result { +class ThriftHiveMetastore_partition_name_has_valid_characters_result { static $_TSPEC; public $success = null; public $o1 = null; - public $o2 = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 0 => array( 'var' => 'success', - 'type' => TType::LST, - 'etype' => TType::STRUCT, - 'elem' => array( - 'type' => TType::STRUCT, - 'class' => '\metastore\Partition', - ), + 'type' => TType::BOOL, ), 1 => array( 'var' => 'o1', 'type' => TType::STRUCT, - 'class' => '\metastore\NoSuchObjectException', - ), - 2 => array( - 'var' => 'o2', - 'type' => TType::STRUCT, 'class' => '\metastore\MetaException', ), ); @@ -15492,14 +19293,11 @@ class ThriftHiveMetastore_get_partitions_with_auth_result { if (isset($vals['o1'])) { $this->o1 = $vals['o1']; } - if (isset($vals['o2'])) { - $this->o2 = $vals['o2']; - } } } public function getName() { - return 'ThriftHiveMetastore_get_partitions_with_auth_result'; + return 'ThriftHiveMetastore_partition_name_has_valid_characters_result'; } public function read($input) @@ -15518,39 +19316,20 @@ class ThriftHiveMetastore_get_partitions_with_auth_result { switch ($fid) { case 0: - if ($ftype == TType::LST) { - $this->success = array(); - $_size471 = 0; - $_etype474 = 0; - $xfer += $input->readListBegin($_etype474, $_size471); - for ($_i475 = 0; $_i475 < $_size471; ++$_i475) - { - $elem476 = null; - $elem476 = new \metastore\Partition(); - $xfer += $elem476->read($input); - $this->success []= $elem476; - } - $xfer += $input->readListEnd(); + if ($ftype == TType::BOOL) { + $xfer += $input->readBool($this->success); } else { $xfer += $input->skip($ftype); } break; case 1: if ($ftype == TType::STRUCT) { - $this->o1 = new \metastore\NoSuchObjectException(); + $this->o1 = new \metastore\MetaException(); $xfer += $this->o1->read($input); } else { $xfer += $input->skip($ftype); } break; - case 2: - if ($ftype == TType::STRUCT) { - $this->o2 = new \metastore\MetaException(); - $xfer += $this->o2->read($input); - } else { - $xfer += $input->skip($ftype); - } - break; default: $xfer += $input->skip($ftype); break; @@ -15563,22 +19342,10 @@ class ThriftHiveMetastore_get_partitions_with_auth_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_partitions_with_auth_result'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_partition_name_has_valid_characters_result'); if ($this->success !== null) { - if (!is_array($this->success)) { - throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); - } - $xfer += $output->writeFieldBegin('success', TType::LST, 0); - { - $output->writeListBegin(TType::STRUCT, count($this->success)); - { - foreach ($this->success as $iter477) - { - $xfer += $iter477->write($output); - } - } - $output->writeListEnd(); - } + $xfer += $output->writeFieldBegin('success', TType::BOOL, 0); + $xfer += $output->writeBool($this->success); $xfer += $output->writeFieldEnd(); } if ($this->o1 !== null) { @@ -15586,11 +19353,6 @@ class ThriftHiveMetastore_get_partitions_with_auth_result { $xfer += $this->o1->write($output); $xfer += $output->writeFieldEnd(); } - if ($this->o2 !== null) { - $xfer += $output->writeFieldBegin('o2', TType::STRUCT, 2); - $xfer += $this->o2->write($output); - $xfer += $output->writeFieldEnd(); - } $xfer += $output->writeFieldStop(); $xfer += $output->writeStructEnd(); return $xfer; @@ -15598,45 +19360,37 @@ class ThriftHiveMetastore_get_partitions_with_auth_result { } -class ThriftHiveMetastore_get_partition_names_args { +class ThriftHiveMetastore_get_config_value_args { static $_TSPEC; - public $db_name = null; - public $tbl_name = null; - public $max_parts = -1; + public $name = null; + public $defaultValue = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 1 => array( - 'var' => 'db_name', + 'var' => 'name', 'type' => TType::STRING, ), 2 => array( - 'var' => 'tbl_name', + 'var' => 'defaultValue', 'type' => TType::STRING, ), - 3 => array( - 'var' => 'max_parts', - 'type' => TType::I16, - ), ); } if (is_array($vals)) { - if (isset($vals['db_name'])) { - $this->db_name = $vals['db_name']; - } - if (isset($vals['tbl_name'])) { - $this->tbl_name = $vals['tbl_name']; + if (isset($vals['name'])) { + $this->name = $vals['name']; } - if (isset($vals['max_parts'])) { - $this->max_parts = $vals['max_parts']; + if (isset($vals['defaultValue'])) { + $this->defaultValue = $vals['defaultValue']; } } } public function getName() { - return 'ThriftHiveMetastore_get_partition_names_args'; + return 'ThriftHiveMetastore_get_config_value_args'; } public function read($input) @@ -15656,21 +19410,14 @@ class ThriftHiveMetastore_get_partition_names_args { { case 1: if ($ftype == TType::STRING) { - $xfer += $input->readString($this->db_name); + $xfer += $input->readString($this->name); } else { $xfer += $input->skip($ftype); } break; case 2: if ($ftype == TType::STRING) { - $xfer += $input->readString($this->tbl_name); - } else { - $xfer += $input->skip($ftype); - } - break; - case 3: - if ($ftype == TType::I16) { - $xfer += $input->readI16($this->max_parts); + $xfer += $input->readString($this->defaultValue); } else { $xfer += $input->skip($ftype); } @@ -15687,20 +19434,15 @@ class ThriftHiveMetastore_get_partition_names_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_partition_names_args'); - if ($this->db_name !== null) { - $xfer += $output->writeFieldBegin('db_name', TType::STRING, 1); - $xfer += $output->writeString($this->db_name); - $xfer += $output->writeFieldEnd(); - } - if ($this->tbl_name !== null) { - $xfer += $output->writeFieldBegin('tbl_name', TType::STRING, 2); - $xfer += $output->writeString($this->tbl_name); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_config_value_args'); + if ($this->name !== null) { + $xfer += $output->writeFieldBegin('name', TType::STRING, 1); + $xfer += $output->writeString($this->name); $xfer += $output->writeFieldEnd(); } - if ($this->max_parts !== null) { - $xfer += $output->writeFieldBegin('max_parts', TType::I16, 3); - $xfer += $output->writeI16($this->max_parts); + if ($this->defaultValue !== null) { + $xfer += $output->writeFieldBegin('defaultValue', TType::STRING, 2); + $xfer += $output->writeString($this->defaultValue); $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -15710,27 +19452,23 @@ class ThriftHiveMetastore_get_partition_names_args { } -class ThriftHiveMetastore_get_partition_names_result { +class ThriftHiveMetastore_get_config_value_result { static $_TSPEC; public $success = null; - public $o2 = null; + public $o1 = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 0 => array( 'var' => 'success', - 'type' => TType::LST, - 'etype' => TType::STRING, - 'elem' => array( - 'type' => TType::STRING, - ), + 'type' => TType::STRING, ), 1 => array( - 'var' => 'o2', + 'var' => 'o1', 'type' => TType::STRUCT, - 'class' => '\metastore\MetaException', + 'class' => '\metastore\ConfigValSecurityException', ), ); } @@ -15738,14 +19476,14 @@ class ThriftHiveMetastore_get_partition_names_result { if (isset($vals['success'])) { $this->success = $vals['success']; } - if (isset($vals['o2'])) { - $this->o2 = $vals['o2']; + if (isset($vals['o1'])) { + $this->o1 = $vals['o1']; } } } public function getName() { - return 'ThriftHiveMetastore_get_partition_names_result'; + return 'ThriftHiveMetastore_get_config_value_result'; } public function read($input) @@ -15764,26 +19502,16 @@ class ThriftHiveMetastore_get_partition_names_result { switch ($fid) { case 0: - if ($ftype == TType::LST) { - $this->success = array(); - $_size478 = 0; - $_etype481 = 0; - $xfer += $input->readListBegin($_etype481, $_size478); - for ($_i482 = 0; $_i482 < $_size478; ++$_i482) - { - $elem483 = null; - $xfer += $input->readString($elem483); - $this->success []= $elem483; - } - $xfer += $input->readListEnd(); + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->success); } else { $xfer += $input->skip($ftype); } break; case 1: if ($ftype == TType::STRUCT) { - $this->o2 = new \metastore\MetaException(); - $xfer += $this->o2->read($input); + $this->o1 = new \metastore\ConfigValSecurityException(); + $xfer += $this->o1->read($input); } else { $xfer += $input->skip($ftype); } @@ -15800,27 +19528,15 @@ class ThriftHiveMetastore_get_partition_names_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_partition_names_result'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_config_value_result'); if ($this->success !== null) { - if (!is_array($this->success)) { - throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); - } - $xfer += $output->writeFieldBegin('success', TType::LST, 0); - { - $output->writeListBegin(TType::STRING, count($this->success)); - { - foreach ($this->success as $iter484) - { - $xfer += $output->writeString($iter484); - } - } - $output->writeListEnd(); - } + $xfer += $output->writeFieldBegin('success', TType::STRING, 0); + $xfer += $output->writeString($this->success); $xfer += $output->writeFieldEnd(); } - if ($this->o2 !== null) { - $xfer += $output->writeFieldBegin('o2', TType::STRUCT, 1); - $xfer += $this->o2->write($output); + if ($this->o1 !== null) { + $xfer += $output->writeFieldBegin('o1', TType::STRUCT, 1); + $xfer += $this->o1->write($output); $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -15830,57 +19546,29 @@ class ThriftHiveMetastore_get_partition_names_result { } -class ThriftHiveMetastore_get_partitions_ps_args { +class ThriftHiveMetastore_partition_name_to_vals_args { static $_TSPEC; - public $db_name = null; - public $tbl_name = null; - public $part_vals = null; - public $max_parts = -1; + public $part_name = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 1 => array( - 'var' => 'db_name', - 'type' => TType::STRING, - ), - 2 => array( - 'var' => 'tbl_name', + 'var' => 'part_name', 'type' => TType::STRING, ), - 3 => array( - 'var' => 'part_vals', - 'type' => TType::LST, - 'etype' => TType::STRING, - 'elem' => array( - 'type' => TType::STRING, - ), - ), - 4 => array( - 'var' => 'max_parts', - 'type' => TType::I16, - ), ); } if (is_array($vals)) { - if (isset($vals['db_name'])) { - $this->db_name = $vals['db_name']; - } - if (isset($vals['tbl_name'])) { - $this->tbl_name = $vals['tbl_name']; - } - if (isset($vals['part_vals'])) { - $this->part_vals = $vals['part_vals']; - } - if (isset($vals['max_parts'])) { - $this->max_parts = $vals['max_parts']; + if (isset($vals['part_name'])) { + $this->part_name = $vals['part_name']; } } } public function getName() { - return 'ThriftHiveMetastore_get_partitions_ps_args'; + return 'ThriftHiveMetastore_partition_name_to_vals_args'; } public function read($input) @@ -15897,41 +19585,10 @@ class ThriftHiveMetastore_get_partitions_ps_args { break; } switch ($fid) - { - case 1: - if ($ftype == TType::STRING) { - $xfer += $input->readString($this->db_name); - } else { - $xfer += $input->skip($ftype); - } - break; - case 2: + { + case 1: if ($ftype == TType::STRING) { - $xfer += $input->readString($this->tbl_name); - } else { - $xfer += $input->skip($ftype); - } - break; - case 3: - if ($ftype == TType::LST) { - $this->part_vals = array(); - $_size485 = 0; - $_etype488 = 0; - $xfer += $input->readListBegin($_etype488, $_size485); - for ($_i489 = 0; $_i489 < $_size485; ++$_i489) - { - $elem490 = null; - $xfer += $input->readString($elem490); - $this->part_vals []= $elem490; - } - $xfer += $input->readListEnd(); - } else { - $xfer += $input->skip($ftype); - } - break; - case 4: - if ($ftype == TType::I16) { - $xfer += $input->readI16($this->max_parts); + $xfer += $input->readString($this->part_name); } else { $xfer += $input->skip($ftype); } @@ -15948,37 +19605,10 @@ class ThriftHiveMetastore_get_partitions_ps_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_partitions_ps_args'); - if ($this->db_name !== null) { - $xfer += $output->writeFieldBegin('db_name', TType::STRING, 1); - $xfer += $output->writeString($this->db_name); - $xfer += $output->writeFieldEnd(); - } - if ($this->tbl_name !== null) { - $xfer += $output->writeFieldBegin('tbl_name', TType::STRING, 2); - $xfer += $output->writeString($this->tbl_name); - $xfer += $output->writeFieldEnd(); - } - if ($this->part_vals !== null) { - if (!is_array($this->part_vals)) { - throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); - } - $xfer += $output->writeFieldBegin('part_vals', TType::LST, 3); - { - $output->writeListBegin(TType::STRING, count($this->part_vals)); - { - foreach ($this->part_vals as $iter491) - { - $xfer += $output->writeString($iter491); - } - } - $output->writeListEnd(); - } - $xfer += $output->writeFieldEnd(); - } - if ($this->max_parts !== null) { - $xfer += $output->writeFieldBegin('max_parts', TType::I16, 4); - $xfer += $output->writeI16($this->max_parts); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_partition_name_to_vals_args'); + if ($this->part_name !== null) { + $xfer += $output->writeFieldBegin('part_name', TType::STRING, 1); + $xfer += $output->writeString($this->part_name); $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -15988,12 +19618,11 @@ class ThriftHiveMetastore_get_partitions_ps_args { } -class ThriftHiveMetastore_get_partitions_ps_result { +class ThriftHiveMetastore_partition_name_to_vals_result { static $_TSPEC; public $success = null; public $o1 = null; - public $o2 = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { @@ -16001,10 +19630,9 @@ class ThriftHiveMetastore_get_partitions_ps_result { 0 => array( 'var' => 'success', 'type' => TType::LST, - 'etype' => TType::STRUCT, + 'etype' => TType::STRING, 'elem' => array( - 'type' => TType::STRUCT, - 'class' => '\metastore\Partition', + 'type' => TType::STRING, ), ), 1 => array( @@ -16012,11 +19640,6 @@ class ThriftHiveMetastore_get_partitions_ps_result { 'type' => TType::STRUCT, 'class' => '\metastore\MetaException', ), - 2 => array( - 'var' => 'o2', - 'type' => TType::STRUCT, - 'class' => '\metastore\NoSuchObjectException', - ), ); } if (is_array($vals)) { @@ -16026,14 +19649,11 @@ class ThriftHiveMetastore_get_partitions_ps_result { if (isset($vals['o1'])) { $this->o1 = $vals['o1']; } - if (isset($vals['o2'])) { - $this->o2 = $vals['o2']; - } } } public function getName() { - return 'ThriftHiveMetastore_get_partitions_ps_result'; + return 'ThriftHiveMetastore_partition_name_to_vals_result'; } public function read($input) @@ -16054,15 +19674,14 @@ class ThriftHiveMetastore_get_partitions_ps_result { case 0: if ($ftype == TType::LST) { $this->success = array(); - $_size492 = 0; - $_etype495 = 0; - $xfer += $input->readListBegin($_etype495, $_size492); - for ($_i496 = 0; $_i496 < $_size492; ++$_i496) + $_size619 = 0; + $_etype622 = 0; + $xfer += $input->readListBegin($_etype622, $_size619); + for ($_i623 = 0; $_i623 < $_size619; ++$_i623) { - $elem497 = null; - $elem497 = new \metastore\Partition(); - $xfer += $elem497->read($input); - $this->success []= $elem497; + $elem624 = null; + $xfer += $input->readString($elem624); + $this->success []= $elem624; } $xfer += $input->readListEnd(); } else { @@ -16077,14 +19696,6 @@ class ThriftHiveMetastore_get_partitions_ps_result { $xfer += $input->skip($ftype); } break; - case 2: - if ($ftype == TType::STRUCT) { - $this->o2 = new \metastore\NoSuchObjectException(); - $xfer += $this->o2->read($input); - } else { - $xfer += $input->skip($ftype); - } - break; default: $xfer += $input->skip($ftype); break; @@ -16097,18 +19708,18 @@ class ThriftHiveMetastore_get_partitions_ps_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_partitions_ps_result'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_partition_name_to_vals_result'); if ($this->success !== null) { if (!is_array($this->success)) { throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); } $xfer += $output->writeFieldBegin('success', TType::LST, 0); { - $output->writeListBegin(TType::STRUCT, count($this->success)); + $output->writeListBegin(TType::STRING, count($this->success)); { - foreach ($this->success as $iter498) + foreach ($this->success as $iter625) { - $xfer += $iter498->write($output); + $xfer += $output->writeString($iter625); } } $output->writeListEnd(); @@ -16120,11 +19731,6 @@ class ThriftHiveMetastore_get_partitions_ps_result { $xfer += $this->o1->write($output); $xfer += $output->writeFieldEnd(); } - if ($this->o2 !== null) { - $xfer += $output->writeFieldBegin('o2', TType::STRUCT, 2); - $xfer += $this->o2->write($output); - $xfer += $output->writeFieldEnd(); - } $xfer += $output->writeFieldStop(); $xfer += $output->writeStructEnd(); return $xfer; @@ -16132,77 +19738,29 @@ class ThriftHiveMetastore_get_partitions_ps_result { } -class ThriftHiveMetastore_get_partitions_ps_with_auth_args { +class ThriftHiveMetastore_partition_name_to_spec_args { static $_TSPEC; - public $db_name = null; - public $tbl_name = null; - public $part_vals = null; - public $max_parts = -1; - public $user_name = null; - public $group_names = null; + public $part_name = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 1 => array( - 'var' => 'db_name', - 'type' => TType::STRING, - ), - 2 => array( - 'var' => 'tbl_name', - 'type' => TType::STRING, - ), - 3 => array( - 'var' => 'part_vals', - 'type' => TType::LST, - 'etype' => TType::STRING, - 'elem' => array( - 'type' => TType::STRING, - ), - ), - 4 => array( - 'var' => 'max_parts', - 'type' => TType::I16, - ), - 5 => array( - 'var' => 'user_name', + 'var' => 'part_name', 'type' => TType::STRING, ), - 6 => array( - 'var' => 'group_names', - 'type' => TType::LST, - 'etype' => TType::STRING, - 'elem' => array( - 'type' => TType::STRING, - ), - ), ); } if (is_array($vals)) { - if (isset($vals['db_name'])) { - $this->db_name = $vals['db_name']; - } - if (isset($vals['tbl_name'])) { - $this->tbl_name = $vals['tbl_name']; - } - if (isset($vals['part_vals'])) { - $this->part_vals = $vals['part_vals']; - } - if (isset($vals['max_parts'])) { - $this->max_parts = $vals['max_parts']; - } - if (isset($vals['user_name'])) { - $this->user_name = $vals['user_name']; - } - if (isset($vals['group_names'])) { - $this->group_names = $vals['group_names']; + if (isset($vals['part_name'])) { + $this->part_name = $vals['part_name']; } } } public function getName() { - return 'ThriftHiveMetastore_get_partitions_ps_with_auth_args'; + return 'ThriftHiveMetastore_partition_name_to_spec_args'; } public function read($input) @@ -16218,135 +19776,31 @@ class ThriftHiveMetastore_get_partitions_ps_with_auth_args { if ($ftype == TType::STOP) { break; } - switch ($fid) - { - case 1: - if ($ftype == TType::STRING) { - $xfer += $input->readString($this->db_name); - } else { - $xfer += $input->skip($ftype); - } - break; - case 2: - if ($ftype == TType::STRING) { - $xfer += $input->readString($this->tbl_name); - } else { - $xfer += $input->skip($ftype); - } - break; - case 3: - if ($ftype == TType::LST) { - $this->part_vals = array(); - $_size499 = 0; - $_etype502 = 0; - $xfer += $input->readListBegin($_etype502, $_size499); - for ($_i503 = 0; $_i503 < $_size499; ++$_i503) - { - $elem504 = null; - $xfer += $input->readString($elem504); - $this->part_vals []= $elem504; - } - $xfer += $input->readListEnd(); - } else { - $xfer += $input->skip($ftype); - } - break; - case 4: - if ($ftype == TType::I16) { - $xfer += $input->readI16($this->max_parts); - } else { - $xfer += $input->skip($ftype); - } - break; - case 5: - if ($ftype == TType::STRING) { - $xfer += $input->readString($this->user_name); - } else { - $xfer += $input->skip($ftype); - } - break; - case 6: - if ($ftype == TType::LST) { - $this->group_names = array(); - $_size505 = 0; - $_etype508 = 0; - $xfer += $input->readListBegin($_etype508, $_size505); - for ($_i509 = 0; $_i509 < $_size505; ++$_i509) - { - $elem510 = null; - $xfer += $input->readString($elem510); - $this->group_names []= $elem510; - } - $xfer += $input->readListEnd(); - } else { - $xfer += $input->skip($ftype); - } - break; - default: - $xfer += $input->skip($ftype); - break; - } - $xfer += $input->readFieldEnd(); - } - $xfer += $input->readStructEnd(); - return $xfer; - } - - public function write($output) { - $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_partitions_ps_with_auth_args'); - if ($this->db_name !== null) { - $xfer += $output->writeFieldBegin('db_name', TType::STRING, 1); - $xfer += $output->writeString($this->db_name); - $xfer += $output->writeFieldEnd(); - } - if ($this->tbl_name !== null) { - $xfer += $output->writeFieldBegin('tbl_name', TType::STRING, 2); - $xfer += $output->writeString($this->tbl_name); - $xfer += $output->writeFieldEnd(); - } - if ($this->part_vals !== null) { - if (!is_array($this->part_vals)) { - throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); - } - $xfer += $output->writeFieldBegin('part_vals', TType::LST, 3); - { - $output->writeListBegin(TType::STRING, count($this->part_vals)); - { - foreach ($this->part_vals as $iter511) - { - $xfer += $output->writeString($iter511); - } - } - $output->writeListEnd(); - } - $xfer += $output->writeFieldEnd(); - } - if ($this->max_parts !== null) { - $xfer += $output->writeFieldBegin('max_parts', TType::I16, 4); - $xfer += $output->writeI16($this->max_parts); - $xfer += $output->writeFieldEnd(); - } - if ($this->user_name !== null) { - $xfer += $output->writeFieldBegin('user_name', TType::STRING, 5); - $xfer += $output->writeString($this->user_name); - $xfer += $output->writeFieldEnd(); - } - if ($this->group_names !== null) { - if (!is_array($this->group_names)) { - throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); - } - $xfer += $output->writeFieldBegin('group_names', TType::LST, 6); + switch ($fid) { - $output->writeListBegin(TType::STRING, count($this->group_names)); - { - foreach ($this->group_names as $iter512) - { - $xfer += $output->writeString($iter512); + case 1: + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->part_name); + } else { + $xfer += $input->skip($ftype); } - } - $output->writeListEnd(); + break; + default: + $xfer += $input->skip($ftype); + break; } + $xfer += $input->readFieldEnd(); + } + $xfer += $input->readStructEnd(); + return $xfer; + } + + public function write($output) { + $xfer = 0; + $xfer += $output->writeStructBegin('ThriftHiveMetastore_partition_name_to_spec_args'); + if ($this->part_name !== null) { + $xfer += $output->writeFieldBegin('part_name', TType::STRING, 1); + $xfer += $output->writeString($this->part_name); $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -16356,33 +19810,30 @@ class ThriftHiveMetastore_get_partitions_ps_with_auth_args { } -class ThriftHiveMetastore_get_partitions_ps_with_auth_result { +class ThriftHiveMetastore_partition_name_to_spec_result { static $_TSPEC; public $success = null; public $o1 = null; - public $o2 = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 0 => array( 'var' => 'success', - 'type' => TType::LST, - 'etype' => TType::STRUCT, - 'elem' => array( - 'type' => TType::STRUCT, - 'class' => '\metastore\Partition', + 'type' => TType::MAP, + 'ktype' => TType::STRING, + 'vtype' => TType::STRING, + 'key' => array( + 'type' => TType::STRING, + ), + 'val' => array( + 'type' => TType::STRING, ), ), 1 => array( 'var' => 'o1', 'type' => TType::STRUCT, - 'class' => '\metastore\NoSuchObjectException', - ), - 2 => array( - 'var' => 'o2', - 'type' => TType::STRUCT, 'class' => '\metastore\MetaException', ), ); @@ -16394,14 +19845,11 @@ class ThriftHiveMetastore_get_partitions_ps_with_auth_result { if (isset($vals['o1'])) { $this->o1 = $vals['o1']; } - if (isset($vals['o2'])) { - $this->o2 = $vals['o2']; - } } } public function getName() { - return 'ThriftHiveMetastore_get_partitions_ps_with_auth_result'; + return 'ThriftHiveMetastore_partition_name_to_spec_result'; } public function read($input) @@ -16420,39 +19868,33 @@ class ThriftHiveMetastore_get_partitions_ps_with_auth_result { switch ($fid) { case 0: - if ($ftype == TType::LST) { + if ($ftype == TType::MAP) { $this->success = array(); - $_size513 = 0; - $_etype516 = 0; - $xfer += $input->readListBegin($_etype516, $_size513); - for ($_i517 = 0; $_i517 < $_size513; ++$_i517) + $_size626 = 0; + $_ktype627 = 0; + $_vtype628 = 0; + $xfer += $input->readMapBegin($_ktype627, $_vtype628, $_size626); + for ($_i630 = 0; $_i630 < $_size626; ++$_i630) { - $elem518 = null; - $elem518 = new \metastore\Partition(); - $xfer += $elem518->read($input); - $this->success []= $elem518; + $key631 = ''; + $val632 = ''; + $xfer += $input->readString($key631); + $xfer += $input->readString($val632); + $this->success[$key631] = $val632; } - $xfer += $input->readListEnd(); + $xfer += $input->readMapEnd(); } else { $xfer += $input->skip($ftype); } break; case 1: if ($ftype == TType::STRUCT) { - $this->o1 = new \metastore\NoSuchObjectException(); + $this->o1 = new \metastore\MetaException(); $xfer += $this->o1->read($input); } else { $xfer += $input->skip($ftype); } break; - case 2: - if ($ftype == TType::STRUCT) { - $this->o2 = new \metastore\MetaException(); - $xfer += $this->o2->read($input); - } else { - $xfer += $input->skip($ftype); - } - break; default: $xfer += $input->skip($ftype); break; @@ -16465,21 +19907,22 @@ class ThriftHiveMetastore_get_partitions_ps_with_auth_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_partitions_ps_with_auth_result'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_partition_name_to_spec_result'); if ($this->success !== null) { if (!is_array($this->success)) { throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); } - $xfer += $output->writeFieldBegin('success', TType::LST, 0); + $xfer += $output->writeFieldBegin('success', TType::MAP, 0); { - $output->writeListBegin(TType::STRUCT, count($this->success)); + $output->writeMapBegin(TType::STRING, TType::STRING, count($this->success)); { - foreach ($this->success as $iter519) + foreach ($this->success as $kiter633 => $viter634) { - $xfer += $iter519->write($output); + $xfer += $output->writeString($kiter633); + $xfer += $output->writeString($viter634); } } - $output->writeListEnd(); + $output->writeMapEnd(); } $xfer += $output->writeFieldEnd(); } @@ -16488,11 +19931,6 @@ class ThriftHiveMetastore_get_partitions_ps_with_auth_result { $xfer += $this->o1->write($output); $xfer += $output->writeFieldEnd(); } - if ($this->o2 !== null) { - $xfer += $output->writeFieldBegin('o2', TType::STRUCT, 2); - $xfer += $this->o2->write($output); - $xfer += $output->writeFieldEnd(); - } $xfer += $output->writeFieldStop(); $xfer += $output->writeStructEnd(); return $xfer; @@ -16500,13 +19938,13 @@ class ThriftHiveMetastore_get_partitions_ps_with_auth_result { } -class ThriftHiveMetastore_get_partition_names_ps_args { +class ThriftHiveMetastore_markPartitionForEvent_args { static $_TSPEC; public $db_name = null; public $tbl_name = null; public $part_vals = null; - public $max_parts = -1; + public $eventType = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { @@ -16521,15 +19959,19 @@ class ThriftHiveMetastore_get_partition_names_ps_args { ), 3 => array( 'var' => 'part_vals', - 'type' => TType::LST, - 'etype' => TType::STRING, - 'elem' => array( + 'type' => TType::MAP, + 'ktype' => TType::STRING, + 'vtype' => TType::STRING, + 'key' => array( + 'type' => TType::STRING, + ), + 'val' => array( 'type' => TType::STRING, ), ), 4 => array( - 'var' => 'max_parts', - 'type' => TType::I16, + 'var' => 'eventType', + 'type' => TType::I32, ), ); } @@ -16543,14 +19985,14 @@ class ThriftHiveMetastore_get_partition_names_ps_args { if (isset($vals['part_vals'])) { $this->part_vals = $vals['part_vals']; } - if (isset($vals['max_parts'])) { - $this->max_parts = $vals['max_parts']; + if (isset($vals['eventType'])) { + $this->eventType = $vals['eventType']; } } } public function getName() { - return 'ThriftHiveMetastore_get_partition_names_ps_args'; + return 'ThriftHiveMetastore_markPartitionForEvent_args'; } public function read($input) @@ -16583,25 +20025,28 @@ class ThriftHiveMetastore_get_partition_names_ps_args { } break; case 3: - if ($ftype == TType::LST) { + if ($ftype == TType::MAP) { $this->part_vals = array(); - $_size520 = 0; - $_etype523 = 0; - $xfer += $input->readListBegin($_etype523, $_size520); - for ($_i524 = 0; $_i524 < $_size520; ++$_i524) + $_size635 = 0; + $_ktype636 = 0; + $_vtype637 = 0; + $xfer += $input->readMapBegin($_ktype636, $_vtype637, $_size635); + for ($_i639 = 0; $_i639 < $_size635; ++$_i639) { - $elem525 = null; - $xfer += $input->readString($elem525); - $this->part_vals []= $elem525; + $key640 = ''; + $val641 = ''; + $xfer += $input->readString($key640); + $xfer += $input->readString($val641); + $this->part_vals[$key640] = $val641; } - $xfer += $input->readListEnd(); + $xfer += $input->readMapEnd(); } else { $xfer += $input->skip($ftype); } break; case 4: - if ($ftype == TType::I16) { - $xfer += $input->readI16($this->max_parts); + if ($ftype == TType::I32) { + $xfer += $input->readI32($this->eventType); } else { $xfer += $input->skip($ftype); } @@ -16618,7 +20063,7 @@ class ThriftHiveMetastore_get_partition_names_ps_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_partition_names_ps_args'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_markPartitionForEvent_args'); if ($this->db_name !== null) { $xfer += $output->writeFieldBegin('db_name', TType::STRING, 1); $xfer += $output->writeString($this->db_name); @@ -16633,22 +20078,23 @@ class ThriftHiveMetastore_get_partition_names_ps_args { if (!is_array($this->part_vals)) { throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); } - $xfer += $output->writeFieldBegin('part_vals', TType::LST, 3); + $xfer += $output->writeFieldBegin('part_vals', TType::MAP, 3); { - $output->writeListBegin(TType::STRING, count($this->part_vals)); + $output->writeMapBegin(TType::STRING, TType::STRING, count($this->part_vals)); { - foreach ($this->part_vals as $iter526) + foreach ($this->part_vals as $kiter642 => $viter643) { - $xfer += $output->writeString($iter526); + $xfer += $output->writeString($kiter642); + $xfer += $output->writeString($viter643); } } - $output->writeListEnd(); + $output->writeMapEnd(); } $xfer += $output->writeFieldEnd(); } - if ($this->max_parts !== null) { - $xfer += $output->writeFieldBegin('max_parts', TType::I16, 4); - $xfer += $output->writeI16($this->max_parts); + if ($this->eventType !== null) { + $xfer += $output->writeFieldBegin('eventType', TType::I32, 4); + $xfer += $output->writeI32($this->eventType); $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -16658,24 +20104,19 @@ class ThriftHiveMetastore_get_partition_names_ps_args { } -class ThriftHiveMetastore_get_partition_names_ps_result { +class ThriftHiveMetastore_markPartitionForEvent_result { static $_TSPEC; - public $success = null; public $o1 = null; public $o2 = null; + public $o3 = null; + public $o4 = null; + public $o5 = null; + public $o6 = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( - 0 => array( - 'var' => 'success', - 'type' => TType::LST, - 'etype' => TType::STRING, - 'elem' => array( - 'type' => TType::STRING, - ), - ), 1 => array( 'var' => 'o1', 'type' => TType::STRUCT, @@ -16686,23 +20127,52 @@ class ThriftHiveMetastore_get_partition_names_ps_result { 'type' => TType::STRUCT, 'class' => '\metastore\NoSuchObjectException', ), + 3 => array( + 'var' => 'o3', + 'type' => TType::STRUCT, + 'class' => '\metastore\UnknownDBException', + ), + 4 => array( + 'var' => 'o4', + 'type' => TType::STRUCT, + 'class' => '\metastore\UnknownTableException', + ), + 5 => array( + 'var' => 'o5', + 'type' => TType::STRUCT, + 'class' => '\metastore\UnknownPartitionException', + ), + 6 => array( + 'var' => 'o6', + 'type' => TType::STRUCT, + 'class' => '\metastore\InvalidPartitionException', + ), ); } if (is_array($vals)) { - if (isset($vals['success'])) { - $this->success = $vals['success']; - } if (isset($vals['o1'])) { $this->o1 = $vals['o1']; } if (isset($vals['o2'])) { $this->o2 = $vals['o2']; } + if (isset($vals['o3'])) { + $this->o3 = $vals['o3']; + } + if (isset($vals['o4'])) { + $this->o4 = $vals['o4']; + } + if (isset($vals['o5'])) { + $this->o5 = $vals['o5']; + } + if (isset($vals['o6'])) { + $this->o6 = $vals['o6']; + } } } public function getName() { - return 'ThriftHiveMetastore_get_partition_names_ps_result'; + return 'ThriftHiveMetastore_markPartitionForEvent_result'; } public function read($input) @@ -16720,23 +20190,6 @@ class ThriftHiveMetastore_get_partition_names_ps_result { } switch ($fid) { - case 0: - if ($ftype == TType::LST) { - $this->success = array(); - $_size527 = 0; - $_etype530 = 0; - $xfer += $input->readListBegin($_etype530, $_size527); - for ($_i531 = 0; $_i531 < $_size527; ++$_i531) - { - $elem532 = null; - $xfer += $input->readString($elem532); - $this->success []= $elem532; - } - $xfer += $input->readListEnd(); - } else { - $xfer += $input->skip($ftype); - } - break; case 1: if ($ftype == TType::STRUCT) { $this->o1 = new \metastore\MetaException(); @@ -16753,6 +20206,38 @@ class ThriftHiveMetastore_get_partition_names_ps_result { $xfer += $input->skip($ftype); } break; + case 3: + if ($ftype == TType::STRUCT) { + $this->o3 = new \metastore\UnknownDBException(); + $xfer += $this->o3->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; + case 4: + if ($ftype == TType::STRUCT) { + $this->o4 = new \metastore\UnknownTableException(); + $xfer += $this->o4->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; + case 5: + if ($ftype == TType::STRUCT) { + $this->o5 = new \metastore\UnknownPartitionException(); + $xfer += $this->o5->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; + case 6: + if ($ftype == TType::STRUCT) { + $this->o6 = new \metastore\InvalidPartitionException(); + $xfer += $this->o6->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; default: $xfer += $input->skip($ftype); break; @@ -16765,24 +20250,7 @@ class ThriftHiveMetastore_get_partition_names_ps_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_partition_names_ps_result'); - if ($this->success !== null) { - if (!is_array($this->success)) { - throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); - } - $xfer += $output->writeFieldBegin('success', TType::LST, 0); - { - $output->writeListBegin(TType::STRING, count($this->success)); - { - foreach ($this->success as $iter533) - { - $xfer += $output->writeString($iter533); - } - } - $output->writeListEnd(); - } - $xfer += $output->writeFieldEnd(); - } + $xfer += $output->writeStructBegin('ThriftHiveMetastore_markPartitionForEvent_result'); if ($this->o1 !== null) { $xfer += $output->writeFieldBegin('o1', TType::STRUCT, 1); $xfer += $this->o1->write($output); @@ -16793,6 +20261,26 @@ class ThriftHiveMetastore_get_partition_names_ps_result { $xfer += $this->o2->write($output); $xfer += $output->writeFieldEnd(); } + if ($this->o3 !== null) { + $xfer += $output->writeFieldBegin('o3', TType::STRUCT, 3); + $xfer += $this->o3->write($output); + $xfer += $output->writeFieldEnd(); + } + if ($this->o4 !== null) { + $xfer += $output->writeFieldBegin('o4', TType::STRUCT, 4); + $xfer += $this->o4->write($output); + $xfer += $output->writeFieldEnd(); + } + if ($this->o5 !== null) { + $xfer += $output->writeFieldBegin('o5', TType::STRUCT, 5); + $xfer += $this->o5->write($output); + $xfer += $output->writeFieldEnd(); + } + if ($this->o6 !== null) { + $xfer += $output->writeFieldBegin('o6', TType::STRUCT, 6); + $xfer += $this->o6->write($output); + $xfer += $output->writeFieldEnd(); + } $xfer += $output->writeFieldStop(); $xfer += $output->writeStructEnd(); return $xfer; @@ -16800,13 +20288,13 @@ class ThriftHiveMetastore_get_partition_names_ps_result { } -class ThriftHiveMetastore_get_partitions_by_filter_args { +class ThriftHiveMetastore_isPartitionMarkedForEvent_args { static $_TSPEC; public $db_name = null; public $tbl_name = null; - public $filter = null; - public $max_parts = -1; + public $part_vals = null; + public $eventType = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { @@ -16820,12 +20308,20 @@ class ThriftHiveMetastore_get_partitions_by_filter_args { 'type' => TType::STRING, ), 3 => array( - 'var' => 'filter', - 'type' => TType::STRING, + 'var' => 'part_vals', + 'type' => TType::MAP, + 'ktype' => TType::STRING, + 'vtype' => TType::STRING, + 'key' => array( + 'type' => TType::STRING, + ), + 'val' => array( + 'type' => TType::STRING, + ), ), 4 => array( - 'var' => 'max_parts', - 'type' => TType::I16, + 'var' => 'eventType', + 'type' => TType::I32, ), ); } @@ -16836,17 +20332,17 @@ class ThriftHiveMetastore_get_partitions_by_filter_args { if (isset($vals['tbl_name'])) { $this->tbl_name = $vals['tbl_name']; } - if (isset($vals['filter'])) { - $this->filter = $vals['filter']; + if (isset($vals['part_vals'])) { + $this->part_vals = $vals['part_vals']; } - if (isset($vals['max_parts'])) { - $this->max_parts = $vals['max_parts']; + if (isset($vals['eventType'])) { + $this->eventType = $vals['eventType']; } } } public function getName() { - return 'ThriftHiveMetastore_get_partitions_by_filter_args'; + return 'ThriftHiveMetastore_isPartitionMarkedForEvent_args'; } public function read($input) @@ -16879,15 +20375,28 @@ class ThriftHiveMetastore_get_partitions_by_filter_args { } break; case 3: - if ($ftype == TType::STRING) { - $xfer += $input->readString($this->filter); + if ($ftype == TType::MAP) { + $this->part_vals = array(); + $_size644 = 0; + $_ktype645 = 0; + $_vtype646 = 0; + $xfer += $input->readMapBegin($_ktype645, $_vtype646, $_size644); + for ($_i648 = 0; $_i648 < $_size644; ++$_i648) + { + $key649 = ''; + $val650 = ''; + $xfer += $input->readString($key649); + $xfer += $input->readString($val650); + $this->part_vals[$key649] = $val650; + } + $xfer += $input->readMapEnd(); } else { $xfer += $input->skip($ftype); } break; case 4: - if ($ftype == TType::I16) { - $xfer += $input->readI16($this->max_parts); + if ($ftype == TType::I32) { + $xfer += $input->readI32($this->eventType); } else { $xfer += $input->skip($ftype); } @@ -16904,7 +20413,7 @@ class ThriftHiveMetastore_get_partitions_by_filter_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_partitions_by_filter_args'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_isPartitionMarkedForEvent_args'); if ($this->db_name !== null) { $xfer += $output->writeFieldBegin('db_name', TType::STRING, 1); $xfer += $output->writeString($this->db_name); @@ -16915,14 +20424,27 @@ class ThriftHiveMetastore_get_partitions_by_filter_args { $xfer += $output->writeString($this->tbl_name); $xfer += $output->writeFieldEnd(); } - if ($this->filter !== null) { - $xfer += $output->writeFieldBegin('filter', TType::STRING, 3); - $xfer += $output->writeString($this->filter); + if ($this->part_vals !== null) { + if (!is_array($this->part_vals)) { + throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); + } + $xfer += $output->writeFieldBegin('part_vals', TType::MAP, 3); + { + $output->writeMapBegin(TType::STRING, TType::STRING, count($this->part_vals)); + { + foreach ($this->part_vals as $kiter651 => $viter652) + { + $xfer += $output->writeString($kiter651); + $xfer += $output->writeString($viter652); + } + } + $output->writeMapEnd(); + } $xfer += $output->writeFieldEnd(); } - if ($this->max_parts !== null) { - $xfer += $output->writeFieldBegin('max_parts', TType::I16, 4); - $xfer += $output->writeI16($this->max_parts); + if ($this->eventType !== null) { + $xfer += $output->writeFieldBegin('eventType', TType::I32, 4); + $xfer += $output->writeI32($this->eventType); $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -16932,24 +20454,23 @@ class ThriftHiveMetastore_get_partitions_by_filter_args { } -class ThriftHiveMetastore_get_partitions_by_filter_result { +class ThriftHiveMetastore_isPartitionMarkedForEvent_result { static $_TSPEC; public $success = null; public $o1 = null; public $o2 = null; + public $o3 = null; + public $o4 = null; + public $o5 = null; + public $o6 = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 0 => array( 'var' => 'success', - 'type' => TType::LST, - 'etype' => TType::STRUCT, - 'elem' => array( - 'type' => TType::STRUCT, - 'class' => '\metastore\Partition', - ), + 'type' => TType::BOOL, ), 1 => array( 'var' => 'o1', @@ -16961,6 +20482,26 @@ class ThriftHiveMetastore_get_partitions_by_filter_result { 'type' => TType::STRUCT, 'class' => '\metastore\NoSuchObjectException', ), + 3 => array( + 'var' => 'o3', + 'type' => TType::STRUCT, + 'class' => '\metastore\UnknownDBException', + ), + 4 => array( + 'var' => 'o4', + 'type' => TType::STRUCT, + 'class' => '\metastore\UnknownTableException', + ), + 5 => array( + 'var' => 'o5', + 'type' => TType::STRUCT, + 'class' => '\metastore\UnknownPartitionException', + ), + 6 => array( + 'var' => 'o6', + 'type' => TType::STRUCT, + 'class' => '\metastore\InvalidPartitionException', + ), ); } if (is_array($vals)) { @@ -16973,11 +20514,23 @@ class ThriftHiveMetastore_get_partitions_by_filter_result { if (isset($vals['o2'])) { $this->o2 = $vals['o2']; } + if (isset($vals['o3'])) { + $this->o3 = $vals['o3']; + } + if (isset($vals['o4'])) { + $this->o4 = $vals['o4']; + } + if (isset($vals['o5'])) { + $this->o5 = $vals['o5']; + } + if (isset($vals['o6'])) { + $this->o6 = $vals['o6']; + } } } public function getName() { - return 'ThriftHiveMetastore_get_partitions_by_filter_result'; + return 'ThriftHiveMetastore_isPartitionMarkedForEvent_result'; } public function read($input) @@ -16996,19 +20549,8 @@ class ThriftHiveMetastore_get_partitions_by_filter_result { switch ($fid) { case 0: - if ($ftype == TType::LST) { - $this->success = array(); - $_size534 = 0; - $_etype537 = 0; - $xfer += $input->readListBegin($_etype537, $_size534); - for ($_i538 = 0; $_i538 < $_size534; ++$_i538) - { - $elem539 = null; - $elem539 = new \metastore\Partition(); - $xfer += $elem539->read($input); - $this->success []= $elem539; - } - $xfer += $input->readListEnd(); + if ($ftype == TType::BOOL) { + $xfer += $input->readBool($this->success); } else { $xfer += $input->skip($ftype); } @@ -17029,6 +20571,38 @@ class ThriftHiveMetastore_get_partitions_by_filter_result { $xfer += $input->skip($ftype); } break; + case 3: + if ($ftype == TType::STRUCT) { + $this->o3 = new \metastore\UnknownDBException(); + $xfer += $this->o3->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; + case 4: + if ($ftype == TType::STRUCT) { + $this->o4 = new \metastore\UnknownTableException(); + $xfer += $this->o4->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; + case 5: + if ($ftype == TType::STRUCT) { + $this->o5 = new \metastore\UnknownPartitionException(); + $xfer += $this->o5->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; + case 6: + if ($ftype == TType::STRUCT) { + $this->o6 = new \metastore\InvalidPartitionException(); + $xfer += $this->o6->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; default: $xfer += $input->skip($ftype); break; @@ -17041,22 +20615,10 @@ class ThriftHiveMetastore_get_partitions_by_filter_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_partitions_by_filter_result'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_isPartitionMarkedForEvent_result'); if ($this->success !== null) { - if (!is_array($this->success)) { - throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); - } - $xfer += $output->writeFieldBegin('success', TType::LST, 0); - { - $output->writeListBegin(TType::STRUCT, count($this->success)); - { - foreach ($this->success as $iter540) - { - $xfer += $iter540->write($output); - } - } - $output->writeListEnd(); - } + $xfer += $output->writeFieldBegin('success', TType::BOOL, 0); + $xfer += $output->writeBool($this->success); $xfer += $output->writeFieldEnd(); } if ($this->o1 !== null) { @@ -17069,6 +20631,26 @@ class ThriftHiveMetastore_get_partitions_by_filter_result { $xfer += $this->o2->write($output); $xfer += $output->writeFieldEnd(); } + if ($this->o3 !== null) { + $xfer += $output->writeFieldBegin('o3', TType::STRUCT, 3); + $xfer += $this->o3->write($output); + $xfer += $output->writeFieldEnd(); + } + if ($this->o4 !== null) { + $xfer += $output->writeFieldBegin('o4', TType::STRUCT, 4); + $xfer += $this->o4->write($output); + $xfer += $output->writeFieldEnd(); + } + if ($this->o5 !== null) { + $xfer += $output->writeFieldBegin('o5', TType::STRUCT, 5); + $xfer += $this->o5->write($output); + $xfer += $output->writeFieldEnd(); + } + if ($this->o6 !== null) { + $xfer += $output->writeFieldBegin('o6', TType::STRUCT, 6); + $xfer += $this->o6->write($output); + $xfer += $output->writeFieldEnd(); + } $xfer += $output->writeFieldStop(); $xfer += $output->writeStructEnd(); return $xfer; @@ -17076,30 +20658,39 @@ class ThriftHiveMetastore_get_partitions_by_filter_result { } -class ThriftHiveMetastore_get_partitions_by_expr_args { +class ThriftHiveMetastore_add_index_args { static $_TSPEC; - public $req = null; + public $new_index = null; + public $index_table = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 1 => array( - 'var' => 'req', + 'var' => 'new_index', 'type' => TType::STRUCT, - 'class' => '\metastore\PartitionsByExprRequest', + 'class' => '\metastore\Index', + ), + 2 => array( + 'var' => 'index_table', + 'type' => TType::STRUCT, + 'class' => '\metastore\Table', ), ); } if (is_array($vals)) { - if (isset($vals['req'])) { - $this->req = $vals['req']; + if (isset($vals['new_index'])) { + $this->new_index = $vals['new_index']; + } + if (isset($vals['index_table'])) { + $this->index_table = $vals['index_table']; } } } public function getName() { - return 'ThriftHiveMetastore_get_partitions_by_expr_args'; + return 'ThriftHiveMetastore_add_index_args'; } public function read($input) @@ -17119,8 +20710,16 @@ class ThriftHiveMetastore_get_partitions_by_expr_args { { case 1: if ($ftype == TType::STRUCT) { - $this->req = new \metastore\PartitionsByExprRequest(); - $xfer += $this->req->read($input); + $this->new_index = new \metastore\Index(); + $xfer += $this->new_index->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; + case 2: + if ($ftype == TType::STRUCT) { + $this->index_table = new \metastore\Table(); + $xfer += $this->index_table->read($input); } else { $xfer += $input->skip($ftype); } @@ -17137,13 +20736,21 @@ class ThriftHiveMetastore_get_partitions_by_expr_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_partitions_by_expr_args'); - if ($this->req !== null) { - if (!is_object($this->req)) { + $xfer += $output->writeStructBegin('ThriftHiveMetastore_add_index_args'); + if ($this->new_index !== null) { + if (!is_object($this->new_index)) { throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); } - $xfer += $output->writeFieldBegin('req', TType::STRUCT, 1); - $xfer += $this->req->write($output); + $xfer += $output->writeFieldBegin('new_index', TType::STRUCT, 1); + $xfer += $this->new_index->write($output); + $xfer += $output->writeFieldEnd(); + } + if ($this->index_table !== null) { + if (!is_object($this->index_table)) { + throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); + } + $xfer += $output->writeFieldBegin('index_table', TType::STRUCT, 2); + $xfer += $this->index_table->write($output); $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -17153,12 +20760,13 @@ class ThriftHiveMetastore_get_partitions_by_expr_args { } -class ThriftHiveMetastore_get_partitions_by_expr_result { +class ThriftHiveMetastore_add_index_result { static $_TSPEC; public $success = null; public $o1 = null; public $o2 = null; + public $o3 = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { @@ -17166,17 +20774,22 @@ class ThriftHiveMetastore_get_partitions_by_expr_result { 0 => array( 'var' => 'success', 'type' => TType::STRUCT, - 'class' => '\metastore\PartitionsByExprResult', + 'class' => '\metastore\Index', ), 1 => array( 'var' => 'o1', 'type' => TType::STRUCT, - 'class' => '\metastore\MetaException', + 'class' => '\metastore\InvalidObjectException', + ), + 2 => array( + 'var' => 'o2', + 'type' => TType::STRUCT, + 'class' => '\metastore\AlreadyExistsException', ), - 2 => array( - 'var' => 'o2', + 3 => array( + 'var' => 'o3', 'type' => TType::STRUCT, - 'class' => '\metastore\NoSuchObjectException', + 'class' => '\metastore\MetaException', ), ); } @@ -17190,11 +20803,14 @@ class ThriftHiveMetastore_get_partitions_by_expr_result { if (isset($vals['o2'])) { $this->o2 = $vals['o2']; } + if (isset($vals['o3'])) { + $this->o3 = $vals['o3']; + } } } public function getName() { - return 'ThriftHiveMetastore_get_partitions_by_expr_result'; + return 'ThriftHiveMetastore_add_index_result'; } public function read($input) @@ -17214,7 +20830,7 @@ class ThriftHiveMetastore_get_partitions_by_expr_result { { case 0: if ($ftype == TType::STRUCT) { - $this->success = new \metastore\PartitionsByExprResult(); + $this->success = new \metastore\Index(); $xfer += $this->success->read($input); } else { $xfer += $input->skip($ftype); @@ -17222,7 +20838,7 @@ class ThriftHiveMetastore_get_partitions_by_expr_result { break; case 1: if ($ftype == TType::STRUCT) { - $this->o1 = new \metastore\MetaException(); + $this->o1 = new \metastore\InvalidObjectException(); $xfer += $this->o1->read($input); } else { $xfer += $input->skip($ftype); @@ -17230,12 +20846,20 @@ class ThriftHiveMetastore_get_partitions_by_expr_result { break; case 2: if ($ftype == TType::STRUCT) { - $this->o2 = new \metastore\NoSuchObjectException(); + $this->o2 = new \metastore\AlreadyExistsException(); $xfer += $this->o2->read($input); } else { $xfer += $input->skip($ftype); } break; + case 3: + if ($ftype == TType::STRUCT) { + $this->o3 = new \metastore\MetaException(); + $xfer += $this->o3->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; default: $xfer += $input->skip($ftype); break; @@ -17248,7 +20872,7 @@ class ThriftHiveMetastore_get_partitions_by_expr_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_partitions_by_expr_result'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_add_index_result'); if ($this->success !== null) { if (!is_object($this->success)) { throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); @@ -17267,6 +20891,11 @@ class ThriftHiveMetastore_get_partitions_by_expr_result { $xfer += $this->o2->write($output); $xfer += $output->writeFieldEnd(); } + if ($this->o3 !== null) { + $xfer += $output->writeFieldBegin('o3', TType::STRUCT, 3); + $xfer += $this->o3->write($output); + $xfer += $output->writeFieldEnd(); + } $xfer += $output->writeFieldStop(); $xfer += $output->writeStructEnd(); return $xfer; @@ -17274,49 +20903,54 @@ class ThriftHiveMetastore_get_partitions_by_expr_result { } -class ThriftHiveMetastore_get_partitions_by_names_args { +class ThriftHiveMetastore_alter_index_args { static $_TSPEC; - public $db_name = null; - public $tbl_name = null; - public $names = null; + public $dbname = null; + public $base_tbl_name = null; + public $idx_name = null; + public $new_idx = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 1 => array( - 'var' => 'db_name', + 'var' => 'dbname', 'type' => TType::STRING, ), 2 => array( - 'var' => 'tbl_name', + 'var' => 'base_tbl_name', 'type' => TType::STRING, ), 3 => array( - 'var' => 'names', - 'type' => TType::LST, - 'etype' => TType::STRING, - 'elem' => array( - 'type' => TType::STRING, - ), + 'var' => 'idx_name', + 'type' => TType::STRING, + ), + 4 => array( + 'var' => 'new_idx', + 'type' => TType::STRUCT, + 'class' => '\metastore\Index', ), ); } if (is_array($vals)) { - if (isset($vals['db_name'])) { - $this->db_name = $vals['db_name']; + if (isset($vals['dbname'])) { + $this->dbname = $vals['dbname']; } - if (isset($vals['tbl_name'])) { - $this->tbl_name = $vals['tbl_name']; + if (isset($vals['base_tbl_name'])) { + $this->base_tbl_name = $vals['base_tbl_name']; } - if (isset($vals['names'])) { - $this->names = $vals['names']; + if (isset($vals['idx_name'])) { + $this->idx_name = $vals['idx_name']; + } + if (isset($vals['new_idx'])) { + $this->new_idx = $vals['new_idx']; } } } public function getName() { - return 'ThriftHiveMetastore_get_partitions_by_names_args'; + return 'ThriftHiveMetastore_alter_index_args'; } public function read($input) @@ -17336,31 +20970,29 @@ class ThriftHiveMetastore_get_partitions_by_names_args { { case 1: if ($ftype == TType::STRING) { - $xfer += $input->readString($this->db_name); + $xfer += $input->readString($this->dbname); } else { $xfer += $input->skip($ftype); } break; case 2: if ($ftype == TType::STRING) { - $xfer += $input->readString($this->tbl_name); + $xfer += $input->readString($this->base_tbl_name); } else { $xfer += $input->skip($ftype); } break; case 3: - if ($ftype == TType::LST) { - $this->names = array(); - $_size541 = 0; - $_etype544 = 0; - $xfer += $input->readListBegin($_etype544, $_size541); - for ($_i545 = 0; $_i545 < $_size541; ++$_i545) - { - $elem546 = null; - $xfer += $input->readString($elem546); - $this->names []= $elem546; - } - $xfer += $input->readListEnd(); + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->idx_name); + } else { + $xfer += $input->skip($ftype); + } + break; + case 4: + if ($ftype == TType::STRUCT) { + $this->new_idx = new \metastore\Index(); + $xfer += $this->new_idx->read($input); } else { $xfer += $input->skip($ftype); } @@ -17377,32 +21009,28 @@ class ThriftHiveMetastore_get_partitions_by_names_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_partitions_by_names_args'); - if ($this->db_name !== null) { - $xfer += $output->writeFieldBegin('db_name', TType::STRING, 1); - $xfer += $output->writeString($this->db_name); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_alter_index_args'); + if ($this->dbname !== null) { + $xfer += $output->writeFieldBegin('dbname', TType::STRING, 1); + $xfer += $output->writeString($this->dbname); $xfer += $output->writeFieldEnd(); } - if ($this->tbl_name !== null) { - $xfer += $output->writeFieldBegin('tbl_name', TType::STRING, 2); - $xfer += $output->writeString($this->tbl_name); + if ($this->base_tbl_name !== null) { + $xfer += $output->writeFieldBegin('base_tbl_name', TType::STRING, 2); + $xfer += $output->writeString($this->base_tbl_name); $xfer += $output->writeFieldEnd(); } - if ($this->names !== null) { - if (!is_array($this->names)) { + if ($this->idx_name !== null) { + $xfer += $output->writeFieldBegin('idx_name', TType::STRING, 3); + $xfer += $output->writeString($this->idx_name); + $xfer += $output->writeFieldEnd(); + } + if ($this->new_idx !== null) { + if (!is_object($this->new_idx)) { throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); } - $xfer += $output->writeFieldBegin('names', TType::LST, 3); - { - $output->writeListBegin(TType::STRING, count($this->names)); - { - foreach ($this->names as $iter547) - { - $xfer += $output->writeString($iter547); - } - } - $output->writeListEnd(); - } + $xfer += $output->writeFieldBegin('new_idx', TType::STRUCT, 4); + $xfer += $this->new_idx->write($output); $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -17412,41 +21040,28 @@ class ThriftHiveMetastore_get_partitions_by_names_args { } -class ThriftHiveMetastore_get_partitions_by_names_result { +class ThriftHiveMetastore_alter_index_result { static $_TSPEC; - public $success = null; public $o1 = null; public $o2 = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( - 0 => array( - 'var' => 'success', - 'type' => TType::LST, - 'etype' => TType::STRUCT, - 'elem' => array( - 'type' => TType::STRUCT, - 'class' => '\metastore\Partition', - ), - ), 1 => array( 'var' => 'o1', 'type' => TType::STRUCT, - 'class' => '\metastore\MetaException', + 'class' => '\metastore\InvalidOperationException', ), 2 => array( 'var' => 'o2', 'type' => TType::STRUCT, - 'class' => '\metastore\NoSuchObjectException', + 'class' => '\metastore\MetaException', ), ); } if (is_array($vals)) { - if (isset($vals['success'])) { - $this->success = $vals['success']; - } if (isset($vals['o1'])) { $this->o1 = $vals['o1']; } @@ -17457,7 +21072,7 @@ class ThriftHiveMetastore_get_partitions_by_names_result { } public function getName() { - return 'ThriftHiveMetastore_get_partitions_by_names_result'; + return 'ThriftHiveMetastore_alter_index_result'; } public function read($input) @@ -17475,27 +21090,9 @@ class ThriftHiveMetastore_get_partitions_by_names_result { } switch ($fid) { - case 0: - if ($ftype == TType::LST) { - $this->success = array(); - $_size548 = 0; - $_etype551 = 0; - $xfer += $input->readListBegin($_etype551, $_size548); - for ($_i552 = 0; $_i552 < $_size548; ++$_i552) - { - $elem553 = null; - $elem553 = new \metastore\Partition(); - $xfer += $elem553->read($input); - $this->success []= $elem553; - } - $xfer += $input->readListEnd(); - } else { - $xfer += $input->skip($ftype); - } - break; case 1: if ($ftype == TType::STRUCT) { - $this->o1 = new \metastore\MetaException(); + $this->o1 = new \metastore\InvalidOperationException(); $xfer += $this->o1->read($input); } else { $xfer += $input->skip($ftype); @@ -17503,7 +21100,7 @@ class ThriftHiveMetastore_get_partitions_by_names_result { break; case 2: if ($ftype == TType::STRUCT) { - $this->o2 = new \metastore\NoSuchObjectException(); + $this->o2 = new \metastore\MetaException(); $xfer += $this->o2->read($input); } else { $xfer += $input->skip($ftype); @@ -17521,24 +21118,7 @@ class ThriftHiveMetastore_get_partitions_by_names_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_partitions_by_names_result'); - if ($this->success !== null) { - if (!is_array($this->success)) { - throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); - } - $xfer += $output->writeFieldBegin('success', TType::LST, 0); - { - $output->writeListBegin(TType::STRUCT, count($this->success)); - { - foreach ($this->success as $iter554) - { - $xfer += $iter554->write($output); - } - } - $output->writeListEnd(); - } - $xfer += $output->writeFieldEnd(); - } + $xfer += $output->writeStructBegin('ThriftHiveMetastore_alter_index_result'); if ($this->o1 !== null) { $xfer += $output->writeFieldBegin('o1', TType::STRUCT, 1); $xfer += $this->o1->write($output); @@ -17556,12 +21136,13 @@ class ThriftHiveMetastore_get_partitions_by_names_result { } -class ThriftHiveMetastore_alter_partition_args { +class ThriftHiveMetastore_drop_index_by_name_args { static $_TSPEC; public $db_name = null; public $tbl_name = null; - public $new_part = null; + public $index_name = null; + public $deleteData = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { @@ -17575,9 +21156,12 @@ class ThriftHiveMetastore_alter_partition_args { 'type' => TType::STRING, ), 3 => array( - 'var' => 'new_part', - 'type' => TType::STRUCT, - 'class' => '\metastore\Partition', + 'var' => 'index_name', + 'type' => TType::STRING, + ), + 4 => array( + 'var' => 'deleteData', + 'type' => TType::BOOL, ), ); } @@ -17588,14 +21172,17 @@ class ThriftHiveMetastore_alter_partition_args { if (isset($vals['tbl_name'])) { $this->tbl_name = $vals['tbl_name']; } - if (isset($vals['new_part'])) { - $this->new_part = $vals['new_part']; + if (isset($vals['index_name'])) { + $this->index_name = $vals['index_name']; + } + if (isset($vals['deleteData'])) { + $this->deleteData = $vals['deleteData']; } } } public function getName() { - return 'ThriftHiveMetastore_alter_partition_args'; + return 'ThriftHiveMetastore_drop_index_by_name_args'; } public function read($input) @@ -17628,9 +21215,15 @@ class ThriftHiveMetastore_alter_partition_args { } break; case 3: - if ($ftype == TType::STRUCT) { - $this->new_part = new \metastore\Partition(); - $xfer += $this->new_part->read($input); + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->index_name); + } else { + $xfer += $input->skip($ftype); + } + break; + case 4: + if ($ftype == TType::BOOL) { + $xfer += $input->readBool($this->deleteData); } else { $xfer += $input->skip($ftype); } @@ -17647,7 +21240,7 @@ class ThriftHiveMetastore_alter_partition_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_alter_partition_args'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_drop_index_by_name_args'); if ($this->db_name !== null) { $xfer += $output->writeFieldBegin('db_name', TType::STRING, 1); $xfer += $output->writeString($this->db_name); @@ -17658,12 +21251,14 @@ class ThriftHiveMetastore_alter_partition_args { $xfer += $output->writeString($this->tbl_name); $xfer += $output->writeFieldEnd(); } - if ($this->new_part !== null) { - if (!is_object($this->new_part)) { - throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); - } - $xfer += $output->writeFieldBegin('new_part', TType::STRUCT, 3); - $xfer += $this->new_part->write($output); + if ($this->index_name !== null) { + $xfer += $output->writeFieldBegin('index_name', TType::STRING, 3); + $xfer += $output->writeString($this->index_name); + $xfer += $output->writeFieldEnd(); + } + if ($this->deleteData !== null) { + $xfer += $output->writeFieldBegin('deleteData', TType::BOOL, 4); + $xfer += $output->writeBool($this->deleteData); $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -17673,19 +21268,24 @@ class ThriftHiveMetastore_alter_partition_args { } -class ThriftHiveMetastore_alter_partition_result { +class ThriftHiveMetastore_drop_index_by_name_result { static $_TSPEC; + public $success = null; public $o1 = null; public $o2 = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( + 0 => array( + 'var' => 'success', + 'type' => TType::BOOL, + ), 1 => array( 'var' => 'o1', 'type' => TType::STRUCT, - 'class' => '\metastore\InvalidOperationException', + 'class' => '\metastore\NoSuchObjectException', ), 2 => array( 'var' => 'o2', @@ -17695,6 +21295,9 @@ class ThriftHiveMetastore_alter_partition_result { ); } if (is_array($vals)) { + if (isset($vals['success'])) { + $this->success = $vals['success']; + } if (isset($vals['o1'])) { $this->o1 = $vals['o1']; } @@ -17705,7 +21308,7 @@ class ThriftHiveMetastore_alter_partition_result { } public function getName() { - return 'ThriftHiveMetastore_alter_partition_result'; + return 'ThriftHiveMetastore_drop_index_by_name_result'; } public function read($input) @@ -17723,9 +21326,16 @@ class ThriftHiveMetastore_alter_partition_result { } switch ($fid) { + case 0: + if ($ftype == TType::BOOL) { + $xfer += $input->readBool($this->success); + } else { + $xfer += $input->skip($ftype); + } + break; case 1: if ($ftype == TType::STRUCT) { - $this->o1 = new \metastore\InvalidOperationException(); + $this->o1 = new \metastore\NoSuchObjectException(); $xfer += $this->o1->read($input); } else { $xfer += $input->skip($ftype); @@ -17751,7 +21361,12 @@ class ThriftHiveMetastore_alter_partition_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_alter_partition_result'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_drop_index_by_name_result'); + if ($this->success !== null) { + $xfer += $output->writeFieldBegin('success', TType::BOOL, 0); + $xfer += $output->writeBool($this->success); + $xfer += $output->writeFieldEnd(); + } if ($this->o1 !== null) { $xfer += $output->writeFieldBegin('o1', TType::STRUCT, 1); $xfer += $this->o1->write($output); @@ -17769,12 +21384,12 @@ class ThriftHiveMetastore_alter_partition_result { } -class ThriftHiveMetastore_alter_partitions_args { +class ThriftHiveMetastore_get_index_by_name_args { static $_TSPEC; public $db_name = null; public $tbl_name = null; - public $new_parts = null; + public $index_name = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { @@ -17788,13 +21403,8 @@ class ThriftHiveMetastore_alter_partitions_args { 'type' => TType::STRING, ), 3 => array( - 'var' => 'new_parts', - 'type' => TType::LST, - 'etype' => TType::STRUCT, - 'elem' => array( - 'type' => TType::STRUCT, - 'class' => '\metastore\Partition', - ), + 'var' => 'index_name', + 'type' => TType::STRING, ), ); } @@ -17805,14 +21415,14 @@ class ThriftHiveMetastore_alter_partitions_args { if (isset($vals['tbl_name'])) { $this->tbl_name = $vals['tbl_name']; } - if (isset($vals['new_parts'])) { - $this->new_parts = $vals['new_parts']; + if (isset($vals['index_name'])) { + $this->index_name = $vals['index_name']; } } } public function getName() { - return 'ThriftHiveMetastore_alter_partitions_args'; + return 'ThriftHiveMetastore_get_index_by_name_args'; } public function read($input) @@ -17845,19 +21455,8 @@ class ThriftHiveMetastore_alter_partitions_args { } break; case 3: - if ($ftype == TType::LST) { - $this->new_parts = array(); - $_size555 = 0; - $_etype558 = 0; - $xfer += $input->readListBegin($_etype558, $_size555); - for ($_i559 = 0; $_i559 < $_size555; ++$_i559) - { - $elem560 = null; - $elem560 = new \metastore\Partition(); - $xfer += $elem560->read($input); - $this->new_parts []= $elem560; - } - $xfer += $input->readListEnd(); + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->index_name); } else { $xfer += $input->skip($ftype); } @@ -17874,7 +21473,7 @@ class ThriftHiveMetastore_alter_partitions_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_alter_partitions_args'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_index_by_name_args'); if ($this->db_name !== null) { $xfer += $output->writeFieldBegin('db_name', TType::STRING, 1); $xfer += $output->writeString($this->db_name); @@ -17885,21 +21484,9 @@ class ThriftHiveMetastore_alter_partitions_args { $xfer += $output->writeString($this->tbl_name); $xfer += $output->writeFieldEnd(); } - if ($this->new_parts !== null) { - if (!is_array($this->new_parts)) { - throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); - } - $xfer += $output->writeFieldBegin('new_parts', TType::LST, 3); - { - $output->writeListBegin(TType::STRUCT, count($this->new_parts)); - { - foreach ($this->new_parts as $iter561) - { - $xfer += $iter561->write($output); - } - } - $output->writeListEnd(); - } + if ($this->index_name !== null) { + $xfer += $output->writeFieldBegin('index_name', TType::STRING, 3); + $xfer += $output->writeString($this->index_name); $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -17909,28 +21496,37 @@ class ThriftHiveMetastore_alter_partitions_args { } -class ThriftHiveMetastore_alter_partitions_result { +class ThriftHiveMetastore_get_index_by_name_result { static $_TSPEC; + public $success = null; public $o1 = null; public $o2 = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( + 0 => array( + 'var' => 'success', + 'type' => TType::STRUCT, + 'class' => '\metastore\Index', + ), 1 => array( 'var' => 'o1', 'type' => TType::STRUCT, - 'class' => '\metastore\InvalidOperationException', + 'class' => '\metastore\MetaException', ), 2 => array( 'var' => 'o2', 'type' => TType::STRUCT, - 'class' => '\metastore\MetaException', + 'class' => '\metastore\NoSuchObjectException', ), ); } if (is_array($vals)) { + if (isset($vals['success'])) { + $this->success = $vals['success']; + } if (isset($vals['o1'])) { $this->o1 = $vals['o1']; } @@ -17941,7 +21537,7 @@ class ThriftHiveMetastore_alter_partitions_result { } public function getName() { - return 'ThriftHiveMetastore_alter_partitions_result'; + return 'ThriftHiveMetastore_get_index_by_name_result'; } public function read($input) @@ -17959,9 +21555,17 @@ class ThriftHiveMetastore_alter_partitions_result { } switch ($fid) { + case 0: + if ($ftype == TType::STRUCT) { + $this->success = new \metastore\Index(); + $xfer += $this->success->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; case 1: if ($ftype == TType::STRUCT) { - $this->o1 = new \metastore\InvalidOperationException(); + $this->o1 = new \metastore\MetaException(); $xfer += $this->o1->read($input); } else { $xfer += $input->skip($ftype); @@ -17969,7 +21573,7 @@ class ThriftHiveMetastore_alter_partitions_result { break; case 2: if ($ftype == TType::STRUCT) { - $this->o2 = new \metastore\MetaException(); + $this->o2 = new \metastore\NoSuchObjectException(); $xfer += $this->o2->read($input); } else { $xfer += $input->skip($ftype); @@ -17987,7 +21591,15 @@ class ThriftHiveMetastore_alter_partitions_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_alter_partitions_result'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_index_by_name_result'); + if ($this->success !== null) { + if (!is_object($this->success)) { + throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); + } + $xfer += $output->writeFieldBegin('success', TType::STRUCT, 0); + $xfer += $this->success->write($output); + $xfer += $output->writeFieldEnd(); + } if ($this->o1 !== null) { $xfer += $output->writeFieldBegin('o1', TType::STRUCT, 1); $xfer += $this->o1->write($output); @@ -18005,13 +21617,12 @@ class ThriftHiveMetastore_alter_partitions_result { } -class ThriftHiveMetastore_alter_partition_with_environment_context_args { +class ThriftHiveMetastore_get_indexes_args { static $_TSPEC; public $db_name = null; public $tbl_name = null; - public $new_part = null; - public $environment_context = null; + public $max_indexes = -1; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { @@ -18025,14 +21636,8 @@ class ThriftHiveMetastore_alter_partition_with_environment_context_args { 'type' => TType::STRING, ), 3 => array( - 'var' => 'new_part', - 'type' => TType::STRUCT, - 'class' => '\metastore\Partition', - ), - 4 => array( - 'var' => 'environment_context', - 'type' => TType::STRUCT, - 'class' => '\metastore\EnvironmentContext', + 'var' => 'max_indexes', + 'type' => TType::I16, ), ); } @@ -18043,17 +21648,14 @@ class ThriftHiveMetastore_alter_partition_with_environment_context_args { if (isset($vals['tbl_name'])) { $this->tbl_name = $vals['tbl_name']; } - if (isset($vals['new_part'])) { - $this->new_part = $vals['new_part']; - } - if (isset($vals['environment_context'])) { - $this->environment_context = $vals['environment_context']; + if (isset($vals['max_indexes'])) { + $this->max_indexes = $vals['max_indexes']; } } } public function getName() { - return 'ThriftHiveMetastore_alter_partition_with_environment_context_args'; + return 'ThriftHiveMetastore_get_indexes_args'; } public function read($input) @@ -18086,17 +21688,8 @@ class ThriftHiveMetastore_alter_partition_with_environment_context_args { } break; case 3: - if ($ftype == TType::STRUCT) { - $this->new_part = new \metastore\Partition(); - $xfer += $this->new_part->read($input); - } else { - $xfer += $input->skip($ftype); - } - break; - case 4: - if ($ftype == TType::STRUCT) { - $this->environment_context = new \metastore\EnvironmentContext(); - $xfer += $this->environment_context->read($input); + if ($ftype == TType::I16) { + $xfer += $input->readI16($this->max_indexes); } else { $xfer += $input->skip($ftype); } @@ -18113,7 +21706,7 @@ class ThriftHiveMetastore_alter_partition_with_environment_context_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_alter_partition_with_environment_context_args'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_indexes_args'); if ($this->db_name !== null) { $xfer += $output->writeFieldBegin('db_name', TType::STRING, 1); $xfer += $output->writeString($this->db_name); @@ -18124,20 +21717,9 @@ class ThriftHiveMetastore_alter_partition_with_environment_context_args { $xfer += $output->writeString($this->tbl_name); $xfer += $output->writeFieldEnd(); } - if ($this->new_part !== null) { - if (!is_object($this->new_part)) { - throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); - } - $xfer += $output->writeFieldBegin('new_part', TType::STRUCT, 3); - $xfer += $this->new_part->write($output); - $xfer += $output->writeFieldEnd(); - } - if ($this->environment_context !== null) { - if (!is_object($this->environment_context)) { - throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); - } - $xfer += $output->writeFieldBegin('environment_context', TType::STRUCT, 4); - $xfer += $this->environment_context->write($output); + if ($this->max_indexes !== null) { + $xfer += $output->writeFieldBegin('max_indexes', TType::I16, 3); + $xfer += $output->writeI16($this->max_indexes); $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -18147,19 +21729,29 @@ class ThriftHiveMetastore_alter_partition_with_environment_context_args { } -class ThriftHiveMetastore_alter_partition_with_environment_context_result { +class ThriftHiveMetastore_get_indexes_result { static $_TSPEC; + public $success = null; public $o1 = null; public $o2 = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( + 0 => array( + 'var' => 'success', + 'type' => TType::LST, + 'etype' => TType::STRUCT, + 'elem' => array( + 'type' => TType::STRUCT, + 'class' => '\metastore\Index', + ), + ), 1 => array( 'var' => 'o1', 'type' => TType::STRUCT, - 'class' => '\metastore\InvalidOperationException', + 'class' => '\metastore\NoSuchObjectException', ), 2 => array( 'var' => 'o2', @@ -18169,6 +21761,9 @@ class ThriftHiveMetastore_alter_partition_with_environment_context_result { ); } if (is_array($vals)) { + if (isset($vals['success'])) { + $this->success = $vals['success']; + } if (isset($vals['o1'])) { $this->o1 = $vals['o1']; } @@ -18179,7 +21774,7 @@ class ThriftHiveMetastore_alter_partition_with_environment_context_result { } public function getName() { - return 'ThriftHiveMetastore_alter_partition_with_environment_context_result'; + return 'ThriftHiveMetastore_get_indexes_result'; } public function read($input) @@ -18197,9 +21792,27 @@ class ThriftHiveMetastore_alter_partition_with_environment_context_result { } switch ($fid) { + case 0: + if ($ftype == TType::LST) { + $this->success = array(); + $_size653 = 0; + $_etype656 = 0; + $xfer += $input->readListBegin($_etype656, $_size653); + for ($_i657 = 0; $_i657 < $_size653; ++$_i657) + { + $elem658 = null; + $elem658 = new \metastore\Index(); + $xfer += $elem658->read($input); + $this->success []= $elem658; + } + $xfer += $input->readListEnd(); + } else { + $xfer += $input->skip($ftype); + } + break; case 1: if ($ftype == TType::STRUCT) { - $this->o1 = new \metastore\InvalidOperationException(); + $this->o1 = new \metastore\NoSuchObjectException(); $xfer += $this->o1->read($input); } else { $xfer += $input->skip($ftype); @@ -18225,7 +21838,24 @@ class ThriftHiveMetastore_alter_partition_with_environment_context_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_alter_partition_with_environment_context_result'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_indexes_result'); + if ($this->success !== null) { + if (!is_array($this->success)) { + throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); + } + $xfer += $output->writeFieldBegin('success', TType::LST, 0); + { + $output->writeListBegin(TType::STRUCT, count($this->success)); + { + foreach ($this->success as $iter659) + { + $xfer += $iter659->write($output); + } + } + $output->writeListEnd(); + } + $xfer += $output->writeFieldEnd(); + } if ($this->o1 !== null) { $xfer += $output->writeFieldBegin('o1', TType::STRUCT, 1); $xfer += $this->o1->write($output); @@ -18243,13 +21873,12 @@ class ThriftHiveMetastore_alter_partition_with_environment_context_result { } -class ThriftHiveMetastore_rename_partition_args { +class ThriftHiveMetastore_get_index_names_args { static $_TSPEC; public $db_name = null; public $tbl_name = null; - public $part_vals = null; - public $new_part = null; + public $max_indexes = -1; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { @@ -18263,17 +21892,8 @@ class ThriftHiveMetastore_rename_partition_args { 'type' => TType::STRING, ), 3 => array( - 'var' => 'part_vals', - 'type' => TType::LST, - 'etype' => TType::STRING, - 'elem' => array( - 'type' => TType::STRING, - ), - ), - 4 => array( - 'var' => 'new_part', - 'type' => TType::STRUCT, - 'class' => '\metastore\Partition', + 'var' => 'max_indexes', + 'type' => TType::I16, ), ); } @@ -18284,17 +21904,14 @@ class ThriftHiveMetastore_rename_partition_args { if (isset($vals['tbl_name'])) { $this->tbl_name = $vals['tbl_name']; } - if (isset($vals['part_vals'])) { - $this->part_vals = $vals['part_vals']; - } - if (isset($vals['new_part'])) { - $this->new_part = $vals['new_part']; + if (isset($vals['max_indexes'])) { + $this->max_indexes = $vals['max_indexes']; } } } public function getName() { - return 'ThriftHiveMetastore_rename_partition_args'; + return 'ThriftHiveMetastore_get_index_names_args'; } public function read($input) @@ -18327,26 +21944,8 @@ class ThriftHiveMetastore_rename_partition_args { } break; case 3: - if ($ftype == TType::LST) { - $this->part_vals = array(); - $_size562 = 0; - $_etype565 = 0; - $xfer += $input->readListBegin($_etype565, $_size562); - for ($_i566 = 0; $_i566 < $_size562; ++$_i566) - { - $elem567 = null; - $xfer += $input->readString($elem567); - $this->part_vals []= $elem567; - } - $xfer += $input->readListEnd(); - } else { - $xfer += $input->skip($ftype); - } - break; - case 4: - if ($ftype == TType::STRUCT) { - $this->new_part = new \metastore\Partition(); - $xfer += $this->new_part->read($input); + if ($ftype == TType::I16) { + $xfer += $input->readI16($this->max_indexes); } else { $xfer += $input->skip($ftype); } @@ -18363,7 +21962,7 @@ class ThriftHiveMetastore_rename_partition_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_rename_partition_args'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_index_names_args'); if ($this->db_name !== null) { $xfer += $output->writeFieldBegin('db_name', TType::STRING, 1); $xfer += $output->writeString($this->db_name); @@ -18374,29 +21973,9 @@ class ThriftHiveMetastore_rename_partition_args { $xfer += $output->writeString($this->tbl_name); $xfer += $output->writeFieldEnd(); } - if ($this->part_vals !== null) { - if (!is_array($this->part_vals)) { - throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); - } - $xfer += $output->writeFieldBegin('part_vals', TType::LST, 3); - { - $output->writeListBegin(TType::STRING, count($this->part_vals)); - { - foreach ($this->part_vals as $iter568) - { - $xfer += $output->writeString($iter568); - } - } - $output->writeListEnd(); - } - $xfer += $output->writeFieldEnd(); - } - if ($this->new_part !== null) { - if (!is_object($this->new_part)) { - throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); - } - $xfer += $output->writeFieldBegin('new_part', TType::STRUCT, 4); - $xfer += $this->new_part->write($output); + if ($this->max_indexes !== null) { + $xfer += $output->writeFieldBegin('max_indexes', TType::I16, 3); + $xfer += $output->writeI16($this->max_indexes); $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -18406,21 +21985,24 @@ class ThriftHiveMetastore_rename_partition_args { } -class ThriftHiveMetastore_rename_partition_result { +class ThriftHiveMetastore_get_index_names_result { static $_TSPEC; - public $o1 = null; + public $success = null; public $o2 = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( - 1 => array( - 'var' => 'o1', - 'type' => TType::STRUCT, - 'class' => '\metastore\InvalidOperationException', + 0 => array( + 'var' => 'success', + 'type' => TType::LST, + 'etype' => TType::STRING, + 'elem' => array( + 'type' => TType::STRING, + ), ), - 2 => array( + 1 => array( 'var' => 'o2', 'type' => TType::STRUCT, 'class' => '\metastore\MetaException', @@ -18428,8 +22010,8 @@ class ThriftHiveMetastore_rename_partition_result { ); } if (is_array($vals)) { - if (isset($vals['o1'])) { - $this->o1 = $vals['o1']; + if (isset($vals['success'])) { + $this->success = $vals['success']; } if (isset($vals['o2'])) { $this->o2 = $vals['o2']; @@ -18438,7 +22020,7 @@ class ThriftHiveMetastore_rename_partition_result { } public function getName() { - return 'ThriftHiveMetastore_rename_partition_result'; + return 'ThriftHiveMetastore_get_index_names_result'; } public function read($input) @@ -18456,15 +22038,24 @@ class ThriftHiveMetastore_rename_partition_result { } switch ($fid) { - case 1: - if ($ftype == TType::STRUCT) { - $this->o1 = new \metastore\InvalidOperationException(); - $xfer += $this->o1->read($input); + case 0: + if ($ftype == TType::LST) { + $this->success = array(); + $_size660 = 0; + $_etype663 = 0; + $xfer += $input->readListBegin($_etype663, $_size660); + for ($_i664 = 0; $_i664 < $_size660; ++$_i664) + { + $elem665 = null; + $xfer += $input->readString($elem665); + $this->success []= $elem665; + } + $xfer += $input->readListEnd(); } else { $xfer += $input->skip($ftype); } break; - case 2: + case 1: if ($ftype == TType::STRUCT) { $this->o2 = new \metastore\MetaException(); $xfer += $this->o2->read($input); @@ -18484,14 +22075,26 @@ class ThriftHiveMetastore_rename_partition_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_rename_partition_result'); - if ($this->o1 !== null) { - $xfer += $output->writeFieldBegin('o1', TType::STRUCT, 1); - $xfer += $this->o1->write($output); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_index_names_result'); + if ($this->success !== null) { + if (!is_array($this->success)) { + throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); + } + $xfer += $output->writeFieldBegin('success', TType::LST, 0); + { + $output->writeListBegin(TType::STRING, count($this->success)); + { + foreach ($this->success as $iter666) + { + $xfer += $output->writeString($iter666); + } + } + $output->writeListEnd(); + } $xfer += $output->writeFieldEnd(); } if ($this->o2 !== null) { - $xfer += $output->writeFieldBegin('o2', TType::STRUCT, 2); + $xfer += $output->writeFieldBegin('o2', TType::STRUCT, 1); $xfer += $this->o2->write($output); $xfer += $output->writeFieldEnd(); } @@ -18502,41 +22105,30 @@ class ThriftHiveMetastore_rename_partition_result { } -class ThriftHiveMetastore_partition_name_has_valid_characters_args { +class ThriftHiveMetastore_update_table_column_statistics_args { static $_TSPEC; - public $part_vals = null; - public $throw_exception = null; + public $stats_obj = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 1 => array( - 'var' => 'part_vals', - 'type' => TType::LST, - 'etype' => TType::STRING, - 'elem' => array( - 'type' => TType::STRING, - ), - ), - 2 => array( - 'var' => 'throw_exception', - 'type' => TType::BOOL, + 'var' => 'stats_obj', + 'type' => TType::STRUCT, + 'class' => '\metastore\ColumnStatistics', ), ); } if (is_array($vals)) { - if (isset($vals['part_vals'])) { - $this->part_vals = $vals['part_vals']; - } - if (isset($vals['throw_exception'])) { - $this->throw_exception = $vals['throw_exception']; + if (isset($vals['stats_obj'])) { + $this->stats_obj = $vals['stats_obj']; } } } public function getName() { - return 'ThriftHiveMetastore_partition_name_has_valid_characters_args'; + return 'ThriftHiveMetastore_update_table_column_statistics_args'; } public function read($input) @@ -18555,25 +22147,9 @@ class ThriftHiveMetastore_partition_name_has_valid_characters_args { switch ($fid) { case 1: - if ($ftype == TType::LST) { - $this->part_vals = array(); - $_size569 = 0; - $_etype572 = 0; - $xfer += $input->readListBegin($_etype572, $_size569); - for ($_i573 = 0; $_i573 < $_size569; ++$_i573) - { - $elem574 = null; - $xfer += $input->readString($elem574); - $this->part_vals []= $elem574; - } - $xfer += $input->readListEnd(); - } else { - $xfer += $input->skip($ftype); - } - break; - case 2: - if ($ftype == TType::BOOL) { - $xfer += $input->readBool($this->throw_exception); + if ($ftype == TType::STRUCT) { + $this->stats_obj = new \metastore\ColumnStatistics(); + $xfer += $this->stats_obj->read($input); } else { $xfer += $input->skip($ftype); } @@ -18590,27 +22166,13 @@ class ThriftHiveMetastore_partition_name_has_valid_characters_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_partition_name_has_valid_characters_args'); - if ($this->part_vals !== null) { - if (!is_array($this->part_vals)) { + $xfer += $output->writeStructBegin('ThriftHiveMetastore_update_table_column_statistics_args'); + if ($this->stats_obj !== null) { + if (!is_object($this->stats_obj)) { throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); } - $xfer += $output->writeFieldBegin('part_vals', TType::LST, 1); - { - $output->writeListBegin(TType::STRING, count($this->part_vals)); - { - foreach ($this->part_vals as $iter575) - { - $xfer += $output->writeString($iter575); - } - } - $output->writeListEnd(); - } - $xfer += $output->writeFieldEnd(); - } - if ($this->throw_exception !== null) { - $xfer += $output->writeFieldBegin('throw_exception', TType::BOOL, 2); - $xfer += $output->writeBool($this->throw_exception); + $xfer += $output->writeFieldBegin('stats_obj', TType::STRUCT, 1); + $xfer += $this->stats_obj->write($output); $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -18620,11 +22182,14 @@ class ThriftHiveMetastore_partition_name_has_valid_characters_args { } -class ThriftHiveMetastore_partition_name_has_valid_characters_result { +class ThriftHiveMetastore_update_table_column_statistics_result { static $_TSPEC; public $success = null; public $o1 = null; + public $o2 = null; + public $o3 = null; + public $o4 = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { @@ -18636,8 +22201,23 @@ class ThriftHiveMetastore_partition_name_has_valid_characters_result { 1 => array( 'var' => 'o1', 'type' => TType::STRUCT, + 'class' => '\metastore\NoSuchObjectException', + ), + 2 => array( + 'var' => 'o2', + 'type' => TType::STRUCT, + 'class' => '\metastore\InvalidObjectException', + ), + 3 => array( + 'var' => 'o3', + 'type' => TType::STRUCT, 'class' => '\metastore\MetaException', ), + 4 => array( + 'var' => 'o4', + 'type' => TType::STRUCT, + 'class' => '\metastore\InvalidInputException', + ), ); } if (is_array($vals)) { @@ -18647,11 +22227,20 @@ class ThriftHiveMetastore_partition_name_has_valid_characters_result { if (isset($vals['o1'])) { $this->o1 = $vals['o1']; } + if (isset($vals['o2'])) { + $this->o2 = $vals['o2']; + } + if (isset($vals['o3'])) { + $this->o3 = $vals['o3']; + } + if (isset($vals['o4'])) { + $this->o4 = $vals['o4']; + } } } public function getName() { - return 'ThriftHiveMetastore_partition_name_has_valid_characters_result'; + return 'ThriftHiveMetastore_update_table_column_statistics_result'; } public function read($input) @@ -18678,12 +22267,36 @@ class ThriftHiveMetastore_partition_name_has_valid_characters_result { break; case 1: if ($ftype == TType::STRUCT) { - $this->o1 = new \metastore\MetaException(); + $this->o1 = new \metastore\NoSuchObjectException(); $xfer += $this->o1->read($input); } else { $xfer += $input->skip($ftype); } break; + case 2: + if ($ftype == TType::STRUCT) { + $this->o2 = new \metastore\InvalidObjectException(); + $xfer += $this->o2->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; + case 3: + if ($ftype == TType::STRUCT) { + $this->o3 = new \metastore\MetaException(); + $xfer += $this->o3->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; + case 4: + if ($ftype == TType::STRUCT) { + $this->o4 = new \metastore\InvalidInputException(); + $xfer += $this->o4->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; default: $xfer += $input->skip($ftype); break; @@ -18696,7 +22309,7 @@ class ThriftHiveMetastore_partition_name_has_valid_characters_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_partition_name_has_valid_characters_result'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_update_table_column_statistics_result'); if ($this->success !== null) { $xfer += $output->writeFieldBegin('success', TType::BOOL, 0); $xfer += $output->writeBool($this->success); @@ -18707,6 +22320,21 @@ class ThriftHiveMetastore_partition_name_has_valid_characters_result { $xfer += $this->o1->write($output); $xfer += $output->writeFieldEnd(); } + if ($this->o2 !== null) { + $xfer += $output->writeFieldBegin('o2', TType::STRUCT, 2); + $xfer += $this->o2->write($output); + $xfer += $output->writeFieldEnd(); + } + if ($this->o3 !== null) { + $xfer += $output->writeFieldBegin('o3', TType::STRUCT, 3); + $xfer += $this->o3->write($output); + $xfer += $output->writeFieldEnd(); + } + if ($this->o4 !== null) { + $xfer += $output->writeFieldBegin('o4', TType::STRUCT, 4); + $xfer += $this->o4->write($output); + $xfer += $output->writeFieldEnd(); + } $xfer += $output->writeFieldStop(); $xfer += $output->writeStructEnd(); return $xfer; @@ -18714,37 +22342,30 @@ class ThriftHiveMetastore_partition_name_has_valid_characters_result { } -class ThriftHiveMetastore_get_config_value_args { +class ThriftHiveMetastore_update_partition_column_statistics_args { static $_TSPEC; - public $name = null; - public $defaultValue = null; + public $stats_obj = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 1 => array( - 'var' => 'name', - 'type' => TType::STRING, - ), - 2 => array( - 'var' => 'defaultValue', - 'type' => TType::STRING, + 'var' => 'stats_obj', + 'type' => TType::STRUCT, + 'class' => '\metastore\ColumnStatistics', ), ); } if (is_array($vals)) { - if (isset($vals['name'])) { - $this->name = $vals['name']; - } - if (isset($vals['defaultValue'])) { - $this->defaultValue = $vals['defaultValue']; + if (isset($vals['stats_obj'])) { + $this->stats_obj = $vals['stats_obj']; } } } public function getName() { - return 'ThriftHiveMetastore_get_config_value_args'; + return 'ThriftHiveMetastore_update_partition_column_statistics_args'; } public function read($input) @@ -18763,15 +22384,9 @@ class ThriftHiveMetastore_get_config_value_args { switch ($fid) { case 1: - if ($ftype == TType::STRING) { - $xfer += $input->readString($this->name); - } else { - $xfer += $input->skip($ftype); - } - break; - case 2: - if ($ftype == TType::STRING) { - $xfer += $input->readString($this->defaultValue); + if ($ftype == TType::STRUCT) { + $this->stats_obj = new \metastore\ColumnStatistics(); + $xfer += $this->stats_obj->read($input); } else { $xfer += $input->skip($ftype); } @@ -18788,15 +22403,13 @@ class ThriftHiveMetastore_get_config_value_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_config_value_args'); - if ($this->name !== null) { - $xfer += $output->writeFieldBegin('name', TType::STRING, 1); - $xfer += $output->writeString($this->name); - $xfer += $output->writeFieldEnd(); - } - if ($this->defaultValue !== null) { - $xfer += $output->writeFieldBegin('defaultValue', TType::STRING, 2); - $xfer += $output->writeString($this->defaultValue); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_update_partition_column_statistics_args'); + if ($this->stats_obj !== null) { + if (!is_object($this->stats_obj)) { + throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); + } + $xfer += $output->writeFieldBegin('stats_obj', TType::STRUCT, 1); + $xfer += $this->stats_obj->write($output); $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -18806,23 +22419,41 @@ class ThriftHiveMetastore_get_config_value_args { } -class ThriftHiveMetastore_get_config_value_result { +class ThriftHiveMetastore_update_partition_column_statistics_result { static $_TSPEC; public $success = null; public $o1 = null; + public $o2 = null; + public $o3 = null; + public $o4 = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 0 => array( 'var' => 'success', - 'type' => TType::STRING, + 'type' => TType::BOOL, ), 1 => array( 'var' => 'o1', 'type' => TType::STRUCT, - 'class' => '\metastore\ConfigValSecurityException', + 'class' => '\metastore\NoSuchObjectException', + ), + 2 => array( + 'var' => 'o2', + 'type' => TType::STRUCT, + 'class' => '\metastore\InvalidObjectException', + ), + 3 => array( + 'var' => 'o3', + 'type' => TType::STRUCT, + 'class' => '\metastore\MetaException', + ), + 4 => array( + 'var' => 'o4', + 'type' => TType::STRUCT, + 'class' => '\metastore\InvalidInputException', ), ); } @@ -18833,11 +22464,20 @@ class ThriftHiveMetastore_get_config_value_result { if (isset($vals['o1'])) { $this->o1 = $vals['o1']; } + if (isset($vals['o2'])) { + $this->o2 = $vals['o2']; + } + if (isset($vals['o3'])) { + $this->o3 = $vals['o3']; + } + if (isset($vals['o4'])) { + $this->o4 = $vals['o4']; + } } } public function getName() { - return 'ThriftHiveMetastore_get_config_value_result'; + return 'ThriftHiveMetastore_update_partition_column_statistics_result'; } public function read($input) @@ -18856,20 +22496,44 @@ class ThriftHiveMetastore_get_config_value_result { switch ($fid) { case 0: - if ($ftype == TType::STRING) { - $xfer += $input->readString($this->success); + if ($ftype == TType::BOOL) { + $xfer += $input->readBool($this->success); } else { $xfer += $input->skip($ftype); } break; case 1: if ($ftype == TType::STRUCT) { - $this->o1 = new \metastore\ConfigValSecurityException(); + $this->o1 = new \metastore\NoSuchObjectException(); $xfer += $this->o1->read($input); } else { $xfer += $input->skip($ftype); } break; + case 2: + if ($ftype == TType::STRUCT) { + $this->o2 = new \metastore\InvalidObjectException(); + $xfer += $this->o2->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; + case 3: + if ($ftype == TType::STRUCT) { + $this->o3 = new \metastore\MetaException(); + $xfer += $this->o3->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; + case 4: + if ($ftype == TType::STRUCT) { + $this->o4 = new \metastore\InvalidInputException(); + $xfer += $this->o4->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; default: $xfer += $input->skip($ftype); break; @@ -18882,10 +22546,10 @@ class ThriftHiveMetastore_get_config_value_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_config_value_result'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_update_partition_column_statistics_result'); if ($this->success !== null) { - $xfer += $output->writeFieldBegin('success', TType::STRING, 0); - $xfer += $output->writeString($this->success); + $xfer += $output->writeFieldBegin('success', TType::BOOL, 0); + $xfer += $output->writeBool($this->success); $xfer += $output->writeFieldEnd(); } if ($this->o1 !== null) { @@ -18893,6 +22557,21 @@ class ThriftHiveMetastore_get_config_value_result { $xfer += $this->o1->write($output); $xfer += $output->writeFieldEnd(); } + if ($this->o2 !== null) { + $xfer += $output->writeFieldBegin('o2', TType::STRUCT, 2); + $xfer += $this->o2->write($output); + $xfer += $output->writeFieldEnd(); + } + if ($this->o3 !== null) { + $xfer += $output->writeFieldBegin('o3', TType::STRUCT, 3); + $xfer += $this->o3->write($output); + $xfer += $output->writeFieldEnd(); + } + if ($this->o4 !== null) { + $xfer += $output->writeFieldBegin('o4', TType::STRUCT, 4); + $xfer += $this->o4->write($output); + $xfer += $output->writeFieldEnd(); + } $xfer += $output->writeFieldStop(); $xfer += $output->writeStructEnd(); return $xfer; @@ -18900,29 +22579,45 @@ class ThriftHiveMetastore_get_config_value_result { } -class ThriftHiveMetastore_partition_name_to_vals_args { +class ThriftHiveMetastore_get_table_column_statistics_args { static $_TSPEC; - public $part_name = null; + public $db_name = null; + public $tbl_name = null; + public $col_name = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 1 => array( - 'var' => 'part_name', + 'var' => 'db_name', + 'type' => TType::STRING, + ), + 2 => array( + 'var' => 'tbl_name', + 'type' => TType::STRING, + ), + 3 => array( + 'var' => 'col_name', 'type' => TType::STRING, ), ); } if (is_array($vals)) { - if (isset($vals['part_name'])) { - $this->part_name = $vals['part_name']; + if (isset($vals['db_name'])) { + $this->db_name = $vals['db_name']; + } + if (isset($vals['tbl_name'])) { + $this->tbl_name = $vals['tbl_name']; + } + if (isset($vals['col_name'])) { + $this->col_name = $vals['col_name']; } } } public function getName() { - return 'ThriftHiveMetastore_partition_name_to_vals_args'; + return 'ThriftHiveMetastore_get_table_column_statistics_args'; } public function read($input) @@ -18942,7 +22637,21 @@ class ThriftHiveMetastore_partition_name_to_vals_args { { case 1: if ($ftype == TType::STRING) { - $xfer += $input->readString($this->part_name); + $xfer += $input->readString($this->db_name); + } else { + $xfer += $input->skip($ftype); + } + break; + case 2: + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->tbl_name); + } else { + $xfer += $input->skip($ftype); + } + break; + case 3: + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->col_name); } else { $xfer += $input->skip($ftype); } @@ -18959,10 +22668,20 @@ class ThriftHiveMetastore_partition_name_to_vals_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_partition_name_to_vals_args'); - if ($this->part_name !== null) { - $xfer += $output->writeFieldBegin('part_name', TType::STRING, 1); - $xfer += $output->writeString($this->part_name); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_table_column_statistics_args'); + if ($this->db_name !== null) { + $xfer += $output->writeFieldBegin('db_name', TType::STRING, 1); + $xfer += $output->writeString($this->db_name); + $xfer += $output->writeFieldEnd(); + } + if ($this->tbl_name !== null) { + $xfer += $output->writeFieldBegin('tbl_name', TType::STRING, 2); + $xfer += $output->writeString($this->tbl_name); + $xfer += $output->writeFieldEnd(); + } + if ($this->col_name !== null) { + $xfer += $output->writeFieldBegin('col_name', TType::STRING, 3); + $xfer += $output->writeString($this->col_name); $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -18972,28 +22691,43 @@ class ThriftHiveMetastore_partition_name_to_vals_args { } -class ThriftHiveMetastore_partition_name_to_vals_result { +class ThriftHiveMetastore_get_table_column_statistics_result { static $_TSPEC; public $success = null; public $o1 = null; + public $o2 = null; + public $o3 = null; + public $o4 = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 0 => array( 'var' => 'success', - 'type' => TType::LST, - 'etype' => TType::STRING, - 'elem' => array( - 'type' => TType::STRING, - ), + 'type' => TType::STRUCT, + 'class' => '\metastore\ColumnStatistics', ), 1 => array( 'var' => 'o1', 'type' => TType::STRUCT, + 'class' => '\metastore\NoSuchObjectException', + ), + 2 => array( + 'var' => 'o2', + 'type' => TType::STRUCT, 'class' => '\metastore\MetaException', ), + 3 => array( + 'var' => 'o3', + 'type' => TType::STRUCT, + 'class' => '\metastore\InvalidInputException', + ), + 4 => array( + 'var' => 'o4', + 'type' => TType::STRUCT, + 'class' => '\metastore\InvalidObjectException', + ), ); } if (is_array($vals)) { @@ -19003,11 +22737,20 @@ class ThriftHiveMetastore_partition_name_to_vals_result { if (isset($vals['o1'])) { $this->o1 = $vals['o1']; } + if (isset($vals['o2'])) { + $this->o2 = $vals['o2']; + } + if (isset($vals['o3'])) { + $this->o3 = $vals['o3']; + } + if (isset($vals['o4'])) { + $this->o4 = $vals['o4']; + } } } public function getName() { - return 'ThriftHiveMetastore_partition_name_to_vals_result'; + return 'ThriftHiveMetastore_get_table_column_statistics_result'; } public function read($input) @@ -19026,30 +22769,45 @@ class ThriftHiveMetastore_partition_name_to_vals_result { switch ($fid) { case 0: - if ($ftype == TType::LST) { - $this->success = array(); - $_size576 = 0; - $_etype579 = 0; - $xfer += $input->readListBegin($_etype579, $_size576); - for ($_i580 = 0; $_i580 < $_size576; ++$_i580) - { - $elem581 = null; - $xfer += $input->readString($elem581); - $this->success []= $elem581; - } - $xfer += $input->readListEnd(); + if ($ftype == TType::STRUCT) { + $this->success = new \metastore\ColumnStatistics(); + $xfer += $this->success->read($input); } else { $xfer += $input->skip($ftype); } break; case 1: if ($ftype == TType::STRUCT) { - $this->o1 = new \metastore\MetaException(); + $this->o1 = new \metastore\NoSuchObjectException(); $xfer += $this->o1->read($input); } else { $xfer += $input->skip($ftype); } break; + case 2: + if ($ftype == TType::STRUCT) { + $this->o2 = new \metastore\MetaException(); + $xfer += $this->o2->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; + case 3: + if ($ftype == TType::STRUCT) { + $this->o3 = new \metastore\InvalidInputException(); + $xfer += $this->o3->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; + case 4: + if ($ftype == TType::STRUCT) { + $this->o4 = new \metastore\InvalidObjectException(); + $xfer += $this->o4->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; default: $xfer += $input->skip($ftype); break; @@ -19062,22 +22820,13 @@ class ThriftHiveMetastore_partition_name_to_vals_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_partition_name_to_vals_result'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_table_column_statistics_result'); if ($this->success !== null) { - if (!is_array($this->success)) { + if (!is_object($this->success)) { throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); } - $xfer += $output->writeFieldBegin('success', TType::LST, 0); - { - $output->writeListBegin(TType::STRING, count($this->success)); - { - foreach ($this->success as $iter582) - { - $xfer += $output->writeString($iter582); - } - } - $output->writeListEnd(); - } + $xfer += $output->writeFieldBegin('success', TType::STRUCT, 0); + $xfer += $this->success->write($output); $xfer += $output->writeFieldEnd(); } if ($this->o1 !== null) { @@ -19085,6 +22834,21 @@ class ThriftHiveMetastore_partition_name_to_vals_result { $xfer += $this->o1->write($output); $xfer += $output->writeFieldEnd(); } + if ($this->o2 !== null) { + $xfer += $output->writeFieldBegin('o2', TType::STRUCT, 2); + $xfer += $this->o2->write($output); + $xfer += $output->writeFieldEnd(); + } + if ($this->o3 !== null) { + $xfer += $output->writeFieldBegin('o3', TType::STRUCT, 3); + $xfer += $this->o3->write($output); + $xfer += $output->writeFieldEnd(); + } + if ($this->o4 !== null) { + $xfer += $output->writeFieldBegin('o4', TType::STRUCT, 4); + $xfer += $this->o4->write($output); + $xfer += $output->writeFieldEnd(); + } $xfer += $output->writeFieldStop(); $xfer += $output->writeStructEnd(); return $xfer; @@ -19092,29 +22856,53 @@ class ThriftHiveMetastore_partition_name_to_vals_result { } -class ThriftHiveMetastore_partition_name_to_spec_args { +class ThriftHiveMetastore_get_partition_column_statistics_args { static $_TSPEC; + public $db_name = null; + public $tbl_name = null; public $part_name = null; + public $col_name = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 1 => array( + 'var' => 'db_name', + 'type' => TType::STRING, + ), + 2 => array( + 'var' => 'tbl_name', + 'type' => TType::STRING, + ), + 3 => array( 'var' => 'part_name', 'type' => TType::STRING, ), + 4 => array( + 'var' => 'col_name', + 'type' => TType::STRING, + ), ); } if (is_array($vals)) { + if (isset($vals['db_name'])) { + $this->db_name = $vals['db_name']; + } + if (isset($vals['tbl_name'])) { + $this->tbl_name = $vals['tbl_name']; + } if (isset($vals['part_name'])) { $this->part_name = $vals['part_name']; } + if (isset($vals['col_name'])) { + $this->col_name = $vals['col_name']; + } } } public function getName() { - return 'ThriftHiveMetastore_partition_name_to_spec_args'; + return 'ThriftHiveMetastore_get_partition_column_statistics_args'; } public function read($input) @@ -19134,11 +22922,32 @@ class ThriftHiveMetastore_partition_name_to_spec_args { { case 1: if ($ftype == TType::STRING) { + $xfer += $input->readString($this->db_name); + } else { + $xfer += $input->skip($ftype); + } + break; + case 2: + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->tbl_name); + } else { + $xfer += $input->skip($ftype); + } + break; + case 3: + if ($ftype == TType::STRING) { $xfer += $input->readString($this->part_name); } else { $xfer += $input->skip($ftype); } break; + case 4: + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->col_name); + } else { + $xfer += $input->skip($ftype); + } + break; default: $xfer += $input->skip($ftype); break; @@ -19151,12 +22960,27 @@ class ThriftHiveMetastore_partition_name_to_spec_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_partition_name_to_spec_args'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_partition_column_statistics_args'); + if ($this->db_name !== null) { + $xfer += $output->writeFieldBegin('db_name', TType::STRING, 1); + $xfer += $output->writeString($this->db_name); + $xfer += $output->writeFieldEnd(); + } + if ($this->tbl_name !== null) { + $xfer += $output->writeFieldBegin('tbl_name', TType::STRING, 2); + $xfer += $output->writeString($this->tbl_name); + $xfer += $output->writeFieldEnd(); + } if ($this->part_name !== null) { - $xfer += $output->writeFieldBegin('part_name', TType::STRING, 1); + $xfer += $output->writeFieldBegin('part_name', TType::STRING, 3); $xfer += $output->writeString($this->part_name); $xfer += $output->writeFieldEnd(); } + if ($this->col_name !== null) { + $xfer += $output->writeFieldBegin('col_name', TType::STRING, 4); + $xfer += $output->writeString($this->col_name); + $xfer += $output->writeFieldEnd(); + } $xfer += $output->writeFieldStop(); $xfer += $output->writeStructEnd(); return $xfer; @@ -19164,32 +22988,43 @@ class ThriftHiveMetastore_partition_name_to_spec_args { } -class ThriftHiveMetastore_partition_name_to_spec_result { +class ThriftHiveMetastore_get_partition_column_statistics_result { static $_TSPEC; public $success = null; public $o1 = null; + public $o2 = null; + public $o3 = null; + public $o4 = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 0 => array( 'var' => 'success', - 'type' => TType::MAP, - 'ktype' => TType::STRING, - 'vtype' => TType::STRING, - 'key' => array( - 'type' => TType::STRING, - ), - 'val' => array( - 'type' => TType::STRING, - ), + 'type' => TType::STRUCT, + 'class' => '\metastore\ColumnStatistics', ), 1 => array( 'var' => 'o1', 'type' => TType::STRUCT, + 'class' => '\metastore\NoSuchObjectException', + ), + 2 => array( + 'var' => 'o2', + 'type' => TType::STRUCT, 'class' => '\metastore\MetaException', ), + 3 => array( + 'var' => 'o3', + 'type' => TType::STRUCT, + 'class' => '\metastore\InvalidInputException', + ), + 4 => array( + 'var' => 'o4', + 'type' => TType::STRUCT, + 'class' => '\metastore\InvalidObjectException', + ), ); } if (is_array($vals)) { @@ -19199,11 +23034,20 @@ class ThriftHiveMetastore_partition_name_to_spec_result { if (isset($vals['o1'])) { $this->o1 = $vals['o1']; } + if (isset($vals['o2'])) { + $this->o2 = $vals['o2']; + } + if (isset($vals['o3'])) { + $this->o3 = $vals['o3']; + } + if (isset($vals['o4'])) { + $this->o4 = $vals['o4']; + } } } public function getName() { - return 'ThriftHiveMetastore_partition_name_to_spec_result'; + return 'ThriftHiveMetastore_get_partition_column_statistics_result'; } public function read($input) @@ -19222,33 +23066,45 @@ class ThriftHiveMetastore_partition_name_to_spec_result { switch ($fid) { case 0: - if ($ftype == TType::MAP) { - $this->success = array(); - $_size583 = 0; - $_ktype584 = 0; - $_vtype585 = 0; - $xfer += $input->readMapBegin($_ktype584, $_vtype585, $_size583); - for ($_i587 = 0; $_i587 < $_size583; ++$_i587) - { - $key588 = ''; - $val589 = ''; - $xfer += $input->readString($key588); - $xfer += $input->readString($val589); - $this->success[$key588] = $val589; - } - $xfer += $input->readMapEnd(); + if ($ftype == TType::STRUCT) { + $this->success = new \metastore\ColumnStatistics(); + $xfer += $this->success->read($input); } else { $xfer += $input->skip($ftype); } break; case 1: if ($ftype == TType::STRUCT) { - $this->o1 = new \metastore\MetaException(); + $this->o1 = new \metastore\NoSuchObjectException(); $xfer += $this->o1->read($input); } else { $xfer += $input->skip($ftype); } break; + case 2: + if ($ftype == TType::STRUCT) { + $this->o2 = new \metastore\MetaException(); + $xfer += $this->o2->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; + case 3: + if ($ftype == TType::STRUCT) { + $this->o3 = new \metastore\InvalidInputException(); + $xfer += $this->o3->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; + case 4: + if ($ftype == TType::STRUCT) { + $this->o4 = new \metastore\InvalidObjectException(); + $xfer += $this->o4->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; default: $xfer += $input->skip($ftype); break; @@ -19261,23 +23117,13 @@ class ThriftHiveMetastore_partition_name_to_spec_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_partition_name_to_spec_result'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_partition_column_statistics_result'); if ($this->success !== null) { - if (!is_array($this->success)) { + if (!is_object($this->success)) { throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); } - $xfer += $output->writeFieldBegin('success', TType::MAP, 0); - { - $output->writeMapBegin(TType::STRING, TType::STRING, count($this->success)); - { - foreach ($this->success as $kiter590 => $viter591) - { - $xfer += $output->writeString($kiter590); - $xfer += $output->writeString($viter591); - } - } - $output->writeMapEnd(); - } + $xfer += $output->writeFieldBegin('success', TType::STRUCT, 0); + $xfer += $this->success->write($output); $xfer += $output->writeFieldEnd(); } if ($this->o1 !== null) { @@ -19285,6 +23131,21 @@ class ThriftHiveMetastore_partition_name_to_spec_result { $xfer += $this->o1->write($output); $xfer += $output->writeFieldEnd(); } + if ($this->o2 !== null) { + $xfer += $output->writeFieldBegin('o2', TType::STRUCT, 2); + $xfer += $this->o2->write($output); + $xfer += $output->writeFieldEnd(); + } + if ($this->o3 !== null) { + $xfer += $output->writeFieldBegin('o3', TType::STRUCT, 3); + $xfer += $this->o3->write($output); + $xfer += $output->writeFieldEnd(); + } + if ($this->o4 !== null) { + $xfer += $output->writeFieldBegin('o4', TType::STRUCT, 4); + $xfer += $this->o4->write($output); + $xfer += $output->writeFieldEnd(); + } $xfer += $output->writeFieldStop(); $xfer += $output->writeStructEnd(); return $xfer; @@ -19292,61 +23153,30 @@ class ThriftHiveMetastore_partition_name_to_spec_result { } -class ThriftHiveMetastore_markPartitionForEvent_args { +class ThriftHiveMetastore_get_table_statistics_req_args { static $_TSPEC; - public $db_name = null; - public $tbl_name = null; - public $part_vals = null; - public $eventType = null; + public $request = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 1 => array( - 'var' => 'db_name', - 'type' => TType::STRING, - ), - 2 => array( - 'var' => 'tbl_name', - 'type' => TType::STRING, - ), - 3 => array( - 'var' => 'part_vals', - 'type' => TType::MAP, - 'ktype' => TType::STRING, - 'vtype' => TType::STRING, - 'key' => array( - 'type' => TType::STRING, - ), - 'val' => array( - 'type' => TType::STRING, - ), - ), - 4 => array( - 'var' => 'eventType', - 'type' => TType::I32, + 'var' => 'request', + 'type' => TType::STRUCT, + 'class' => '\metastore\TableStatsRequest', ), ); } if (is_array($vals)) { - if (isset($vals['db_name'])) { - $this->db_name = $vals['db_name']; - } - if (isset($vals['tbl_name'])) { - $this->tbl_name = $vals['tbl_name']; - } - if (isset($vals['part_vals'])) { - $this->part_vals = $vals['part_vals']; - } - if (isset($vals['eventType'])) { - $this->eventType = $vals['eventType']; + if (isset($vals['request'])) { + $this->request = $vals['request']; } } } public function getName() { - return 'ThriftHiveMetastore_markPartitionForEvent_args'; + return 'ThriftHiveMetastore_get_table_statistics_req_args'; } public function read($input) @@ -19365,42 +23195,9 @@ class ThriftHiveMetastore_markPartitionForEvent_args { switch ($fid) { case 1: - if ($ftype == TType::STRING) { - $xfer += $input->readString($this->db_name); - } else { - $xfer += $input->skip($ftype); - } - break; - case 2: - if ($ftype == TType::STRING) { - $xfer += $input->readString($this->tbl_name); - } else { - $xfer += $input->skip($ftype); - } - break; - case 3: - if ($ftype == TType::MAP) { - $this->part_vals = array(); - $_size592 = 0; - $_ktype593 = 0; - $_vtype594 = 0; - $xfer += $input->readMapBegin($_ktype593, $_vtype594, $_size592); - for ($_i596 = 0; $_i596 < $_size592; ++$_i596) - { - $key597 = ''; - $val598 = ''; - $xfer += $input->readString($key597); - $xfer += $input->readString($val598); - $this->part_vals[$key597] = $val598; - } - $xfer += $input->readMapEnd(); - } else { - $xfer += $input->skip($ftype); - } - break; - case 4: - if ($ftype == TType::I32) { - $xfer += $input->readI32($this->eventType); + if ($ftype == TType::STRUCT) { + $this->request = new \metastore\TableStatsRequest(); + $xfer += $this->request->read($input); } else { $xfer += $input->skip($ftype); } @@ -19417,38 +23214,13 @@ class ThriftHiveMetastore_markPartitionForEvent_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_markPartitionForEvent_args'); - if ($this->db_name !== null) { - $xfer += $output->writeFieldBegin('db_name', TType::STRING, 1); - $xfer += $output->writeString($this->db_name); - $xfer += $output->writeFieldEnd(); - } - if ($this->tbl_name !== null) { - $xfer += $output->writeFieldBegin('tbl_name', TType::STRING, 2); - $xfer += $output->writeString($this->tbl_name); - $xfer += $output->writeFieldEnd(); - } - if ($this->part_vals !== null) { - if (!is_array($this->part_vals)) { + $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_table_statistics_req_args'); + if ($this->request !== null) { + if (!is_object($this->request)) { throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); } - $xfer += $output->writeFieldBegin('part_vals', TType::MAP, 3); - { - $output->writeMapBegin(TType::STRING, TType::STRING, count($this->part_vals)); - { - foreach ($this->part_vals as $kiter599 => $viter600) - { - $xfer += $output->writeString($kiter599); - $xfer += $output->writeString($viter600); - } - } - $output->writeMapEnd(); - } - $xfer += $output->writeFieldEnd(); - } - if ($this->eventType !== null) { - $xfer += $output->writeFieldBegin('eventType', TType::I32, 4); - $xfer += $output->writeI32($this->eventType); + $xfer += $output->writeFieldBegin('request', TType::STRUCT, 1); + $xfer += $this->request->write($output); $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -19458,75 +23230,48 @@ class ThriftHiveMetastore_markPartitionForEvent_args { } -class ThriftHiveMetastore_markPartitionForEvent_result { +class ThriftHiveMetastore_get_table_statistics_req_result { static $_TSPEC; + public $success = null; public $o1 = null; public $o2 = null; - public $o3 = null; - public $o4 = null; - public $o5 = null; - public $o6 = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( + 0 => array( + 'var' => 'success', + 'type' => TType::STRUCT, + 'class' => '\metastore\TableStatsResult', + ), 1 => array( 'var' => 'o1', 'type' => TType::STRUCT, - 'class' => '\metastore\MetaException', + 'class' => '\metastore\NoSuchObjectException', ), 2 => array( 'var' => 'o2', 'type' => TType::STRUCT, - 'class' => '\metastore\NoSuchObjectException', - ), - 3 => array( - 'var' => 'o3', - 'type' => TType::STRUCT, - 'class' => '\metastore\UnknownDBException', - ), - 4 => array( - 'var' => 'o4', - 'type' => TType::STRUCT, - 'class' => '\metastore\UnknownTableException', - ), - 5 => array( - 'var' => 'o5', - 'type' => TType::STRUCT, - 'class' => '\metastore\UnknownPartitionException', - ), - 6 => array( - 'var' => 'o6', - 'type' => TType::STRUCT, - 'class' => '\metastore\InvalidPartitionException', + 'class' => '\metastore\MetaException', ), ); } if (is_array($vals)) { + if (isset($vals['success'])) { + $this->success = $vals['success']; + } if (isset($vals['o1'])) { $this->o1 = $vals['o1']; } if (isset($vals['o2'])) { $this->o2 = $vals['o2']; } - if (isset($vals['o3'])) { - $this->o3 = $vals['o3']; - } - if (isset($vals['o4'])) { - $this->o4 = $vals['o4']; - } - if (isset($vals['o5'])) { - $this->o5 = $vals['o5']; - } - if (isset($vals['o6'])) { - $this->o6 = $vals['o6']; - } } } public function getName() { - return 'ThriftHiveMetastore_markPartitionForEvent_result'; + return 'ThriftHiveMetastore_get_table_statistics_req_result'; } public function read($input) @@ -19544,50 +23289,26 @@ class ThriftHiveMetastore_markPartitionForEvent_result { } switch ($fid) { - case 1: - if ($ftype == TType::STRUCT) { - $this->o1 = new \metastore\MetaException(); - $xfer += $this->o1->read($input); - } else { - $xfer += $input->skip($ftype); - } - break; - case 2: - if ($ftype == TType::STRUCT) { - $this->o2 = new \metastore\NoSuchObjectException(); - $xfer += $this->o2->read($input); - } else { - $xfer += $input->skip($ftype); - } - break; - case 3: - if ($ftype == TType::STRUCT) { - $this->o3 = new \metastore\UnknownDBException(); - $xfer += $this->o3->read($input); - } else { - $xfer += $input->skip($ftype); - } - break; - case 4: + case 0: if ($ftype == TType::STRUCT) { - $this->o4 = new \metastore\UnknownTableException(); - $xfer += $this->o4->read($input); + $this->success = new \metastore\TableStatsResult(); + $xfer += $this->success->read($input); } else { $xfer += $input->skip($ftype); } break; - case 5: + case 1: if ($ftype == TType::STRUCT) { - $this->o5 = new \metastore\UnknownPartitionException(); - $xfer += $this->o5->read($input); + $this->o1 = new \metastore\NoSuchObjectException(); + $xfer += $this->o1->read($input); } else { $xfer += $input->skip($ftype); } break; - case 6: + case 2: if ($ftype == TType::STRUCT) { - $this->o6 = new \metastore\InvalidPartitionException(); - $xfer += $this->o6->read($input); + $this->o2 = new \metastore\MetaException(); + $xfer += $this->o2->read($input); } else { $xfer += $input->skip($ftype); } @@ -19604,7 +23325,15 @@ class ThriftHiveMetastore_markPartitionForEvent_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_markPartitionForEvent_result'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_table_statistics_req_result'); + if ($this->success !== null) { + if (!is_object($this->success)) { + throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); + } + $xfer += $output->writeFieldBegin('success', TType::STRUCT, 0); + $xfer += $this->success->write($output); + $xfer += $output->writeFieldEnd(); + } if ($this->o1 !== null) { $xfer += $output->writeFieldBegin('o1', TType::STRUCT, 1); $xfer += $this->o1->write($output); @@ -19615,26 +23344,6 @@ class ThriftHiveMetastore_markPartitionForEvent_result { $xfer += $this->o2->write($output); $xfer += $output->writeFieldEnd(); } - if ($this->o3 !== null) { - $xfer += $output->writeFieldBegin('o3', TType::STRUCT, 3); - $xfer += $this->o3->write($output); - $xfer += $output->writeFieldEnd(); - } - if ($this->o4 !== null) { - $xfer += $output->writeFieldBegin('o4', TType::STRUCT, 4); - $xfer += $this->o4->write($output); - $xfer += $output->writeFieldEnd(); - } - if ($this->o5 !== null) { - $xfer += $output->writeFieldBegin('o5', TType::STRUCT, 5); - $xfer += $this->o5->write($output); - $xfer += $output->writeFieldEnd(); - } - if ($this->o6 !== null) { - $xfer += $output->writeFieldBegin('o6', TType::STRUCT, 6); - $xfer += $this->o6->write($output); - $xfer += $output->writeFieldEnd(); - } $xfer += $output->writeFieldStop(); $xfer += $output->writeStructEnd(); return $xfer; @@ -19642,61 +23351,30 @@ class ThriftHiveMetastore_markPartitionForEvent_result { } -class ThriftHiveMetastore_isPartitionMarkedForEvent_args { +class ThriftHiveMetastore_get_partitions_statistics_req_args { static $_TSPEC; - public $db_name = null; - public $tbl_name = null; - public $part_vals = null; - public $eventType = null; + public $request = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 1 => array( - 'var' => 'db_name', - 'type' => TType::STRING, - ), - 2 => array( - 'var' => 'tbl_name', - 'type' => TType::STRING, - ), - 3 => array( - 'var' => 'part_vals', - 'type' => TType::MAP, - 'ktype' => TType::STRING, - 'vtype' => TType::STRING, - 'key' => array( - 'type' => TType::STRING, - ), - 'val' => array( - 'type' => TType::STRING, - ), - ), - 4 => array( - 'var' => 'eventType', - 'type' => TType::I32, + 'var' => 'request', + 'type' => TType::STRUCT, + 'class' => '\metastore\PartitionsStatsRequest', ), ); } if (is_array($vals)) { - if (isset($vals['db_name'])) { - $this->db_name = $vals['db_name']; - } - if (isset($vals['tbl_name'])) { - $this->tbl_name = $vals['tbl_name']; - } - if (isset($vals['part_vals'])) { - $this->part_vals = $vals['part_vals']; - } - if (isset($vals['eventType'])) { - $this->eventType = $vals['eventType']; + if (isset($vals['request'])) { + $this->request = $vals['request']; } } } public function getName() { - return 'ThriftHiveMetastore_isPartitionMarkedForEvent_args'; + return 'ThriftHiveMetastore_get_partitions_statistics_req_args'; } public function read($input) @@ -19715,42 +23393,9 @@ class ThriftHiveMetastore_isPartitionMarkedForEvent_args { switch ($fid) { case 1: - if ($ftype == TType::STRING) { - $xfer += $input->readString($this->db_name); - } else { - $xfer += $input->skip($ftype); - } - break; - case 2: - if ($ftype == TType::STRING) { - $xfer += $input->readString($this->tbl_name); - } else { - $xfer += $input->skip($ftype); - } - break; - case 3: - if ($ftype == TType::MAP) { - $this->part_vals = array(); - $_size601 = 0; - $_ktype602 = 0; - $_vtype603 = 0; - $xfer += $input->readMapBegin($_ktype602, $_vtype603, $_size601); - for ($_i605 = 0; $_i605 < $_size601; ++$_i605) - { - $key606 = ''; - $val607 = ''; - $xfer += $input->readString($key606); - $xfer += $input->readString($val607); - $this->part_vals[$key606] = $val607; - } - $xfer += $input->readMapEnd(); - } else { - $xfer += $input->skip($ftype); - } - break; - case 4: - if ($ftype == TType::I32) { - $xfer += $input->readI32($this->eventType); + if ($ftype == TType::STRUCT) { + $this->request = new \metastore\PartitionsStatsRequest(); + $xfer += $this->request->read($input); } else { $xfer += $input->skip($ftype); } @@ -19767,38 +23412,13 @@ class ThriftHiveMetastore_isPartitionMarkedForEvent_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_isPartitionMarkedForEvent_args'); - if ($this->db_name !== null) { - $xfer += $output->writeFieldBegin('db_name', TType::STRING, 1); - $xfer += $output->writeString($this->db_name); - $xfer += $output->writeFieldEnd(); - } - if ($this->tbl_name !== null) { - $xfer += $output->writeFieldBegin('tbl_name', TType::STRING, 2); - $xfer += $output->writeString($this->tbl_name); - $xfer += $output->writeFieldEnd(); - } - if ($this->part_vals !== null) { - if (!is_array($this->part_vals)) { + $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_partitions_statistics_req_args'); + if ($this->request !== null) { + if (!is_object($this->request)) { throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); } - $xfer += $output->writeFieldBegin('part_vals', TType::MAP, 3); - { - $output->writeMapBegin(TType::STRING, TType::STRING, count($this->part_vals)); - { - foreach ($this->part_vals as $kiter608 => $viter609) - { - $xfer += $output->writeString($kiter608); - $xfer += $output->writeString($viter609); - } - } - $output->writeMapEnd(); - } - $xfer += $output->writeFieldEnd(); - } - if ($this->eventType !== null) { - $xfer += $output->writeFieldBegin('eventType', TType::I32, 4); - $xfer += $output->writeI32($this->eventType); + $xfer += $output->writeFieldBegin('request', TType::STRUCT, 1); + $xfer += $this->request->write($output); $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -19808,53 +23428,30 @@ class ThriftHiveMetastore_isPartitionMarkedForEvent_args { } -class ThriftHiveMetastore_isPartitionMarkedForEvent_result { +class ThriftHiveMetastore_get_partitions_statistics_req_result { static $_TSPEC; public $success = null; public $o1 = null; public $o2 = null; - public $o3 = null; - public $o4 = null; - public $o5 = null; - public $o6 = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 0 => array( 'var' => 'success', - 'type' => TType::BOOL, + 'type' => TType::STRUCT, + 'class' => '\metastore\PartitionsStatsResult', ), 1 => array( 'var' => 'o1', 'type' => TType::STRUCT, - 'class' => '\metastore\MetaException', + 'class' => '\metastore\NoSuchObjectException', ), 2 => array( 'var' => 'o2', 'type' => TType::STRUCT, - 'class' => '\metastore\NoSuchObjectException', - ), - 3 => array( - 'var' => 'o3', - 'type' => TType::STRUCT, - 'class' => '\metastore\UnknownDBException', - ), - 4 => array( - 'var' => 'o4', - 'type' => TType::STRUCT, - 'class' => '\metastore\UnknownTableException', - ), - 5 => array( - 'var' => 'o5', - 'type' => TType::STRUCT, - 'class' => '\metastore\UnknownPartitionException', - ), - 6 => array( - 'var' => 'o6', - 'type' => TType::STRUCT, - 'class' => '\metastore\InvalidPartitionException', + 'class' => '\metastore\MetaException', ), ); } @@ -19868,23 +23465,11 @@ class ThriftHiveMetastore_isPartitionMarkedForEvent_result { if (isset($vals['o2'])) { $this->o2 = $vals['o2']; } - if (isset($vals['o3'])) { - $this->o3 = $vals['o3']; - } - if (isset($vals['o4'])) { - $this->o4 = $vals['o4']; - } - if (isset($vals['o5'])) { - $this->o5 = $vals['o5']; - } - if (isset($vals['o6'])) { - $this->o6 = $vals['o6']; - } } } public function getName() { - return 'ThriftHiveMetastore_isPartitionMarkedForEvent_result'; + return 'ThriftHiveMetastore_get_partitions_statistics_req_result'; } public function read($input) @@ -19903,15 +23488,16 @@ class ThriftHiveMetastore_isPartitionMarkedForEvent_result { switch ($fid) { case 0: - if ($ftype == TType::BOOL) { - $xfer += $input->readBool($this->success); + if ($ftype == TType::STRUCT) { + $this->success = new \metastore\PartitionsStatsResult(); + $xfer += $this->success->read($input); } else { $xfer += $input->skip($ftype); } break; case 1: if ($ftype == TType::STRUCT) { - $this->o1 = new \metastore\MetaException(); + $this->o1 = new \metastore\NoSuchObjectException(); $xfer += $this->o1->read($input); } else { $xfer += $input->skip($ftype); @@ -19919,44 +23505,12 @@ class ThriftHiveMetastore_isPartitionMarkedForEvent_result { break; case 2: if ($ftype == TType::STRUCT) { - $this->o2 = new \metastore\NoSuchObjectException(); + $this->o2 = new \metastore\MetaException(); $xfer += $this->o2->read($input); } else { $xfer += $input->skip($ftype); } break; - case 3: - if ($ftype == TType::STRUCT) { - $this->o3 = new \metastore\UnknownDBException(); - $xfer += $this->o3->read($input); - } else { - $xfer += $input->skip($ftype); - } - break; - case 4: - if ($ftype == TType::STRUCT) { - $this->o4 = new \metastore\UnknownTableException(); - $xfer += $this->o4->read($input); - } else { - $xfer += $input->skip($ftype); - } - break; - case 5: - if ($ftype == TType::STRUCT) { - $this->o5 = new \metastore\UnknownPartitionException(); - $xfer += $this->o5->read($input); - } else { - $xfer += $input->skip($ftype); - } - break; - case 6: - if ($ftype == TType::STRUCT) { - $this->o6 = new \metastore\InvalidPartitionException(); - $xfer += $this->o6->read($input); - } else { - $xfer += $input->skip($ftype); - } - break; default: $xfer += $input->skip($ftype); break; @@ -19969,10 +23523,13 @@ class ThriftHiveMetastore_isPartitionMarkedForEvent_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_isPartitionMarkedForEvent_result'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_partitions_statistics_req_result'); if ($this->success !== null) { - $xfer += $output->writeFieldBegin('success', TType::BOOL, 0); - $xfer += $output->writeBool($this->success); + if (!is_object($this->success)) { + throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); + } + $xfer += $output->writeFieldBegin('success', TType::STRUCT, 0); + $xfer += $this->success->write($output); $xfer += $output->writeFieldEnd(); } if ($this->o1 !== null) { @@ -19985,26 +23542,6 @@ class ThriftHiveMetastore_isPartitionMarkedForEvent_result { $xfer += $this->o2->write($output); $xfer += $output->writeFieldEnd(); } - if ($this->o3 !== null) { - $xfer += $output->writeFieldBegin('o3', TType::STRUCT, 3); - $xfer += $this->o3->write($output); - $xfer += $output->writeFieldEnd(); - } - if ($this->o4 !== null) { - $xfer += $output->writeFieldBegin('o4', TType::STRUCT, 4); - $xfer += $this->o4->write($output); - $xfer += $output->writeFieldEnd(); - } - if ($this->o5 !== null) { - $xfer += $output->writeFieldBegin('o5', TType::STRUCT, 5); - $xfer += $this->o5->write($output); - $xfer += $output->writeFieldEnd(); - } - if ($this->o6 !== null) { - $xfer += $output->writeFieldBegin('o6', TType::STRUCT, 6); - $xfer += $this->o6->write($output); - $xfer += $output->writeFieldEnd(); - } $xfer += $output->writeFieldStop(); $xfer += $output->writeStructEnd(); return $xfer; @@ -20012,39 +23549,53 @@ class ThriftHiveMetastore_isPartitionMarkedForEvent_result { } -class ThriftHiveMetastore_add_index_args { +class ThriftHiveMetastore_delete_partition_column_statistics_args { static $_TSPEC; - public $new_index = null; - public $index_table = null; + public $db_name = null; + public $tbl_name = null; + public $part_name = null; + public $col_name = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 1 => array( - 'var' => 'new_index', - 'type' => TType::STRUCT, - 'class' => '\metastore\Index', + 'var' => 'db_name', + 'type' => TType::STRING, ), 2 => array( - 'var' => 'index_table', - 'type' => TType::STRUCT, - 'class' => '\metastore\Table', + 'var' => 'tbl_name', + 'type' => TType::STRING, + ), + 3 => array( + 'var' => 'part_name', + 'type' => TType::STRING, + ), + 4 => array( + 'var' => 'col_name', + 'type' => TType::STRING, ), ); } if (is_array($vals)) { - if (isset($vals['new_index'])) { - $this->new_index = $vals['new_index']; + if (isset($vals['db_name'])) { + $this->db_name = $vals['db_name']; } - if (isset($vals['index_table'])) { - $this->index_table = $vals['index_table']; + if (isset($vals['tbl_name'])) { + $this->tbl_name = $vals['tbl_name']; + } + if (isset($vals['part_name'])) { + $this->part_name = $vals['part_name']; + } + if (isset($vals['col_name'])) { + $this->col_name = $vals['col_name']; } } } public function getName() { - return 'ThriftHiveMetastore_add_index_args'; + return 'ThriftHiveMetastore_delete_partition_column_statistics_args'; } public function read($input) @@ -20063,17 +23614,29 @@ class ThriftHiveMetastore_add_index_args { switch ($fid) { case 1: - if ($ftype == TType::STRUCT) { - $this->new_index = new \metastore\Index(); - $xfer += $this->new_index->read($input); + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->db_name); } else { $xfer += $input->skip($ftype); } break; case 2: - if ($ftype == TType::STRUCT) { - $this->index_table = new \metastore\Table(); - $xfer += $this->index_table->read($input); + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->tbl_name); + } else { + $xfer += $input->skip($ftype); + } + break; + case 3: + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->part_name); + } else { + $xfer += $input->skip($ftype); + } + break; + case 4: + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->col_name); } else { $xfer += $input->skip($ftype); } @@ -20090,21 +23653,25 @@ class ThriftHiveMetastore_add_index_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_add_index_args'); - if ($this->new_index !== null) { - if (!is_object($this->new_index)) { - throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); - } - $xfer += $output->writeFieldBegin('new_index', TType::STRUCT, 1); - $xfer += $this->new_index->write($output); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_delete_partition_column_statistics_args'); + if ($this->db_name !== null) { + $xfer += $output->writeFieldBegin('db_name', TType::STRING, 1); + $xfer += $output->writeString($this->db_name); $xfer += $output->writeFieldEnd(); } - if ($this->index_table !== null) { - if (!is_object($this->index_table)) { - throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); - } - $xfer += $output->writeFieldBegin('index_table', TType::STRUCT, 2); - $xfer += $this->index_table->write($output); + if ($this->tbl_name !== null) { + $xfer += $output->writeFieldBegin('tbl_name', TType::STRING, 2); + $xfer += $output->writeString($this->tbl_name); + $xfer += $output->writeFieldEnd(); + } + if ($this->part_name !== null) { + $xfer += $output->writeFieldBegin('part_name', TType::STRING, 3); + $xfer += $output->writeString($this->part_name); + $xfer += $output->writeFieldEnd(); + } + if ($this->col_name !== null) { + $xfer += $output->writeFieldBegin('col_name', TType::STRING, 4); + $xfer += $output->writeString($this->col_name); $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -20114,36 +23681,41 @@ class ThriftHiveMetastore_add_index_args { } -class ThriftHiveMetastore_add_index_result { +class ThriftHiveMetastore_delete_partition_column_statistics_result { static $_TSPEC; public $success = null; public $o1 = null; public $o2 = null; public $o3 = null; + public $o4 = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 0 => array( 'var' => 'success', - 'type' => TType::STRUCT, - 'class' => '\metastore\Index', + 'type' => TType::BOOL, ), 1 => array( 'var' => 'o1', 'type' => TType::STRUCT, - 'class' => '\metastore\InvalidObjectException', + 'class' => '\metastore\NoSuchObjectException', ), 2 => array( 'var' => 'o2', 'type' => TType::STRUCT, - 'class' => '\metastore\AlreadyExistsException', + 'class' => '\metastore\MetaException', ), 3 => array( 'var' => 'o3', 'type' => TType::STRUCT, - 'class' => '\metastore\MetaException', + 'class' => '\metastore\InvalidObjectException', + ), + 4 => array( + 'var' => 'o4', + 'type' => TType::STRUCT, + 'class' => '\metastore\InvalidInputException', ), ); } @@ -20160,11 +23732,14 @@ class ThriftHiveMetastore_add_index_result { if (isset($vals['o3'])) { $this->o3 = $vals['o3']; } + if (isset($vals['o4'])) { + $this->o4 = $vals['o4']; + } } } public function getName() { - return 'ThriftHiveMetastore_add_index_result'; + return 'ThriftHiveMetastore_delete_partition_column_statistics_result'; } public function read($input) @@ -20183,16 +23758,15 @@ class ThriftHiveMetastore_add_index_result { switch ($fid) { case 0: - if ($ftype == TType::STRUCT) { - $this->success = new \metastore\Index(); - $xfer += $this->success->read($input); + if ($ftype == TType::BOOL) { + $xfer += $input->readBool($this->success); } else { $xfer += $input->skip($ftype); } break; case 1: if ($ftype == TType::STRUCT) { - $this->o1 = new \metastore\InvalidObjectException(); + $this->o1 = new \metastore\NoSuchObjectException(); $xfer += $this->o1->read($input); } else { $xfer += $input->skip($ftype); @@ -20200,7 +23774,7 @@ class ThriftHiveMetastore_add_index_result { break; case 2: if ($ftype == TType::STRUCT) { - $this->o2 = new \metastore\AlreadyExistsException(); + $this->o2 = new \metastore\MetaException(); $xfer += $this->o2->read($input); } else { $xfer += $input->skip($ftype); @@ -20208,12 +23782,20 @@ class ThriftHiveMetastore_add_index_result { break; case 3: if ($ftype == TType::STRUCT) { - $this->o3 = new \metastore\MetaException(); + $this->o3 = new \metastore\InvalidObjectException(); $xfer += $this->o3->read($input); } else { $xfer += $input->skip($ftype); } break; + case 4: + if ($ftype == TType::STRUCT) { + $this->o4 = new \metastore\InvalidInputException(); + $xfer += $this->o4->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; default: $xfer += $input->skip($ftype); break; @@ -20225,14 +23807,11 @@ class ThriftHiveMetastore_add_index_result { } public function write($output) { - $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_add_index_result'); - if ($this->success !== null) { - if (!is_object($this->success)) { - throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); - } - $xfer += $output->writeFieldBegin('success', TType::STRUCT, 0); - $xfer += $this->success->write($output); + $xfer = 0; + $xfer += $output->writeStructBegin('ThriftHiveMetastore_delete_partition_column_statistics_result'); + if ($this->success !== null) { + $xfer += $output->writeFieldBegin('success', TType::BOOL, 0); + $xfer += $output->writeBool($this->success); $xfer += $output->writeFieldEnd(); } if ($this->o1 !== null) { @@ -20250,6 +23829,11 @@ class ThriftHiveMetastore_add_index_result { $xfer += $this->o3->write($output); $xfer += $output->writeFieldEnd(); } + if ($this->o4 !== null) { + $xfer += $output->writeFieldBegin('o4', TType::STRUCT, 4); + $xfer += $this->o4->write($output); + $xfer += $output->writeFieldEnd(); + } $xfer += $output->writeFieldStop(); $xfer += $output->writeStructEnd(); return $xfer; @@ -20257,54 +23841,45 @@ class ThriftHiveMetastore_add_index_result { } -class ThriftHiveMetastore_alter_index_args { +class ThriftHiveMetastore_delete_table_column_statistics_args { static $_TSPEC; - public $dbname = null; - public $base_tbl_name = null; - public $idx_name = null; - public $new_idx = null; + public $db_name = null; + public $tbl_name = null; + public $col_name = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 1 => array( - 'var' => 'dbname', + 'var' => 'db_name', 'type' => TType::STRING, ), 2 => array( - 'var' => 'base_tbl_name', + 'var' => 'tbl_name', 'type' => TType::STRING, ), 3 => array( - 'var' => 'idx_name', + 'var' => 'col_name', 'type' => TType::STRING, ), - 4 => array( - 'var' => 'new_idx', - 'type' => TType::STRUCT, - 'class' => '\metastore\Index', - ), ); } if (is_array($vals)) { - if (isset($vals['dbname'])) { - $this->dbname = $vals['dbname']; - } - if (isset($vals['base_tbl_name'])) { - $this->base_tbl_name = $vals['base_tbl_name']; + if (isset($vals['db_name'])) { + $this->db_name = $vals['db_name']; } - if (isset($vals['idx_name'])) { - $this->idx_name = $vals['idx_name']; + if (isset($vals['tbl_name'])) { + $this->tbl_name = $vals['tbl_name']; } - if (isset($vals['new_idx'])) { - $this->new_idx = $vals['new_idx']; + if (isset($vals['col_name'])) { + $this->col_name = $vals['col_name']; } } } public function getName() { - return 'ThriftHiveMetastore_alter_index_args'; + return 'ThriftHiveMetastore_delete_table_column_statistics_args'; } public function read($input) @@ -20324,29 +23899,21 @@ class ThriftHiveMetastore_alter_index_args { { case 1: if ($ftype == TType::STRING) { - $xfer += $input->readString($this->dbname); + $xfer += $input->readString($this->db_name); } else { $xfer += $input->skip($ftype); } break; case 2: if ($ftype == TType::STRING) { - $xfer += $input->readString($this->base_tbl_name); + $xfer += $input->readString($this->tbl_name); } else { $xfer += $input->skip($ftype); } break; case 3: if ($ftype == TType::STRING) { - $xfer += $input->readString($this->idx_name); - } else { - $xfer += $input->skip($ftype); - } - break; - case 4: - if ($ftype == TType::STRUCT) { - $this->new_idx = new \metastore\Index(); - $xfer += $this->new_idx->read($input); + $xfer += $input->readString($this->col_name); } else { $xfer += $input->skip($ftype); } @@ -20363,28 +23930,20 @@ class ThriftHiveMetastore_alter_index_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_alter_index_args'); - if ($this->dbname !== null) { - $xfer += $output->writeFieldBegin('dbname', TType::STRING, 1); - $xfer += $output->writeString($this->dbname); - $xfer += $output->writeFieldEnd(); - } - if ($this->base_tbl_name !== null) { - $xfer += $output->writeFieldBegin('base_tbl_name', TType::STRING, 2); - $xfer += $output->writeString($this->base_tbl_name); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_delete_table_column_statistics_args'); + if ($this->db_name !== null) { + $xfer += $output->writeFieldBegin('db_name', TType::STRING, 1); + $xfer += $output->writeString($this->db_name); $xfer += $output->writeFieldEnd(); } - if ($this->idx_name !== null) { - $xfer += $output->writeFieldBegin('idx_name', TType::STRING, 3); - $xfer += $output->writeString($this->idx_name); + if ($this->tbl_name !== null) { + $xfer += $output->writeFieldBegin('tbl_name', TType::STRING, 2); + $xfer += $output->writeString($this->tbl_name); $xfer += $output->writeFieldEnd(); } - if ($this->new_idx !== null) { - if (!is_object($this->new_idx)) { - throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); - } - $xfer += $output->writeFieldBegin('new_idx', TType::STRUCT, 4); - $xfer += $this->new_idx->write($output); + if ($this->col_name !== null) { + $xfer += $output->writeFieldBegin('col_name', TType::STRING, 3); + $xfer += $output->writeString($this->col_name); $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -20394,39 +23953,65 @@ class ThriftHiveMetastore_alter_index_args { } -class ThriftHiveMetastore_alter_index_result { +class ThriftHiveMetastore_delete_table_column_statistics_result { static $_TSPEC; + public $success = null; public $o1 = null; public $o2 = null; + public $o3 = null; + public $o4 = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( + 0 => array( + 'var' => 'success', + 'type' => TType::BOOL, + ), 1 => array( 'var' => 'o1', 'type' => TType::STRUCT, - 'class' => '\metastore\InvalidOperationException', + 'class' => '\metastore\NoSuchObjectException', ), 2 => array( 'var' => 'o2', 'type' => TType::STRUCT, 'class' => '\metastore\MetaException', ), + 3 => array( + 'var' => 'o3', + 'type' => TType::STRUCT, + 'class' => '\metastore\InvalidObjectException', + ), + 4 => array( + 'var' => 'o4', + 'type' => TType::STRUCT, + 'class' => '\metastore\InvalidInputException', + ), ); } if (is_array($vals)) { + if (isset($vals['success'])) { + $this->success = $vals['success']; + } if (isset($vals['o1'])) { $this->o1 = $vals['o1']; } if (isset($vals['o2'])) { $this->o2 = $vals['o2']; } + if (isset($vals['o3'])) { + $this->o3 = $vals['o3']; + } + if (isset($vals['o4'])) { + $this->o4 = $vals['o4']; + } } } public function getName() { - return 'ThriftHiveMetastore_alter_index_result'; + return 'ThriftHiveMetastore_delete_table_column_statistics_result'; } public function read($input) @@ -20444,9 +24029,16 @@ class ThriftHiveMetastore_alter_index_result { } switch ($fid) { + case 0: + if ($ftype == TType::BOOL) { + $xfer += $input->readBool($this->success); + } else { + $xfer += $input->skip($ftype); + } + break; case 1: if ($ftype == TType::STRUCT) { - $this->o1 = new \metastore\InvalidOperationException(); + $this->o1 = new \metastore\NoSuchObjectException(); $xfer += $this->o1->read($input); } else { $xfer += $input->skip($ftype); @@ -20460,6 +24052,22 @@ class ThriftHiveMetastore_alter_index_result { $xfer += $input->skip($ftype); } break; + case 3: + if ($ftype == TType::STRUCT) { + $this->o3 = new \metastore\InvalidObjectException(); + $xfer += $this->o3->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; + case 4: + if ($ftype == TType::STRUCT) { + $this->o4 = new \metastore\InvalidInputException(); + $xfer += $this->o4->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; default: $xfer += $input->skip($ftype); break; @@ -20472,7 +24080,12 @@ class ThriftHiveMetastore_alter_index_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_alter_index_result'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_delete_table_column_statistics_result'); + if ($this->success !== null) { + $xfer += $output->writeFieldBegin('success', TType::BOOL, 0); + $xfer += $output->writeBool($this->success); + $xfer += $output->writeFieldEnd(); + } if ($this->o1 !== null) { $xfer += $output->writeFieldBegin('o1', TType::STRUCT, 1); $xfer += $this->o1->write($output); @@ -20483,6 +24096,16 @@ class ThriftHiveMetastore_alter_index_result { $xfer += $this->o2->write($output); $xfer += $output->writeFieldEnd(); } + if ($this->o3 !== null) { + $xfer += $output->writeFieldBegin('o3', TType::STRUCT, 3); + $xfer += $this->o3->write($output); + $xfer += $output->writeFieldEnd(); + } + if ($this->o4 !== null) { + $xfer += $output->writeFieldBegin('o4', TType::STRUCT, 4); + $xfer += $this->o4->write($output); + $xfer += $output->writeFieldEnd(); + } $xfer += $output->writeFieldStop(); $xfer += $output->writeStructEnd(); return $xfer; @@ -20490,53 +24113,30 @@ class ThriftHiveMetastore_alter_index_result { } -class ThriftHiveMetastore_drop_index_by_name_args { +class ThriftHiveMetastore_create_function_args { static $_TSPEC; - public $db_name = null; - public $tbl_name = null; - public $index_name = null; - public $deleteData = null; + public $func = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 1 => array( - 'var' => 'db_name', - 'type' => TType::STRING, - ), - 2 => array( - 'var' => 'tbl_name', - 'type' => TType::STRING, - ), - 3 => array( - 'var' => 'index_name', - 'type' => TType::STRING, - ), - 4 => array( - 'var' => 'deleteData', - 'type' => TType::BOOL, + 'var' => 'func', + 'type' => TType::STRUCT, + 'class' => '\metastore\Function', ), ); } if (is_array($vals)) { - if (isset($vals['db_name'])) { - $this->db_name = $vals['db_name']; - } - if (isset($vals['tbl_name'])) { - $this->tbl_name = $vals['tbl_name']; - } - if (isset($vals['index_name'])) { - $this->index_name = $vals['index_name']; - } - if (isset($vals['deleteData'])) { - $this->deleteData = $vals['deleteData']; + if (isset($vals['func'])) { + $this->func = $vals['func']; } } } public function getName() { - return 'ThriftHiveMetastore_drop_index_by_name_args'; + return 'ThriftHiveMetastore_create_function_args'; } public function read($input) @@ -20555,64 +24155,32 @@ class ThriftHiveMetastore_drop_index_by_name_args { switch ($fid) { case 1: - if ($ftype == TType::STRING) { - $xfer += $input->readString($this->db_name); - } else { - $xfer += $input->skip($ftype); - } - break; - case 2: - if ($ftype == TType::STRING) { - $xfer += $input->readString($this->tbl_name); - } else { - $xfer += $input->skip($ftype); - } - break; - case 3: - if ($ftype == TType::STRING) { - $xfer += $input->readString($this->index_name); - } else { - $xfer += $input->skip($ftype); - } - break; - case 4: - if ($ftype == TType::BOOL) { - $xfer += $input->readBool($this->deleteData); + if ($ftype == TType::STRUCT) { + $this->func = new \metastore\Function(); + $xfer += $this->func->read($input); } else { $xfer += $input->skip($ftype); } - break; - default: - $xfer += $input->skip($ftype); - break; - } - $xfer += $input->readFieldEnd(); - } - $xfer += $input->readStructEnd(); - return $xfer; - } - - public function write($output) { - $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_drop_index_by_name_args'); - if ($this->db_name !== null) { - $xfer += $output->writeFieldBegin('db_name', TType::STRING, 1); - $xfer += $output->writeString($this->db_name); - $xfer += $output->writeFieldEnd(); - } - if ($this->tbl_name !== null) { - $xfer += $output->writeFieldBegin('tbl_name', TType::STRING, 2); - $xfer += $output->writeString($this->tbl_name); - $xfer += $output->writeFieldEnd(); - } - if ($this->index_name !== null) { - $xfer += $output->writeFieldBegin('index_name', TType::STRING, 3); - $xfer += $output->writeString($this->index_name); - $xfer += $output->writeFieldEnd(); + break; + default: + $xfer += $input->skip($ftype); + break; + } + $xfer += $input->readFieldEnd(); } - if ($this->deleteData !== null) { - $xfer += $output->writeFieldBegin('deleteData', TType::BOOL, 4); - $xfer += $output->writeBool($this->deleteData); + $xfer += $input->readStructEnd(); + return $xfer; + } + + public function write($output) { + $xfer = 0; + $xfer += $output->writeStructBegin('ThriftHiveMetastore_create_function_args'); + if ($this->func !== null) { + if (!is_object($this->func)) { + throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); + } + $xfer += $output->writeFieldBegin('func', TType::STRUCT, 1); + $xfer += $this->func->write($output); $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -20622,47 +24190,57 @@ class ThriftHiveMetastore_drop_index_by_name_args { } -class ThriftHiveMetastore_drop_index_by_name_result { +class ThriftHiveMetastore_create_function_result { static $_TSPEC; - public $success = null; public $o1 = null; public $o2 = null; + public $o3 = null; + public $o4 = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( - 0 => array( - 'var' => 'success', - 'type' => TType::BOOL, - ), 1 => array( 'var' => 'o1', 'type' => TType::STRUCT, - 'class' => '\metastore\NoSuchObjectException', + 'class' => '\metastore\AlreadyExistsException', ), 2 => array( 'var' => 'o2', 'type' => TType::STRUCT, + 'class' => '\metastore\InvalidObjectException', + ), + 3 => array( + 'var' => 'o3', + 'type' => TType::STRUCT, 'class' => '\metastore\MetaException', ), + 4 => array( + 'var' => 'o4', + 'type' => TType::STRUCT, + 'class' => '\metastore\NoSuchObjectException', + ), ); } if (is_array($vals)) { - if (isset($vals['success'])) { - $this->success = $vals['success']; - } if (isset($vals['o1'])) { $this->o1 = $vals['o1']; } if (isset($vals['o2'])) { $this->o2 = $vals['o2']; } + if (isset($vals['o3'])) { + $this->o3 = $vals['o3']; + } + if (isset($vals['o4'])) { + $this->o4 = $vals['o4']; + } } } public function getName() { - return 'ThriftHiveMetastore_drop_index_by_name_result'; + return 'ThriftHiveMetastore_create_function_result'; } public function read($input) @@ -20680,16 +24258,9 @@ class ThriftHiveMetastore_drop_index_by_name_result { } switch ($fid) { - case 0: - if ($ftype == TType::BOOL) { - $xfer += $input->readBool($this->success); - } else { - $xfer += $input->skip($ftype); - } - break; case 1: if ($ftype == TType::STRUCT) { - $this->o1 = new \metastore\NoSuchObjectException(); + $this->o1 = new \metastore\AlreadyExistsException(); $xfer += $this->o1->read($input); } else { $xfer += $input->skip($ftype); @@ -20697,12 +24268,28 @@ class ThriftHiveMetastore_drop_index_by_name_result { break; case 2: if ($ftype == TType::STRUCT) { - $this->o2 = new \metastore\MetaException(); + $this->o2 = new \metastore\InvalidObjectException(); $xfer += $this->o2->read($input); } else { $xfer += $input->skip($ftype); } break; + case 3: + if ($ftype == TType::STRUCT) { + $this->o3 = new \metastore\MetaException(); + $xfer += $this->o3->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; + case 4: + if ($ftype == TType::STRUCT) { + $this->o4 = new \metastore\NoSuchObjectException(); + $xfer += $this->o4->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; default: $xfer += $input->skip($ftype); break; @@ -20715,12 +24302,7 @@ class ThriftHiveMetastore_drop_index_by_name_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_drop_index_by_name_result'); - if ($this->success !== null) { - $xfer += $output->writeFieldBegin('success', TType::BOOL, 0); - $xfer += $output->writeBool($this->success); - $xfer += $output->writeFieldEnd(); - } + $xfer += $output->writeStructBegin('ThriftHiveMetastore_create_function_result'); if ($this->o1 !== null) { $xfer += $output->writeFieldBegin('o1', TType::STRUCT, 1); $xfer += $this->o1->write($output); @@ -20731,6 +24313,16 @@ class ThriftHiveMetastore_drop_index_by_name_result { $xfer += $this->o2->write($output); $xfer += $output->writeFieldEnd(); } + if ($this->o3 !== null) { + $xfer += $output->writeFieldBegin('o3', TType::STRUCT, 3); + $xfer += $this->o3->write($output); + $xfer += $output->writeFieldEnd(); + } + if ($this->o4 !== null) { + $xfer += $output->writeFieldBegin('o4', TType::STRUCT, 4); + $xfer += $this->o4->write($output); + $xfer += $output->writeFieldEnd(); + } $xfer += $output->writeFieldStop(); $xfer += $output->writeStructEnd(); return $xfer; @@ -20738,45 +24330,37 @@ class ThriftHiveMetastore_drop_index_by_name_result { } -class ThriftHiveMetastore_get_index_by_name_args { +class ThriftHiveMetastore_drop_function_args { static $_TSPEC; - public $db_name = null; - public $tbl_name = null; - public $index_name = null; + public $dbName = null; + public $funcName = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 1 => array( - 'var' => 'db_name', + 'var' => 'dbName', 'type' => TType::STRING, ), 2 => array( - 'var' => 'tbl_name', - 'type' => TType::STRING, - ), - 3 => array( - 'var' => 'index_name', + 'var' => 'funcName', 'type' => TType::STRING, ), ); } if (is_array($vals)) { - if (isset($vals['db_name'])) { - $this->db_name = $vals['db_name']; - } - if (isset($vals['tbl_name'])) { - $this->tbl_name = $vals['tbl_name']; + if (isset($vals['dbName'])) { + $this->dbName = $vals['dbName']; } - if (isset($vals['index_name'])) { - $this->index_name = $vals['index_name']; + if (isset($vals['funcName'])) { + $this->funcName = $vals['funcName']; } } } public function getName() { - return 'ThriftHiveMetastore_get_index_by_name_args'; + return 'ThriftHiveMetastore_drop_function_args'; } public function read($input) @@ -20796,21 +24380,14 @@ class ThriftHiveMetastore_get_index_by_name_args { { case 1: if ($ftype == TType::STRING) { - $xfer += $input->readString($this->db_name); + $xfer += $input->readString($this->dbName); } else { $xfer += $input->skip($ftype); } break; case 2: if ($ftype == TType::STRING) { - $xfer += $input->readString($this->tbl_name); - } else { - $xfer += $input->skip($ftype); - } - break; - case 3: - if ($ftype == TType::STRING) { - $xfer += $input->readString($this->index_name); + $xfer += $input->readString($this->funcName); } else { $xfer += $input->skip($ftype); } @@ -20827,20 +24404,15 @@ class ThriftHiveMetastore_get_index_by_name_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_index_by_name_args'); - if ($this->db_name !== null) { - $xfer += $output->writeFieldBegin('db_name', TType::STRING, 1); - $xfer += $output->writeString($this->db_name); - $xfer += $output->writeFieldEnd(); - } - if ($this->tbl_name !== null) { - $xfer += $output->writeFieldBegin('tbl_name', TType::STRING, 2); - $xfer += $output->writeString($this->tbl_name); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_drop_function_args'); + if ($this->dbName !== null) { + $xfer += $output->writeFieldBegin('dbName', TType::STRING, 1); + $xfer += $output->writeString($this->dbName); $xfer += $output->writeFieldEnd(); } - if ($this->index_name !== null) { - $xfer += $output->writeFieldBegin('index_name', TType::STRING, 3); - $xfer += $output->writeString($this->index_name); + if ($this->funcName !== null) { + $xfer += $output->writeFieldBegin('funcName', TType::STRING, 2); + $xfer += $output->writeString($this->funcName); $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -20850,48 +24422,39 @@ class ThriftHiveMetastore_get_index_by_name_args { } -class ThriftHiveMetastore_get_index_by_name_result { +class ThriftHiveMetastore_drop_function_result { static $_TSPEC; - public $success = null; public $o1 = null; - public $o2 = null; + public $o3 = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( - 0 => array( - 'var' => 'success', - 'type' => TType::STRUCT, - 'class' => '\metastore\Index', - ), 1 => array( 'var' => 'o1', 'type' => TType::STRUCT, - 'class' => '\metastore\MetaException', + 'class' => '\metastore\NoSuchObjectException', ), 2 => array( - 'var' => 'o2', + 'var' => 'o3', 'type' => TType::STRUCT, - 'class' => '\metastore\NoSuchObjectException', + 'class' => '\metastore\MetaException', ), ); } if (is_array($vals)) { - if (isset($vals['success'])) { - $this->success = $vals['success']; - } if (isset($vals['o1'])) { $this->o1 = $vals['o1']; } - if (isset($vals['o2'])) { - $this->o2 = $vals['o2']; + if (isset($vals['o3'])) { + $this->o3 = $vals['o3']; } } } public function getName() { - return 'ThriftHiveMetastore_get_index_by_name_result'; + return 'ThriftHiveMetastore_drop_function_result'; } public function read($input) @@ -20909,17 +24472,9 @@ class ThriftHiveMetastore_get_index_by_name_result { } switch ($fid) { - case 0: - if ($ftype == TType::STRUCT) { - $this->success = new \metastore\Index(); - $xfer += $this->success->read($input); - } else { - $xfer += $input->skip($ftype); - } - break; case 1: if ($ftype == TType::STRUCT) { - $this->o1 = new \metastore\MetaException(); + $this->o1 = new \metastore\NoSuchObjectException(); $xfer += $this->o1->read($input); } else { $xfer += $input->skip($ftype); @@ -20927,8 +24482,8 @@ class ThriftHiveMetastore_get_index_by_name_result { break; case 2: if ($ftype == TType::STRUCT) { - $this->o2 = new \metastore\NoSuchObjectException(); - $xfer += $this->o2->read($input); + $this->o3 = new \metastore\MetaException(); + $xfer += $this->o3->read($input); } else { $xfer += $input->skip($ftype); } @@ -20945,23 +24500,15 @@ class ThriftHiveMetastore_get_index_by_name_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_index_by_name_result'); - if ($this->success !== null) { - if (!is_object($this->success)) { - throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); - } - $xfer += $output->writeFieldBegin('success', TType::STRUCT, 0); - $xfer += $this->success->write($output); - $xfer += $output->writeFieldEnd(); - } + $xfer += $output->writeStructBegin('ThriftHiveMetastore_drop_function_result'); if ($this->o1 !== null) { $xfer += $output->writeFieldBegin('o1', TType::STRUCT, 1); $xfer += $this->o1->write($output); $xfer += $output->writeFieldEnd(); } - if ($this->o2 !== null) { - $xfer += $output->writeFieldBegin('o2', TType::STRUCT, 2); - $xfer += $this->o2->write($output); + if ($this->o3 !== null) { + $xfer += $output->writeFieldBegin('o3', TType::STRUCT, 2); + $xfer += $this->o3->write($output); $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -20971,45 +24518,46 @@ class ThriftHiveMetastore_get_index_by_name_result { } -class ThriftHiveMetastore_get_indexes_args { +class ThriftHiveMetastore_alter_function_args { static $_TSPEC; - public $db_name = null; - public $tbl_name = null; - public $max_indexes = -1; + public $dbName = null; + public $funcName = null; + public $newFunc = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 1 => array( - 'var' => 'db_name', + 'var' => 'dbName', 'type' => TType::STRING, ), 2 => array( - 'var' => 'tbl_name', + 'var' => 'funcName', 'type' => TType::STRING, ), 3 => array( - 'var' => 'max_indexes', - 'type' => TType::I16, + 'var' => 'newFunc', + 'type' => TType::STRUCT, + 'class' => '\metastore\Function', ), ); } if (is_array($vals)) { - if (isset($vals['db_name'])) { - $this->db_name = $vals['db_name']; + if (isset($vals['dbName'])) { + $this->dbName = $vals['dbName']; } - if (isset($vals['tbl_name'])) { - $this->tbl_name = $vals['tbl_name']; + if (isset($vals['funcName'])) { + $this->funcName = $vals['funcName']; } - if (isset($vals['max_indexes'])) { - $this->max_indexes = $vals['max_indexes']; + if (isset($vals['newFunc'])) { + $this->newFunc = $vals['newFunc']; } } } public function getName() { - return 'ThriftHiveMetastore_get_indexes_args'; + return 'ThriftHiveMetastore_alter_function_args'; } public function read($input) @@ -21029,21 +24577,22 @@ class ThriftHiveMetastore_get_indexes_args { { case 1: if ($ftype == TType::STRING) { - $xfer += $input->readString($this->db_name); + $xfer += $input->readString($this->dbName); } else { $xfer += $input->skip($ftype); } break; case 2: if ($ftype == TType::STRING) { - $xfer += $input->readString($this->tbl_name); + $xfer += $input->readString($this->funcName); } else { $xfer += $input->skip($ftype); } break; case 3: - if ($ftype == TType::I16) { - $xfer += $input->readI16($this->max_indexes); + if ($ftype == TType::STRUCT) { + $this->newFunc = new \metastore\Function(); + $xfer += $this->newFunc->read($input); } else { $xfer += $input->skip($ftype); } @@ -21059,21 +24608,24 @@ class ThriftHiveMetastore_get_indexes_args { } public function write($output) { - $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_indexes_args'); - if ($this->db_name !== null) { - $xfer += $output->writeFieldBegin('db_name', TType::STRING, 1); - $xfer += $output->writeString($this->db_name); + $xfer = 0; + $xfer += $output->writeStructBegin('ThriftHiveMetastore_alter_function_args'); + if ($this->dbName !== null) { + $xfer += $output->writeFieldBegin('dbName', TType::STRING, 1); + $xfer += $output->writeString($this->dbName); $xfer += $output->writeFieldEnd(); } - if ($this->tbl_name !== null) { - $xfer += $output->writeFieldBegin('tbl_name', TType::STRING, 2); - $xfer += $output->writeString($this->tbl_name); + if ($this->funcName !== null) { + $xfer += $output->writeFieldBegin('funcName', TType::STRING, 2); + $xfer += $output->writeString($this->funcName); $xfer += $output->writeFieldEnd(); } - if ($this->max_indexes !== null) { - $xfer += $output->writeFieldBegin('max_indexes', TType::I16, 3); - $xfer += $output->writeI16($this->max_indexes); + if ($this->newFunc !== null) { + if (!is_object($this->newFunc)) { + throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); + } + $xfer += $output->writeFieldBegin('newFunc', TType::STRUCT, 3); + $xfer += $this->newFunc->write($output); $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -21083,29 +24635,19 @@ class ThriftHiveMetastore_get_indexes_args { } -class ThriftHiveMetastore_get_indexes_result { +class ThriftHiveMetastore_alter_function_result { static $_TSPEC; - public $success = null; public $o1 = null; public $o2 = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( - 0 => array( - 'var' => 'success', - 'type' => TType::LST, - 'etype' => TType::STRUCT, - 'elem' => array( - 'type' => TType::STRUCT, - 'class' => '\metastore\Index', - ), - ), 1 => array( 'var' => 'o1', 'type' => TType::STRUCT, - 'class' => '\metastore\NoSuchObjectException', + 'class' => '\metastore\InvalidOperationException', ), 2 => array( 'var' => 'o2', @@ -21115,9 +24657,6 @@ class ThriftHiveMetastore_get_indexes_result { ); } if (is_array($vals)) { - if (isset($vals['success'])) { - $this->success = $vals['success']; - } if (isset($vals['o1'])) { $this->o1 = $vals['o1']; } @@ -21128,7 +24667,7 @@ class ThriftHiveMetastore_get_indexes_result { } public function getName() { - return 'ThriftHiveMetastore_get_indexes_result'; + return 'ThriftHiveMetastore_alter_function_result'; } public function read($input) @@ -21146,27 +24685,9 @@ class ThriftHiveMetastore_get_indexes_result { } switch ($fid) { - case 0: - if ($ftype == TType::LST) { - $this->success = array(); - $_size610 = 0; - $_etype613 = 0; - $xfer += $input->readListBegin($_etype613, $_size610); - for ($_i614 = 0; $_i614 < $_size610; ++$_i614) - { - $elem615 = null; - $elem615 = new \metastore\Index(); - $xfer += $elem615->read($input); - $this->success []= $elem615; - } - $xfer += $input->readListEnd(); - } else { - $xfer += $input->skip($ftype); - } - break; case 1: if ($ftype == TType::STRUCT) { - $this->o1 = new \metastore\NoSuchObjectException(); + $this->o1 = new \metastore\InvalidOperationException(); $xfer += $this->o1->read($input); } else { $xfer += $input->skip($ftype); @@ -21192,24 +24713,7 @@ class ThriftHiveMetastore_get_indexes_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_indexes_result'); - if ($this->success !== null) { - if (!is_array($this->success)) { - throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); - } - $xfer += $output->writeFieldBegin('success', TType::LST, 0); - { - $output->writeListBegin(TType::STRUCT, count($this->success)); - { - foreach ($this->success as $iter616) - { - $xfer += $iter616->write($output); - } - } - $output->writeListEnd(); - } - $xfer += $output->writeFieldEnd(); - } + $xfer += $output->writeStructBegin('ThriftHiveMetastore_alter_function_result'); if ($this->o1 !== null) { $xfer += $output->writeFieldBegin('o1', TType::STRUCT, 1); $xfer += $this->o1->write($output); @@ -21227,45 +24731,37 @@ class ThriftHiveMetastore_get_indexes_result { } -class ThriftHiveMetastore_get_index_names_args { +class ThriftHiveMetastore_get_functions_args { static $_TSPEC; - public $db_name = null; - public $tbl_name = null; - public $max_indexes = -1; + public $dbName = null; + public $pattern = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 1 => array( - 'var' => 'db_name', + 'var' => 'dbName', 'type' => TType::STRING, ), 2 => array( - 'var' => 'tbl_name', + 'var' => 'pattern', 'type' => TType::STRING, ), - 3 => array( - 'var' => 'max_indexes', - 'type' => TType::I16, - ), ); } if (is_array($vals)) { - if (isset($vals['db_name'])) { - $this->db_name = $vals['db_name']; - } - if (isset($vals['tbl_name'])) { - $this->tbl_name = $vals['tbl_name']; + if (isset($vals['dbName'])) { + $this->dbName = $vals['dbName']; } - if (isset($vals['max_indexes'])) { - $this->max_indexes = $vals['max_indexes']; + if (isset($vals['pattern'])) { + $this->pattern = $vals['pattern']; } } } public function getName() { - return 'ThriftHiveMetastore_get_index_names_args'; + return 'ThriftHiveMetastore_get_functions_args'; } public function read($input) @@ -21285,21 +24781,14 @@ class ThriftHiveMetastore_get_index_names_args { { case 1: if ($ftype == TType::STRING) { - $xfer += $input->readString($this->db_name); + $xfer += $input->readString($this->dbName); } else { $xfer += $input->skip($ftype); } break; case 2: if ($ftype == TType::STRING) { - $xfer += $input->readString($this->tbl_name); - } else { - $xfer += $input->skip($ftype); - } - break; - case 3: - if ($ftype == TType::I16) { - $xfer += $input->readI16($this->max_indexes); + $xfer += $input->readString($this->pattern); } else { $xfer += $input->skip($ftype); } @@ -21316,20 +24805,15 @@ class ThriftHiveMetastore_get_index_names_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_index_names_args'); - if ($this->db_name !== null) { - $xfer += $output->writeFieldBegin('db_name', TType::STRING, 1); - $xfer += $output->writeString($this->db_name); - $xfer += $output->writeFieldEnd(); - } - if ($this->tbl_name !== null) { - $xfer += $output->writeFieldBegin('tbl_name', TType::STRING, 2); - $xfer += $output->writeString($this->tbl_name); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_functions_args'); + if ($this->dbName !== null) { + $xfer += $output->writeFieldBegin('dbName', TType::STRING, 1); + $xfer += $output->writeString($this->dbName); $xfer += $output->writeFieldEnd(); } - if ($this->max_indexes !== null) { - $xfer += $output->writeFieldBegin('max_indexes', TType::I16, 3); - $xfer += $output->writeI16($this->max_indexes); + if ($this->pattern !== null) { + $xfer += $output->writeFieldBegin('pattern', TType::STRING, 2); + $xfer += $output->writeString($this->pattern); $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -21339,11 +24823,11 @@ class ThriftHiveMetastore_get_index_names_args { } -class ThriftHiveMetastore_get_index_names_result { +class ThriftHiveMetastore_get_functions_result { static $_TSPEC; public $success = null; - public $o2 = null; + public $o1 = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { @@ -21357,7 +24841,7 @@ class ThriftHiveMetastore_get_index_names_result { ), ), 1 => array( - 'var' => 'o2', + 'var' => 'o1', 'type' => TType::STRUCT, 'class' => '\metastore\MetaException', ), @@ -21367,14 +24851,14 @@ class ThriftHiveMetastore_get_index_names_result { if (isset($vals['success'])) { $this->success = $vals['success']; } - if (isset($vals['o2'])) { - $this->o2 = $vals['o2']; + if (isset($vals['o1'])) { + $this->o1 = $vals['o1']; } } } public function getName() { - return 'ThriftHiveMetastore_get_index_names_result'; + return 'ThriftHiveMetastore_get_functions_result'; } public function read($input) @@ -21395,14 +24879,14 @@ class ThriftHiveMetastore_get_index_names_result { case 0: if ($ftype == TType::LST) { $this->success = array(); - $_size617 = 0; - $_etype620 = 0; - $xfer += $input->readListBegin($_etype620, $_size617); - for ($_i621 = 0; $_i621 < $_size617; ++$_i621) + $_size667 = 0; + $_etype670 = 0; + $xfer += $input->readListBegin($_etype670, $_size667); + for ($_i671 = 0; $_i671 < $_size667; ++$_i671) { - $elem622 = null; - $xfer += $input->readString($elem622); - $this->success []= $elem622; + $elem672 = null; + $xfer += $input->readString($elem672); + $this->success []= $elem672; } $xfer += $input->readListEnd(); } else { @@ -21411,8 +24895,8 @@ class ThriftHiveMetastore_get_index_names_result { break; case 1: if ($ftype == TType::STRUCT) { - $this->o2 = new \metastore\MetaException(); - $xfer += $this->o2->read($input); + $this->o1 = new \metastore\MetaException(); + $xfer += $this->o1->read($input); } else { $xfer += $input->skip($ftype); } @@ -21429,7 +24913,7 @@ class ThriftHiveMetastore_get_index_names_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_index_names_result'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_functions_result'); if ($this->success !== null) { if (!is_array($this->success)) { throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); @@ -21438,18 +24922,18 @@ class ThriftHiveMetastore_get_index_names_result { { $output->writeListBegin(TType::STRING, count($this->success)); { - foreach ($this->success as $iter623) + foreach ($this->success as $iter673) { - $xfer += $output->writeString($iter623); + $xfer += $output->writeString($iter673); } } $output->writeListEnd(); } $xfer += $output->writeFieldEnd(); } - if ($this->o2 !== null) { - $xfer += $output->writeFieldBegin('o2', TType::STRUCT, 1); - $xfer += $this->o2->write($output); + if ($this->o1 !== null) { + $xfer += $output->writeFieldBegin('o1', TType::STRUCT, 1); + $xfer += $this->o1->write($output); $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -21459,30 +24943,37 @@ class ThriftHiveMetastore_get_index_names_result { } -class ThriftHiveMetastore_update_table_column_statistics_args { +class ThriftHiveMetastore_get_function_args { static $_TSPEC; - public $stats_obj = null; + public $dbName = null; + public $funcName = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 1 => array( - 'var' => 'stats_obj', - 'type' => TType::STRUCT, - 'class' => '\metastore\ColumnStatistics', + 'var' => 'dbName', + 'type' => TType::STRING, + ), + 2 => array( + 'var' => 'funcName', + 'type' => TType::STRING, ), ); } if (is_array($vals)) { - if (isset($vals['stats_obj'])) { - $this->stats_obj = $vals['stats_obj']; + if (isset($vals['dbName'])) { + $this->dbName = $vals['dbName']; + } + if (isset($vals['funcName'])) { + $this->funcName = $vals['funcName']; } } } public function getName() { - return 'ThriftHiveMetastore_update_table_column_statistics_args'; + return 'ThriftHiveMetastore_get_function_args'; } public function read($input) @@ -21501,9 +24992,15 @@ class ThriftHiveMetastore_update_table_column_statistics_args { switch ($fid) { case 1: - if ($ftype == TType::STRUCT) { - $this->stats_obj = new \metastore\ColumnStatistics(); - $xfer += $this->stats_obj->read($input); + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->dbName); + } else { + $xfer += $input->skip($ftype); + } + break; + case 2: + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->funcName); } else { $xfer += $input->skip($ftype); } @@ -21520,13 +25017,15 @@ class ThriftHiveMetastore_update_table_column_statistics_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_update_table_column_statistics_args'); - if ($this->stats_obj !== null) { - if (!is_object($this->stats_obj)) { - throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); - } - $xfer += $output->writeFieldBegin('stats_obj', TType::STRUCT, 1); - $xfer += $this->stats_obj->write($output); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_function_args'); + if ($this->dbName !== null) { + $xfer += $output->writeFieldBegin('dbName', TType::STRING, 1); + $xfer += $output->writeString($this->dbName); + $xfer += $output->writeFieldEnd(); + } + if ($this->funcName !== null) { + $xfer += $output->writeFieldBegin('funcName', TType::STRING, 2); + $xfer += $output->writeString($this->funcName); $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -21536,41 +25035,30 @@ class ThriftHiveMetastore_update_table_column_statistics_args { } -class ThriftHiveMetastore_update_table_column_statistics_result { +class ThriftHiveMetastore_get_function_result { static $_TSPEC; public $success = null; public $o1 = null; public $o2 = null; - public $o3 = null; - public $o4 = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 0 => array( 'var' => 'success', - 'type' => TType::BOOL, + 'type' => TType::STRUCT, + 'class' => '\metastore\Function', ), 1 => array( 'var' => 'o1', 'type' => TType::STRUCT, - 'class' => '\metastore\NoSuchObjectException', + 'class' => '\metastore\MetaException', ), 2 => array( 'var' => 'o2', 'type' => TType::STRUCT, - 'class' => '\metastore\InvalidObjectException', - ), - 3 => array( - 'var' => 'o3', - 'type' => TType::STRUCT, - 'class' => '\metastore\MetaException', - ), - 4 => array( - 'var' => 'o4', - 'type' => TType::STRUCT, - 'class' => '\metastore\InvalidInputException', + 'class' => '\metastore\NoSuchObjectException', ), ); } @@ -21584,17 +25072,11 @@ class ThriftHiveMetastore_update_table_column_statistics_result { if (isset($vals['o2'])) { $this->o2 = $vals['o2']; } - if (isset($vals['o3'])) { - $this->o3 = $vals['o3']; - } - if (isset($vals['o4'])) { - $this->o4 = $vals['o4']; - } } } public function getName() { - return 'ThriftHiveMetastore_update_table_column_statistics_result'; + return 'ThriftHiveMetastore_get_function_result'; } public function read($input) @@ -21613,15 +25095,16 @@ class ThriftHiveMetastore_update_table_column_statistics_result { switch ($fid) { case 0: - if ($ftype == TType::BOOL) { - $xfer += $input->readBool($this->success); + if ($ftype == TType::STRUCT) { + $this->success = new \metastore\Function(); + $xfer += $this->success->read($input); } else { $xfer += $input->skip($ftype); } break; case 1: if ($ftype == TType::STRUCT) { - $this->o1 = new \metastore\NoSuchObjectException(); + $this->o1 = new \metastore\MetaException(); $xfer += $this->o1->read($input); } else { $xfer += $input->skip($ftype); @@ -21629,24 +25112,8 @@ class ThriftHiveMetastore_update_table_column_statistics_result { break; case 2: if ($ftype == TType::STRUCT) { - $this->o2 = new \metastore\InvalidObjectException(); - $xfer += $this->o2->read($input); - } else { - $xfer += $input->skip($ftype); - } - break; - case 3: - if ($ftype == TType::STRUCT) { - $this->o3 = new \metastore\MetaException(); - $xfer += $this->o3->read($input); - } else { - $xfer += $input->skip($ftype); - } - break; - case 4: - if ($ftype == TType::STRUCT) { - $this->o4 = new \metastore\InvalidInputException(); - $xfer += $this->o4->read($input); + $this->o2 = new \metastore\NoSuchObjectException(); + $xfer += $this->o2->read($input); } else { $xfer += $input->skip($ftype); } @@ -21663,10 +25130,13 @@ class ThriftHiveMetastore_update_table_column_statistics_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_update_table_column_statistics_result'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_function_result'); if ($this->success !== null) { - $xfer += $output->writeFieldBegin('success', TType::BOOL, 0); - $xfer += $output->writeBool($this->success); + if (!is_object($this->success)) { + throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); + } + $xfer += $output->writeFieldBegin('success', TType::STRUCT, 0); + $xfer += $this->success->write($output); $xfer += $output->writeFieldEnd(); } if ($this->o1 !== null) { @@ -21679,16 +25149,6 @@ class ThriftHiveMetastore_update_table_column_statistics_result { $xfer += $this->o2->write($output); $xfer += $output->writeFieldEnd(); } - if ($this->o3 !== null) { - $xfer += $output->writeFieldBegin('o3', TType::STRUCT, 3); - $xfer += $this->o3->write($output); - $xfer += $output->writeFieldEnd(); - } - if ($this->o4 !== null) { - $xfer += $output->writeFieldBegin('o4', TType::STRUCT, 4); - $xfer += $this->o4->write($output); - $xfer += $output->writeFieldEnd(); - } $xfer += $output->writeFieldStop(); $xfer += $output->writeStructEnd(); return $xfer; @@ -21696,30 +25156,30 @@ class ThriftHiveMetastore_update_table_column_statistics_result { } -class ThriftHiveMetastore_update_partition_column_statistics_args { +class ThriftHiveMetastore_create_role_args { static $_TSPEC; - public $stats_obj = null; + public $role = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 1 => array( - 'var' => 'stats_obj', + 'var' => 'role', 'type' => TType::STRUCT, - 'class' => '\metastore\ColumnStatistics', + 'class' => '\metastore\Role', ), ); } if (is_array($vals)) { - if (isset($vals['stats_obj'])) { - $this->stats_obj = $vals['stats_obj']; + if (isset($vals['role'])) { + $this->role = $vals['role']; } } } public function getName() { - return 'ThriftHiveMetastore_update_partition_column_statistics_args'; + return 'ThriftHiveMetastore_create_role_args'; } public function read($input) @@ -21739,8 +25199,8 @@ class ThriftHiveMetastore_update_partition_column_statistics_args { { case 1: if ($ftype == TType::STRUCT) { - $this->stats_obj = new \metastore\ColumnStatistics(); - $xfer += $this->stats_obj->read($input); + $this->role = new \metastore\Role(); + $xfer += $this->role->read($input); } else { $xfer += $input->skip($ftype); } @@ -21757,13 +25217,13 @@ class ThriftHiveMetastore_update_partition_column_statistics_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_update_partition_column_statistics_args'); - if ($this->stats_obj !== null) { - if (!is_object($this->stats_obj)) { + $xfer += $output->writeStructBegin('ThriftHiveMetastore_create_role_args'); + if ($this->role !== null) { + if (!is_object($this->role)) { throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); } - $xfer += $output->writeFieldBegin('stats_obj', TType::STRUCT, 1); - $xfer += $this->stats_obj->write($output); + $xfer += $output->writeFieldBegin('role', TType::STRUCT, 1); + $xfer += $this->role->write($output); $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -21773,14 +25233,11 @@ class ThriftHiveMetastore_update_partition_column_statistics_args { } -class ThriftHiveMetastore_update_partition_column_statistics_result { +class ThriftHiveMetastore_create_role_result { static $_TSPEC; public $success = null; public $o1 = null; - public $o2 = null; - public $o3 = null; - public $o4 = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { @@ -21792,23 +25249,8 @@ class ThriftHiveMetastore_update_partition_column_statistics_result { 1 => array( 'var' => 'o1', 'type' => TType::STRUCT, - 'class' => '\metastore\NoSuchObjectException', - ), - 2 => array( - 'var' => 'o2', - 'type' => TType::STRUCT, - 'class' => '\metastore\InvalidObjectException', - ), - 3 => array( - 'var' => 'o3', - 'type' => TType::STRUCT, 'class' => '\metastore\MetaException', ), - 4 => array( - 'var' => 'o4', - 'type' => TType::STRUCT, - 'class' => '\metastore\InvalidInputException', - ), ); } if (is_array($vals)) { @@ -21818,20 +25260,11 @@ class ThriftHiveMetastore_update_partition_column_statistics_result { if (isset($vals['o1'])) { $this->o1 = $vals['o1']; } - if (isset($vals['o2'])) { - $this->o2 = $vals['o2']; - } - if (isset($vals['o3'])) { - $this->o3 = $vals['o3']; - } - if (isset($vals['o4'])) { - $this->o4 = $vals['o4']; - } } } public function getName() { - return 'ThriftHiveMetastore_update_partition_column_statistics_result'; + return 'ThriftHiveMetastore_create_role_result'; } public function read($input) @@ -21858,36 +25291,12 @@ class ThriftHiveMetastore_update_partition_column_statistics_result { break; case 1: if ($ftype == TType::STRUCT) { - $this->o1 = new \metastore\NoSuchObjectException(); + $this->o1 = new \metastore\MetaException(); $xfer += $this->o1->read($input); } else { $xfer += $input->skip($ftype); } break; - case 2: - if ($ftype == TType::STRUCT) { - $this->o2 = new \metastore\InvalidObjectException(); - $xfer += $this->o2->read($input); - } else { - $xfer += $input->skip($ftype); - } - break; - case 3: - if ($ftype == TType::STRUCT) { - $this->o3 = new \metastore\MetaException(); - $xfer += $this->o3->read($input); - } else { - $xfer += $input->skip($ftype); - } - break; - case 4: - if ($ftype == TType::STRUCT) { - $this->o4 = new \metastore\InvalidInputException(); - $xfer += $this->o4->read($input); - } else { - $xfer += $input->skip($ftype); - } - break; default: $xfer += $input->skip($ftype); break; @@ -21900,7 +25309,7 @@ class ThriftHiveMetastore_update_partition_column_statistics_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_update_partition_column_statistics_result'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_create_role_result'); if ($this->success !== null) { $xfer += $output->writeFieldBegin('success', TType::BOOL, 0); $xfer += $output->writeBool($this->success); @@ -21911,21 +25320,6 @@ class ThriftHiveMetastore_update_partition_column_statistics_result { $xfer += $this->o1->write($output); $xfer += $output->writeFieldEnd(); } - if ($this->o2 !== null) { - $xfer += $output->writeFieldBegin('o2', TType::STRUCT, 2); - $xfer += $this->o2->write($output); - $xfer += $output->writeFieldEnd(); - } - if ($this->o3 !== null) { - $xfer += $output->writeFieldBegin('o3', TType::STRUCT, 3); - $xfer += $this->o3->write($output); - $xfer += $output->writeFieldEnd(); - } - if ($this->o4 !== null) { - $xfer += $output->writeFieldBegin('o4', TType::STRUCT, 4); - $xfer += $this->o4->write($output); - $xfer += $output->writeFieldEnd(); - } $xfer += $output->writeFieldStop(); $xfer += $output->writeStructEnd(); return $xfer; @@ -21933,45 +25327,29 @@ class ThriftHiveMetastore_update_partition_column_statistics_result { } -class ThriftHiveMetastore_get_table_column_statistics_args { +class ThriftHiveMetastore_drop_role_args { static $_TSPEC; - public $db_name = null; - public $tbl_name = null; - public $col_name = null; + public $role_name = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 1 => array( - 'var' => 'db_name', - 'type' => TType::STRING, - ), - 2 => array( - 'var' => 'tbl_name', - 'type' => TType::STRING, - ), - 3 => array( - 'var' => 'col_name', + 'var' => 'role_name', 'type' => TType::STRING, ), ); } if (is_array($vals)) { - if (isset($vals['db_name'])) { - $this->db_name = $vals['db_name']; - } - if (isset($vals['tbl_name'])) { - $this->tbl_name = $vals['tbl_name']; - } - if (isset($vals['col_name'])) { - $this->col_name = $vals['col_name']; + if (isset($vals['role_name'])) { + $this->role_name = $vals['role_name']; } } } public function getName() { - return 'ThriftHiveMetastore_get_table_column_statistics_args'; + return 'ThriftHiveMetastore_drop_role_args'; } public function read($input) @@ -21991,21 +25369,7 @@ class ThriftHiveMetastore_get_table_column_statistics_args { { case 1: if ($ftype == TType::STRING) { - $xfer += $input->readString($this->db_name); - } else { - $xfer += $input->skip($ftype); - } - break; - case 2: - if ($ftype == TType::STRING) { - $xfer += $input->readString($this->tbl_name); - } else { - $xfer += $input->skip($ftype); - } - break; - case 3: - if ($ftype == TType::STRING) { - $xfer += $input->readString($this->col_name); + $xfer += $input->readString($this->role_name); } else { $xfer += $input->skip($ftype); } @@ -22022,20 +25386,10 @@ class ThriftHiveMetastore_get_table_column_statistics_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_table_column_statistics_args'); - if ($this->db_name !== null) { - $xfer += $output->writeFieldBegin('db_name', TType::STRING, 1); - $xfer += $output->writeString($this->db_name); - $xfer += $output->writeFieldEnd(); - } - if ($this->tbl_name !== null) { - $xfer += $output->writeFieldBegin('tbl_name', TType::STRING, 2); - $xfer += $output->writeString($this->tbl_name); - $xfer += $output->writeFieldEnd(); - } - if ($this->col_name !== null) { - $xfer += $output->writeFieldBegin('col_name', TType::STRING, 3); - $xfer += $output->writeString($this->col_name); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_drop_role_args'); + if ($this->role_name !== null) { + $xfer += $output->writeFieldBegin('role_name', TType::STRING, 1); + $xfer += $output->writeString($this->role_name); $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -22045,43 +25399,24 @@ class ThriftHiveMetastore_get_table_column_statistics_args { } -class ThriftHiveMetastore_get_table_column_statistics_result { +class ThriftHiveMetastore_drop_role_result { static $_TSPEC; public $success = null; public $o1 = null; - public $o2 = null; - public $o3 = null; - public $o4 = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 0 => array( 'var' => 'success', - 'type' => TType::STRUCT, - 'class' => '\metastore\ColumnStatistics', + 'type' => TType::BOOL, ), 1 => array( 'var' => 'o1', 'type' => TType::STRUCT, - 'class' => '\metastore\NoSuchObjectException', - ), - 2 => array( - 'var' => 'o2', - 'type' => TType::STRUCT, 'class' => '\metastore\MetaException', ), - 3 => array( - 'var' => 'o3', - 'type' => TType::STRUCT, - 'class' => '\metastore\InvalidInputException', - ), - 4 => array( - 'var' => 'o4', - 'type' => TType::STRUCT, - 'class' => '\metastore\InvalidObjectException', - ), ); } if (is_array($vals)) { @@ -22091,20 +25426,11 @@ class ThriftHiveMetastore_get_table_column_statistics_result { if (isset($vals['o1'])) { $this->o1 = $vals['o1']; } - if (isset($vals['o2'])) { - $this->o2 = $vals['o2']; - } - if (isset($vals['o3'])) { - $this->o3 = $vals['o3']; - } - if (isset($vals['o4'])) { - $this->o4 = $vals['o4']; - } } } public function getName() { - return 'ThriftHiveMetastore_get_table_column_statistics_result'; + return 'ThriftHiveMetastore_drop_role_result'; } public function read($input) @@ -22123,45 +25449,20 @@ class ThriftHiveMetastore_get_table_column_statistics_result { switch ($fid) { case 0: - if ($ftype == TType::STRUCT) { - $this->success = new \metastore\ColumnStatistics(); - $xfer += $this->success->read($input); + if ($ftype == TType::BOOL) { + $xfer += $input->readBool($this->success); } else { $xfer += $input->skip($ftype); } break; case 1: if ($ftype == TType::STRUCT) { - $this->o1 = new \metastore\NoSuchObjectException(); + $this->o1 = new \metastore\MetaException(); $xfer += $this->o1->read($input); } else { $xfer += $input->skip($ftype); } break; - case 2: - if ($ftype == TType::STRUCT) { - $this->o2 = new \metastore\MetaException(); - $xfer += $this->o2->read($input); - } else { - $xfer += $input->skip($ftype); - } - break; - case 3: - if ($ftype == TType::STRUCT) { - $this->o3 = new \metastore\InvalidInputException(); - $xfer += $this->o3->read($input); - } else { - $xfer += $input->skip($ftype); - } - break; - case 4: - if ($ftype == TType::STRUCT) { - $this->o4 = new \metastore\InvalidObjectException(); - $xfer += $this->o4->read($input); - } else { - $xfer += $input->skip($ftype); - } - break; default: $xfer += $input->skip($ftype); break; @@ -22174,13 +25475,10 @@ class ThriftHiveMetastore_get_table_column_statistics_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_table_column_statistics_result'); - if ($this->success !== null) { - if (!is_object($this->success)) { - throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); - } - $xfer += $output->writeFieldBegin('success', TType::STRUCT, 0); - $xfer += $this->success->write($output); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_drop_role_result'); + if ($this->success !== null) { + $xfer += $output->writeFieldBegin('success', TType::BOOL, 0); + $xfer += $output->writeBool($this->success); $xfer += $output->writeFieldEnd(); } if ($this->o1 !== null) { @@ -22188,21 +25486,6 @@ class ThriftHiveMetastore_get_table_column_statistics_result { $xfer += $this->o1->write($output); $xfer += $output->writeFieldEnd(); } - if ($this->o2 !== null) { - $xfer += $output->writeFieldBegin('o2', TType::STRUCT, 2); - $xfer += $this->o2->write($output); - $xfer += $output->writeFieldEnd(); - } - if ($this->o3 !== null) { - $xfer += $output->writeFieldBegin('o3', TType::STRUCT, 3); - $xfer += $this->o3->write($output); - $xfer += $output->writeFieldEnd(); - } - if ($this->o4 !== null) { - $xfer += $output->writeFieldBegin('o4', TType::STRUCT, 4); - $xfer += $this->o4->write($output); - $xfer += $output->writeFieldEnd(); - } $xfer += $output->writeFieldStop(); $xfer += $output->writeStructEnd(); return $xfer; @@ -22210,53 +25493,19 @@ class ThriftHiveMetastore_get_table_column_statistics_result { } -class ThriftHiveMetastore_get_partition_column_statistics_args { +class ThriftHiveMetastore_get_role_names_args { static $_TSPEC; - public $db_name = null; - public $tbl_name = null; - public $part_name = null; - public $col_name = null; - public function __construct($vals=null) { + public function __construct() { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( - 1 => array( - 'var' => 'db_name', - 'type' => TType::STRING, - ), - 2 => array( - 'var' => 'tbl_name', - 'type' => TType::STRING, - ), - 3 => array( - 'var' => 'part_name', - 'type' => TType::STRING, - ), - 4 => array( - 'var' => 'col_name', - 'type' => TType::STRING, - ), ); } - if (is_array($vals)) { - if (isset($vals['db_name'])) { - $this->db_name = $vals['db_name']; - } - if (isset($vals['tbl_name'])) { - $this->tbl_name = $vals['tbl_name']; - } - if (isset($vals['part_name'])) { - $this->part_name = $vals['part_name']; - } - if (isset($vals['col_name'])) { - $this->col_name = $vals['col_name']; - } - } } public function getName() { - return 'ThriftHiveMetastore_get_partition_column_statistics_args'; + return 'ThriftHiveMetastore_get_role_names_args'; } public function read($input) @@ -22274,34 +25523,6 @@ class ThriftHiveMetastore_get_partition_column_statistics_args { } switch ($fid) { - case 1: - if ($ftype == TType::STRING) { - $xfer += $input->readString($this->db_name); - } else { - $xfer += $input->skip($ftype); - } - break; - case 2: - if ($ftype == TType::STRING) { - $xfer += $input->readString($this->tbl_name); - } else { - $xfer += $input->skip($ftype); - } - break; - case 3: - if ($ftype == TType::STRING) { - $xfer += $input->readString($this->part_name); - } else { - $xfer += $input->skip($ftype); - } - break; - case 4: - if ($ftype == TType::STRING) { - $xfer += $input->readString($this->col_name); - } else { - $xfer += $input->skip($ftype); - } - break; default: $xfer += $input->skip($ftype); break; @@ -22314,27 +25535,7 @@ class ThriftHiveMetastore_get_partition_column_statistics_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_partition_column_statistics_args'); - if ($this->db_name !== null) { - $xfer += $output->writeFieldBegin('db_name', TType::STRING, 1); - $xfer += $output->writeString($this->db_name); - $xfer += $output->writeFieldEnd(); - } - if ($this->tbl_name !== null) { - $xfer += $output->writeFieldBegin('tbl_name', TType::STRING, 2); - $xfer += $output->writeString($this->tbl_name); - $xfer += $output->writeFieldEnd(); - } - if ($this->part_name !== null) { - $xfer += $output->writeFieldBegin('part_name', TType::STRING, 3); - $xfer += $output->writeString($this->part_name); - $xfer += $output->writeFieldEnd(); - } - if ($this->col_name !== null) { - $xfer += $output->writeFieldBegin('col_name', TType::STRING, 4); - $xfer += $output->writeString($this->col_name); - $xfer += $output->writeFieldEnd(); - } + $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_role_names_args'); $xfer += $output->writeFieldStop(); $xfer += $output->writeStructEnd(); return $xfer; @@ -22342,43 +25543,28 @@ class ThriftHiveMetastore_get_partition_column_statistics_args { } -class ThriftHiveMetastore_get_partition_column_statistics_result { +class ThriftHiveMetastore_get_role_names_result { static $_TSPEC; public $success = null; public $o1 = null; - public $o2 = null; - public $o3 = null; - public $o4 = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 0 => array( 'var' => 'success', - 'type' => TType::STRUCT, - 'class' => '\metastore\ColumnStatistics', + 'type' => TType::LST, + 'etype' => TType::STRING, + 'elem' => array( + 'type' => TType::STRING, + ), ), 1 => array( 'var' => 'o1', 'type' => TType::STRUCT, - 'class' => '\metastore\NoSuchObjectException', - ), - 2 => array( - 'var' => 'o2', - 'type' => TType::STRUCT, 'class' => '\metastore\MetaException', ), - 3 => array( - 'var' => 'o3', - 'type' => TType::STRUCT, - 'class' => '\metastore\InvalidInputException', - ), - 4 => array( - 'var' => 'o4', - 'type' => TType::STRUCT, - 'class' => '\metastore\InvalidObjectException', - ), ); } if (is_array($vals)) { @@ -22388,20 +25574,11 @@ class ThriftHiveMetastore_get_partition_column_statistics_result { if (isset($vals['o1'])) { $this->o1 = $vals['o1']; } - if (isset($vals['o2'])) { - $this->o2 = $vals['o2']; - } - if (isset($vals['o3'])) { - $this->o3 = $vals['o3']; - } - if (isset($vals['o4'])) { - $this->o4 = $vals['o4']; - } } } public function getName() { - return 'ThriftHiveMetastore_get_partition_column_statistics_result'; + return 'ThriftHiveMetastore_get_role_names_result'; } public function read($input) @@ -22420,45 +25597,30 @@ class ThriftHiveMetastore_get_partition_column_statistics_result { switch ($fid) { case 0: - if ($ftype == TType::STRUCT) { - $this->success = new \metastore\ColumnStatistics(); - $xfer += $this->success->read($input); + if ($ftype == TType::LST) { + $this->success = array(); + $_size674 = 0; + $_etype677 = 0; + $xfer += $input->readListBegin($_etype677, $_size674); + for ($_i678 = 0; $_i678 < $_size674; ++$_i678) + { + $elem679 = null; + $xfer += $input->readString($elem679); + $this->success []= $elem679; + } + $xfer += $input->readListEnd(); } else { $xfer += $input->skip($ftype); } break; case 1: if ($ftype == TType::STRUCT) { - $this->o1 = new \metastore\NoSuchObjectException(); + $this->o1 = new \metastore\MetaException(); $xfer += $this->o1->read($input); } else { $xfer += $input->skip($ftype); } break; - case 2: - if ($ftype == TType::STRUCT) { - $this->o2 = new \metastore\MetaException(); - $xfer += $this->o2->read($input); - } else { - $xfer += $input->skip($ftype); - } - break; - case 3: - if ($ftype == TType::STRUCT) { - $this->o3 = new \metastore\InvalidInputException(); - $xfer += $this->o3->read($input); - } else { - $xfer += $input->skip($ftype); - } - break; - case 4: - if ($ftype == TType::STRUCT) { - $this->o4 = new \metastore\InvalidObjectException(); - $xfer += $this->o4->read($input); - } else { - $xfer += $input->skip($ftype); - } - break; default: $xfer += $input->skip($ftype); break; @@ -22471,13 +25633,22 @@ class ThriftHiveMetastore_get_partition_column_statistics_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_partition_column_statistics_result'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_role_names_result'); if ($this->success !== null) { - if (!is_object($this->success)) { + if (!is_array($this->success)) { throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); } - $xfer += $output->writeFieldBegin('success', TType::STRUCT, 0); - $xfer += $this->success->write($output); + $xfer += $output->writeFieldBegin('success', TType::LST, 0); + { + $output->writeListBegin(TType::STRING, count($this->success)); + { + foreach ($this->success as $iter680) + { + $xfer += $output->writeString($iter680); + } + } + $output->writeListEnd(); + } $xfer += $output->writeFieldEnd(); } if ($this->o1 !== null) { @@ -22485,21 +25656,6 @@ class ThriftHiveMetastore_get_partition_column_statistics_result { $xfer += $this->o1->write($output); $xfer += $output->writeFieldEnd(); } - if ($this->o2 !== null) { - $xfer += $output->writeFieldBegin('o2', TType::STRUCT, 2); - $xfer += $this->o2->write($output); - $xfer += $output->writeFieldEnd(); - } - if ($this->o3 !== null) { - $xfer += $output->writeFieldBegin('o3', TType::STRUCT, 3); - $xfer += $this->o3->write($output); - $xfer += $output->writeFieldEnd(); - } - if ($this->o4 !== null) { - $xfer += $output->writeFieldBegin('o4', TType::STRUCT, 4); - $xfer += $this->o4->write($output); - $xfer += $output->writeFieldEnd(); - } $xfer += $output->writeFieldStop(); $xfer += $output->writeStructEnd(); return $xfer; @@ -22507,30 +25663,69 @@ class ThriftHiveMetastore_get_partition_column_statistics_result { } -class ThriftHiveMetastore_get_table_statistics_req_args { +class ThriftHiveMetastore_grant_role_args { static $_TSPEC; - public $request = null; + public $role_name = null; + public $principal_name = null; + public $principal_type = null; + public $grantor = null; + public $grantorType = null; + public $grant_option = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 1 => array( - 'var' => 'request', - 'type' => TType::STRUCT, - 'class' => '\metastore\TableStatsRequest', + 'var' => 'role_name', + 'type' => TType::STRING, + ), + 2 => array( + 'var' => 'principal_name', + 'type' => TType::STRING, + ), + 3 => array( + 'var' => 'principal_type', + 'type' => TType::I32, + ), + 4 => array( + 'var' => 'grantor', + 'type' => TType::STRING, + ), + 5 => array( + 'var' => 'grantorType', + 'type' => TType::I32, + ), + 6 => array( + 'var' => 'grant_option', + 'type' => TType::BOOL, ), ); } if (is_array($vals)) { - if (isset($vals['request'])) { - $this->request = $vals['request']; + if (isset($vals['role_name'])) { + $this->role_name = $vals['role_name']; + } + if (isset($vals['principal_name'])) { + $this->principal_name = $vals['principal_name']; + } + if (isset($vals['principal_type'])) { + $this->principal_type = $vals['principal_type']; + } + if (isset($vals['grantor'])) { + $this->grantor = $vals['grantor']; + } + if (isset($vals['grantorType'])) { + $this->grantorType = $vals['grantorType']; + } + if (isset($vals['grant_option'])) { + $this->grant_option = $vals['grant_option']; } } } public function getName() { - return 'ThriftHiveMetastore_get_table_statistics_req_args'; + return 'ThriftHiveMetastore_grant_role_args'; } public function read($input) @@ -22549,9 +25744,43 @@ class ThriftHiveMetastore_get_table_statistics_req_args { switch ($fid) { case 1: - if ($ftype == TType::STRUCT) { - $this->request = new \metastore\TableStatsRequest(); - $xfer += $this->request->read($input); + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->role_name); + } else { + $xfer += $input->skip($ftype); + } + break; + case 2: + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->principal_name); + } else { + $xfer += $input->skip($ftype); + } + break; + case 3: + if ($ftype == TType::I32) { + $xfer += $input->readI32($this->principal_type); + } else { + $xfer += $input->skip($ftype); + } + break; + case 4: + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->grantor); + } else { + $xfer += $input->skip($ftype); + } + break; + case 5: + if ($ftype == TType::I32) { + $xfer += $input->readI32($this->grantorType); + } else { + $xfer += $input->skip($ftype); + } + break; + case 6: + if ($ftype == TType::BOOL) { + $xfer += $input->readBool($this->grant_option); } else { $xfer += $input->skip($ftype); } @@ -22568,13 +25797,35 @@ class ThriftHiveMetastore_get_table_statistics_req_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_table_statistics_req_args'); - if ($this->request !== null) { - if (!is_object($this->request)) { - throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); - } - $xfer += $output->writeFieldBegin('request', TType::STRUCT, 1); - $xfer += $this->request->write($output); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_grant_role_args'); + if ($this->role_name !== null) { + $xfer += $output->writeFieldBegin('role_name', TType::STRING, 1); + $xfer += $output->writeString($this->role_name); + $xfer += $output->writeFieldEnd(); + } + if ($this->principal_name !== null) { + $xfer += $output->writeFieldBegin('principal_name', TType::STRING, 2); + $xfer += $output->writeString($this->principal_name); + $xfer += $output->writeFieldEnd(); + } + if ($this->principal_type !== null) { + $xfer += $output->writeFieldBegin('principal_type', TType::I32, 3); + $xfer += $output->writeI32($this->principal_type); + $xfer += $output->writeFieldEnd(); + } + if ($this->grantor !== null) { + $xfer += $output->writeFieldBegin('grantor', TType::STRING, 4); + $xfer += $output->writeString($this->grantor); + $xfer += $output->writeFieldEnd(); + } + if ($this->grantorType !== null) { + $xfer += $output->writeFieldBegin('grantorType', TType::I32, 5); + $xfer += $output->writeI32($this->grantorType); + $xfer += $output->writeFieldEnd(); + } + if ($this->grant_option !== null) { + $xfer += $output->writeFieldBegin('grant_option', TType::BOOL, 6); + $xfer += $output->writeBool($this->grant_option); $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -22584,29 +25835,22 @@ class ThriftHiveMetastore_get_table_statistics_req_args { } -class ThriftHiveMetastore_get_table_statistics_req_result { +class ThriftHiveMetastore_grant_role_result { static $_TSPEC; public $success = null; public $o1 = null; - public $o2 = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 0 => array( 'var' => 'success', - 'type' => TType::STRUCT, - 'class' => '\metastore\TableStatsResult', + 'type' => TType::BOOL, ), 1 => array( 'var' => 'o1', 'type' => TType::STRUCT, - 'class' => '\metastore\NoSuchObjectException', - ), - 2 => array( - 'var' => 'o2', - 'type' => TType::STRUCT, 'class' => '\metastore\MetaException', ), ); @@ -22618,14 +25862,11 @@ class ThriftHiveMetastore_get_table_statistics_req_result { if (isset($vals['o1'])) { $this->o1 = $vals['o1']; } - if (isset($vals['o2'])) { - $this->o2 = $vals['o2']; - } } } public function getName() { - return 'ThriftHiveMetastore_get_table_statistics_req_result'; + return 'ThriftHiveMetastore_grant_role_result'; } public function read($input) @@ -22644,29 +25885,20 @@ class ThriftHiveMetastore_get_table_statistics_req_result { switch ($fid) { case 0: - if ($ftype == TType::STRUCT) { - $this->success = new \metastore\TableStatsResult(); - $xfer += $this->success->read($input); + if ($ftype == TType::BOOL) { + $xfer += $input->readBool($this->success); } else { $xfer += $input->skip($ftype); } break; case 1: if ($ftype == TType::STRUCT) { - $this->o1 = new \metastore\NoSuchObjectException(); + $this->o1 = new \metastore\MetaException(); $xfer += $this->o1->read($input); } else { $xfer += $input->skip($ftype); } break; - case 2: - if ($ftype == TType::STRUCT) { - $this->o2 = new \metastore\MetaException(); - $xfer += $this->o2->read($input); - } else { - $xfer += $input->skip($ftype); - } - break; default: $xfer += $input->skip($ftype); break; @@ -22679,13 +25911,10 @@ class ThriftHiveMetastore_get_table_statistics_req_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_table_statistics_req_result'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_grant_role_result'); if ($this->success !== null) { - if (!is_object($this->success)) { - throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); - } - $xfer += $output->writeFieldBegin('success', TType::STRUCT, 0); - $xfer += $this->success->write($output); + $xfer += $output->writeFieldBegin('success', TType::BOOL, 0); + $xfer += $output->writeBool($this->success); $xfer += $output->writeFieldEnd(); } if ($this->o1 !== null) { @@ -22693,11 +25922,6 @@ class ThriftHiveMetastore_get_table_statistics_req_result { $xfer += $this->o1->write($output); $xfer += $output->writeFieldEnd(); } - if ($this->o2 !== null) { - $xfer += $output->writeFieldBegin('o2', TType::STRUCT, 2); - $xfer += $this->o2->write($output); - $xfer += $output->writeFieldEnd(); - } $xfer += $output->writeFieldStop(); $xfer += $output->writeStructEnd(); return $xfer; @@ -22705,30 +25929,45 @@ class ThriftHiveMetastore_get_table_statistics_req_result { } -class ThriftHiveMetastore_get_partitions_statistics_req_args { +class ThriftHiveMetastore_revoke_role_args { static $_TSPEC; - public $request = null; + public $role_name = null; + public $principal_name = null; + public $principal_type = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 1 => array( - 'var' => 'request', - 'type' => TType::STRUCT, - 'class' => '\metastore\PartitionsStatsRequest', + 'var' => 'role_name', + 'type' => TType::STRING, + ), + 2 => array( + 'var' => 'principal_name', + 'type' => TType::STRING, + ), + 3 => array( + 'var' => 'principal_type', + 'type' => TType::I32, ), ); } if (is_array($vals)) { - if (isset($vals['request'])) { - $this->request = $vals['request']; + if (isset($vals['role_name'])) { + $this->role_name = $vals['role_name']; + } + if (isset($vals['principal_name'])) { + $this->principal_name = $vals['principal_name']; + } + if (isset($vals['principal_type'])) { + $this->principal_type = $vals['principal_type']; } } } public function getName() { - return 'ThriftHiveMetastore_get_partitions_statistics_req_args'; + return 'ThriftHiveMetastore_revoke_role_args'; } public function read($input) @@ -22747,9 +25986,22 @@ class ThriftHiveMetastore_get_partitions_statistics_req_args { switch ($fid) { case 1: - if ($ftype == TType::STRUCT) { - $this->request = new \metastore\PartitionsStatsRequest(); - $xfer += $this->request->read($input); + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->role_name); + } else { + $xfer += $input->skip($ftype); + } + break; + case 2: + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->principal_name); + } else { + $xfer += $input->skip($ftype); + } + break; + case 3: + if ($ftype == TType::I32) { + $xfer += $input->readI32($this->principal_type); } else { $xfer += $input->skip($ftype); } @@ -22766,13 +26018,20 @@ class ThriftHiveMetastore_get_partitions_statistics_req_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_partitions_statistics_req_args'); - if ($this->request !== null) { - if (!is_object($this->request)) { - throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); - } - $xfer += $output->writeFieldBegin('request', TType::STRUCT, 1); - $xfer += $this->request->write($output); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_revoke_role_args'); + if ($this->role_name !== null) { + $xfer += $output->writeFieldBegin('role_name', TType::STRING, 1); + $xfer += $output->writeString($this->role_name); + $xfer += $output->writeFieldEnd(); + } + if ($this->principal_name !== null) { + $xfer += $output->writeFieldBegin('principal_name', TType::STRING, 2); + $xfer += $output->writeString($this->principal_name); + $xfer += $output->writeFieldEnd(); + } + if ($this->principal_type !== null) { + $xfer += $output->writeFieldBegin('principal_type', TType::I32, 3); + $xfer += $output->writeI32($this->principal_type); $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -22782,29 +26041,22 @@ class ThriftHiveMetastore_get_partitions_statistics_req_args { } -class ThriftHiveMetastore_get_partitions_statistics_req_result { +class ThriftHiveMetastore_revoke_role_result { static $_TSPEC; public $success = null; public $o1 = null; - public $o2 = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 0 => array( 'var' => 'success', - 'type' => TType::STRUCT, - 'class' => '\metastore\PartitionsStatsResult', + 'type' => TType::BOOL, ), 1 => array( 'var' => 'o1', 'type' => TType::STRUCT, - 'class' => '\metastore\NoSuchObjectException', - ), - 2 => array( - 'var' => 'o2', - 'type' => TType::STRUCT, 'class' => '\metastore\MetaException', ), ); @@ -22816,14 +26068,11 @@ class ThriftHiveMetastore_get_partitions_statistics_req_result { if (isset($vals['o1'])) { $this->o1 = $vals['o1']; } - if (isset($vals['o2'])) { - $this->o2 = $vals['o2']; - } } } public function getName() { - return 'ThriftHiveMetastore_get_partitions_statistics_req_result'; + return 'ThriftHiveMetastore_revoke_role_result'; } public function read($input) @@ -22842,29 +26091,20 @@ class ThriftHiveMetastore_get_partitions_statistics_req_result { switch ($fid) { case 0: - if ($ftype == TType::STRUCT) { - $this->success = new \metastore\PartitionsStatsResult(); - $xfer += $this->success->read($input); + if ($ftype == TType::BOOL) { + $xfer += $input->readBool($this->success); } else { $xfer += $input->skip($ftype); } break; case 1: if ($ftype == TType::STRUCT) { - $this->o1 = new \metastore\NoSuchObjectException(); + $this->o1 = new \metastore\MetaException(); $xfer += $this->o1->read($input); } else { $xfer += $input->skip($ftype); } break; - case 2: - if ($ftype == TType::STRUCT) { - $this->o2 = new \metastore\MetaException(); - $xfer += $this->o2->read($input); - } else { - $xfer += $input->skip($ftype); - } - break; default: $xfer += $input->skip($ftype); break; @@ -22877,13 +26117,10 @@ class ThriftHiveMetastore_get_partitions_statistics_req_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_partitions_statistics_req_result'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_revoke_role_result'); if ($this->success !== null) { - if (!is_object($this->success)) { - throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); - } - $xfer += $output->writeFieldBegin('success', TType::STRUCT, 0); - $xfer += $this->success->write($output); + $xfer += $output->writeFieldBegin('success', TType::BOOL, 0); + $xfer += $output->writeBool($this->success); $xfer += $output->writeFieldEnd(); } if ($this->o1 !== null) { @@ -22891,11 +26128,6 @@ class ThriftHiveMetastore_get_partitions_statistics_req_result { $xfer += $this->o1->write($output); $xfer += $output->writeFieldEnd(); } - if ($this->o2 !== null) { - $xfer += $output->writeFieldBegin('o2', TType::STRUCT, 2); - $xfer += $this->o2->write($output); - $xfer += $output->writeFieldEnd(); - } $xfer += $output->writeFieldStop(); $xfer += $output->writeStructEnd(); return $xfer; @@ -22903,53 +26135,37 @@ class ThriftHiveMetastore_get_partitions_statistics_req_result { } -class ThriftHiveMetastore_delete_partition_column_statistics_args { +class ThriftHiveMetastore_list_roles_args { static $_TSPEC; - public $db_name = null; - public $tbl_name = null; - public $part_name = null; - public $col_name = null; + public $principal_name = null; + public $principal_type = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 1 => array( - 'var' => 'db_name', + 'var' => 'principal_name', 'type' => TType::STRING, ), 2 => array( - 'var' => 'tbl_name', - 'type' => TType::STRING, - ), - 3 => array( - 'var' => 'part_name', - 'type' => TType::STRING, - ), - 4 => array( - 'var' => 'col_name', - 'type' => TType::STRING, + 'var' => 'principal_type', + 'type' => TType::I32, ), ); } if (is_array($vals)) { - if (isset($vals['db_name'])) { - $this->db_name = $vals['db_name']; - } - if (isset($vals['tbl_name'])) { - $this->tbl_name = $vals['tbl_name']; - } - if (isset($vals['part_name'])) { - $this->part_name = $vals['part_name']; + if (isset($vals['principal_name'])) { + $this->principal_name = $vals['principal_name']; } - if (isset($vals['col_name'])) { - $this->col_name = $vals['col_name']; + if (isset($vals['principal_type'])) { + $this->principal_type = $vals['principal_type']; } } } public function getName() { - return 'ThriftHiveMetastore_delete_partition_column_statistics_args'; + return 'ThriftHiveMetastore_list_roles_args'; } public function read($input) @@ -22969,28 +26185,14 @@ class ThriftHiveMetastore_delete_partition_column_statistics_args { { case 1: if ($ftype == TType::STRING) { - $xfer += $input->readString($this->db_name); + $xfer += $input->readString($this->principal_name); } else { $xfer += $input->skip($ftype); } break; case 2: - if ($ftype == TType::STRING) { - $xfer += $input->readString($this->tbl_name); - } else { - $xfer += $input->skip($ftype); - } - break; - case 3: - if ($ftype == TType::STRING) { - $xfer += $input->readString($this->part_name); - } else { - $xfer += $input->skip($ftype); - } - break; - case 4: - if ($ftype == TType::STRING) { - $xfer += $input->readString($this->col_name); + if ($ftype == TType::I32) { + $xfer += $input->readI32($this->principal_type); } else { $xfer += $input->skip($ftype); } @@ -23007,25 +26209,15 @@ class ThriftHiveMetastore_delete_partition_column_statistics_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_delete_partition_column_statistics_args'); - if ($this->db_name !== null) { - $xfer += $output->writeFieldBegin('db_name', TType::STRING, 1); - $xfer += $output->writeString($this->db_name); - $xfer += $output->writeFieldEnd(); - } - if ($this->tbl_name !== null) { - $xfer += $output->writeFieldBegin('tbl_name', TType::STRING, 2); - $xfer += $output->writeString($this->tbl_name); - $xfer += $output->writeFieldEnd(); - } - if ($this->part_name !== null) { - $xfer += $output->writeFieldBegin('part_name', TType::STRING, 3); - $xfer += $output->writeString($this->part_name); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_list_roles_args'); + if ($this->principal_name !== null) { + $xfer += $output->writeFieldBegin('principal_name', TType::STRING, 1); + $xfer += $output->writeString($this->principal_name); $xfer += $output->writeFieldEnd(); } - if ($this->col_name !== null) { - $xfer += $output->writeFieldBegin('col_name', TType::STRING, 4); - $xfer += $output->writeString($this->col_name); + if ($this->principal_type !== null) { + $xfer += $output->writeFieldBegin('principal_type', TType::I32, 2); + $xfer += $output->writeI32($this->principal_type); $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -23035,42 +26227,29 @@ class ThriftHiveMetastore_delete_partition_column_statistics_args { } -class ThriftHiveMetastore_delete_partition_column_statistics_result { +class ThriftHiveMetastore_list_roles_result { static $_TSPEC; public $success = null; public $o1 = null; - public $o2 = null; - public $o3 = null; - public $o4 = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 0 => array( 'var' => 'success', - 'type' => TType::BOOL, + 'type' => TType::LST, + 'etype' => TType::STRUCT, + 'elem' => array( + 'type' => TType::STRUCT, + 'class' => '\metastore\Role', + ), ), 1 => array( 'var' => 'o1', 'type' => TType::STRUCT, - 'class' => '\metastore\NoSuchObjectException', - ), - 2 => array( - 'var' => 'o2', - 'type' => TType::STRUCT, 'class' => '\metastore\MetaException', ), - 3 => array( - 'var' => 'o3', - 'type' => TType::STRUCT, - 'class' => '\metastore\InvalidObjectException', - ), - 4 => array( - 'var' => 'o4', - 'type' => TType::STRUCT, - 'class' => '\metastore\InvalidInputException', - ), ); } if (is_array($vals)) { @@ -23080,20 +26259,11 @@ class ThriftHiveMetastore_delete_partition_column_statistics_result { if (isset($vals['o1'])) { $this->o1 = $vals['o1']; } - if (isset($vals['o2'])) { - $this->o2 = $vals['o2']; - } - if (isset($vals['o3'])) { - $this->o3 = $vals['o3']; - } - if (isset($vals['o4'])) { - $this->o4 = $vals['o4']; - } } } public function getName() { - return 'ThriftHiveMetastore_delete_partition_column_statistics_result'; + return 'ThriftHiveMetastore_list_roles_result'; } public function read($input) @@ -23112,44 +26282,31 @@ class ThriftHiveMetastore_delete_partition_column_statistics_result { switch ($fid) { case 0: - if ($ftype == TType::BOOL) { - $xfer += $input->readBool($this->success); + if ($ftype == TType::LST) { + $this->success = array(); + $_size681 = 0; + $_etype684 = 0; + $xfer += $input->readListBegin($_etype684, $_size681); + for ($_i685 = 0; $_i685 < $_size681; ++$_i685) + { + $elem686 = null; + $elem686 = new \metastore\Role(); + $xfer += $elem686->read($input); + $this->success []= $elem686; + } + $xfer += $input->readListEnd(); } else { $xfer += $input->skip($ftype); } break; case 1: if ($ftype == TType::STRUCT) { - $this->o1 = new \metastore\NoSuchObjectException(); + $this->o1 = new \metastore\MetaException(); $xfer += $this->o1->read($input); } else { $xfer += $input->skip($ftype); } break; - case 2: - if ($ftype == TType::STRUCT) { - $this->o2 = new \metastore\MetaException(); - $xfer += $this->o2->read($input); - } else { - $xfer += $input->skip($ftype); - } - break; - case 3: - if ($ftype == TType::STRUCT) { - $this->o3 = new \metastore\InvalidObjectException(); - $xfer += $this->o3->read($input); - } else { - $xfer += $input->skip($ftype); - } - break; - case 4: - if ($ftype == TType::STRUCT) { - $this->o4 = new \metastore\InvalidInputException(); - $xfer += $this->o4->read($input); - } else { - $xfer += $input->skip($ftype); - } - break; default: $xfer += $input->skip($ftype); break; @@ -23162,10 +26319,22 @@ class ThriftHiveMetastore_delete_partition_column_statistics_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_delete_partition_column_statistics_result'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_list_roles_result'); if ($this->success !== null) { - $xfer += $output->writeFieldBegin('success', TType::BOOL, 0); - $xfer += $output->writeBool($this->success); + if (!is_array($this->success)) { + throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); + } + $xfer += $output->writeFieldBegin('success', TType::LST, 0); + { + $output->writeListBegin(TType::STRUCT, count($this->success)); + { + foreach ($this->success as $iter687) + { + $xfer += $iter687->write($output); + } + } + $output->writeListEnd(); + } $xfer += $output->writeFieldEnd(); } if ($this->o1 !== null) { @@ -23173,21 +26342,6 @@ class ThriftHiveMetastore_delete_partition_column_statistics_result { $xfer += $this->o1->write($output); $xfer += $output->writeFieldEnd(); } - if ($this->o2 !== null) { - $xfer += $output->writeFieldBegin('o2', TType::STRUCT, 2); - $xfer += $this->o2->write($output); - $xfer += $output->writeFieldEnd(); - } - if ($this->o3 !== null) { - $xfer += $output->writeFieldBegin('o3', TType::STRUCT, 3); - $xfer += $this->o3->write($output); - $xfer += $output->writeFieldEnd(); - } - if ($this->o4 !== null) { - $xfer += $output->writeFieldBegin('o4', TType::STRUCT, 4); - $xfer += $this->o4->write($output); - $xfer += $output->writeFieldEnd(); - } $xfer += $output->writeFieldStop(); $xfer += $output->writeStructEnd(); return $xfer; @@ -23195,45 +26349,50 @@ class ThriftHiveMetastore_delete_partition_column_statistics_result { } -class ThriftHiveMetastore_delete_table_column_statistics_args { +class ThriftHiveMetastore_get_privilege_set_args { static $_TSPEC; - public $db_name = null; - public $tbl_name = null; - public $col_name = null; + public $hiveObject = null; + public $user_name = null; + public $group_names = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 1 => array( - 'var' => 'db_name', - 'type' => TType::STRING, + 'var' => 'hiveObject', + 'type' => TType::STRUCT, + 'class' => '\metastore\HiveObjectRef', ), 2 => array( - 'var' => 'tbl_name', + 'var' => 'user_name', 'type' => TType::STRING, ), 3 => array( - 'var' => 'col_name', - 'type' => TType::STRING, + 'var' => 'group_names', + 'type' => TType::LST, + 'etype' => TType::STRING, + 'elem' => array( + 'type' => TType::STRING, + ), ), ); } if (is_array($vals)) { - if (isset($vals['db_name'])) { - $this->db_name = $vals['db_name']; + if (isset($vals['hiveObject'])) { + $this->hiveObject = $vals['hiveObject']; } - if (isset($vals['tbl_name'])) { - $this->tbl_name = $vals['tbl_name']; + if (isset($vals['user_name'])) { + $this->user_name = $vals['user_name']; } - if (isset($vals['col_name'])) { - $this->col_name = $vals['col_name']; + if (isset($vals['group_names'])) { + $this->group_names = $vals['group_names']; } } } public function getName() { - return 'ThriftHiveMetastore_delete_table_column_statistics_args'; + return 'ThriftHiveMetastore_get_privilege_set_args'; } public function read($input) @@ -23252,22 +26411,33 @@ class ThriftHiveMetastore_delete_table_column_statistics_args { switch ($fid) { case 1: - if ($ftype == TType::STRING) { - $xfer += $input->readString($this->db_name); + if ($ftype == TType::STRUCT) { + $this->hiveObject = new \metastore\HiveObjectRef(); + $xfer += $this->hiveObject->read($input); } else { $xfer += $input->skip($ftype); } break; case 2: if ($ftype == TType::STRING) { - $xfer += $input->readString($this->tbl_name); + $xfer += $input->readString($this->user_name); } else { $xfer += $input->skip($ftype); } break; case 3: - if ($ftype == TType::STRING) { - $xfer += $input->readString($this->col_name); + if ($ftype == TType::LST) { + $this->group_names = array(); + $_size688 = 0; + $_etype691 = 0; + $xfer += $input->readListBegin($_etype691, $_size688); + for ($_i692 = 0; $_i692 < $_size688; ++$_i692) + { + $elem693 = null; + $xfer += $input->readString($elem693); + $this->group_names []= $elem693; + } + $xfer += $input->readListEnd(); } else { $xfer += $input->skip($ftype); } @@ -23284,20 +26454,35 @@ class ThriftHiveMetastore_delete_table_column_statistics_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_delete_table_column_statistics_args'); - if ($this->db_name !== null) { - $xfer += $output->writeFieldBegin('db_name', TType::STRING, 1); - $xfer += $output->writeString($this->db_name); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_privilege_set_args'); + if ($this->hiveObject !== null) { + if (!is_object($this->hiveObject)) { + throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); + } + $xfer += $output->writeFieldBegin('hiveObject', TType::STRUCT, 1); + $xfer += $this->hiveObject->write($output); $xfer += $output->writeFieldEnd(); } - if ($this->tbl_name !== null) { - $xfer += $output->writeFieldBegin('tbl_name', TType::STRING, 2); - $xfer += $output->writeString($this->tbl_name); + if ($this->user_name !== null) { + $xfer += $output->writeFieldBegin('user_name', TType::STRING, 2); + $xfer += $output->writeString($this->user_name); $xfer += $output->writeFieldEnd(); } - if ($this->col_name !== null) { - $xfer += $output->writeFieldBegin('col_name', TType::STRING, 3); - $xfer += $output->writeString($this->col_name); + if ($this->group_names !== null) { + if (!is_array($this->group_names)) { + throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); + } + $xfer += $output->writeFieldBegin('group_names', TType::LST, 3); + { + $output->writeListBegin(TType::STRING, count($this->group_names)); + { + foreach ($this->group_names as $iter694) + { + $xfer += $output->writeString($iter694); + } + } + $output->writeListEnd(); + } $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -23307,42 +26492,25 @@ class ThriftHiveMetastore_delete_table_column_statistics_args { } -class ThriftHiveMetastore_delete_table_column_statistics_result { +class ThriftHiveMetastore_get_privilege_set_result { static $_TSPEC; public $success = null; public $o1 = null; - public $o2 = null; - public $o3 = null; - public $o4 = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 0 => array( 'var' => 'success', - 'type' => TType::BOOL, + 'type' => TType::STRUCT, + 'class' => '\metastore\PrincipalPrivilegeSet', ), 1 => array( 'var' => 'o1', 'type' => TType::STRUCT, - 'class' => '\metastore\NoSuchObjectException', - ), - 2 => array( - 'var' => 'o2', - 'type' => TType::STRUCT, 'class' => '\metastore\MetaException', ), - 3 => array( - 'var' => 'o3', - 'type' => TType::STRUCT, - 'class' => '\metastore\InvalidObjectException', - ), - 4 => array( - 'var' => 'o4', - 'type' => TType::STRUCT, - 'class' => '\metastore\InvalidInputException', - ), ); } if (is_array($vals)) { @@ -23352,20 +26520,11 @@ class ThriftHiveMetastore_delete_table_column_statistics_result { if (isset($vals['o1'])) { $this->o1 = $vals['o1']; } - if (isset($vals['o2'])) { - $this->o2 = $vals['o2']; - } - if (isset($vals['o3'])) { - $this->o3 = $vals['o3']; - } - if (isset($vals['o4'])) { - $this->o4 = $vals['o4']; - } } } public function getName() { - return 'ThriftHiveMetastore_delete_table_column_statistics_result'; + return 'ThriftHiveMetastore_get_privilege_set_result'; } public function read($input) @@ -23384,44 +26543,21 @@ class ThriftHiveMetastore_delete_table_column_statistics_result { switch ($fid) { case 0: - if ($ftype == TType::BOOL) { - $xfer += $input->readBool($this->success); + if ($ftype == TType::STRUCT) { + $this->success = new \metastore\PrincipalPrivilegeSet(); + $xfer += $this->success->read($input); } else { $xfer += $input->skip($ftype); } break; case 1: if ($ftype == TType::STRUCT) { - $this->o1 = new \metastore\NoSuchObjectException(); + $this->o1 = new \metastore\MetaException(); $xfer += $this->o1->read($input); } else { $xfer += $input->skip($ftype); } break; - case 2: - if ($ftype == TType::STRUCT) { - $this->o2 = new \metastore\MetaException(); - $xfer += $this->o2->read($input); - } else { - $xfer += $input->skip($ftype); - } - break; - case 3: - if ($ftype == TType::STRUCT) { - $this->o3 = new \metastore\InvalidObjectException(); - $xfer += $this->o3->read($input); - } else { - $xfer += $input->skip($ftype); - } - break; - case 4: - if ($ftype == TType::STRUCT) { - $this->o4 = new \metastore\InvalidInputException(); - $xfer += $this->o4->read($input); - } else { - $xfer += $input->skip($ftype); - } - break; default: $xfer += $input->skip($ftype); break; @@ -23434,10 +26570,13 @@ class ThriftHiveMetastore_delete_table_column_statistics_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_delete_table_column_statistics_result'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_privilege_set_result'); if ($this->success !== null) { - $xfer += $output->writeFieldBegin('success', TType::BOOL, 0); - $xfer += $output->writeBool($this->success); + if (!is_object($this->success)) { + throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); + } + $xfer += $output->writeFieldBegin('success', TType::STRUCT, 0); + $xfer += $this->success->write($output); $xfer += $output->writeFieldEnd(); } if ($this->o1 !== null) { @@ -23445,21 +26584,6 @@ class ThriftHiveMetastore_delete_table_column_statistics_result { $xfer += $this->o1->write($output); $xfer += $output->writeFieldEnd(); } - if ($this->o2 !== null) { - $xfer += $output->writeFieldBegin('o2', TType::STRUCT, 2); - $xfer += $this->o2->write($output); - $xfer += $output->writeFieldEnd(); - } - if ($this->o3 !== null) { - $xfer += $output->writeFieldBegin('o3', TType::STRUCT, 3); - $xfer += $this->o3->write($output); - $xfer += $output->writeFieldEnd(); - } - if ($this->o4 !== null) { - $xfer += $output->writeFieldBegin('o4', TType::STRUCT, 4); - $xfer += $this->o4->write($output); - $xfer += $output->writeFieldEnd(); - } $xfer += $output->writeFieldStop(); $xfer += $output->writeStructEnd(); return $xfer; @@ -23467,30 +26591,46 @@ class ThriftHiveMetastore_delete_table_column_statistics_result { } -class ThriftHiveMetastore_create_function_args { +class ThriftHiveMetastore_list_privileges_args { static $_TSPEC; - public $func = null; + public $principal_name = null; + public $principal_type = null; + public $hiveObject = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 1 => array( - 'var' => 'func', + 'var' => 'principal_name', + 'type' => TType::STRING, + ), + 2 => array( + 'var' => 'principal_type', + 'type' => TType::I32, + ), + 3 => array( + 'var' => 'hiveObject', 'type' => TType::STRUCT, - 'class' => '\metastore\Function', + 'class' => '\metastore\HiveObjectRef', ), ); } if (is_array($vals)) { - if (isset($vals['func'])) { - $this->func = $vals['func']; + if (isset($vals['principal_name'])) { + $this->principal_name = $vals['principal_name']; + } + if (isset($vals['principal_type'])) { + $this->principal_type = $vals['principal_type']; + } + if (isset($vals['hiveObject'])) { + $this->hiveObject = $vals['hiveObject']; } } } public function getName() { - return 'ThriftHiveMetastore_create_function_args'; + return 'ThriftHiveMetastore_list_privileges_args'; } public function read($input) @@ -23509,9 +26649,23 @@ class ThriftHiveMetastore_create_function_args { switch ($fid) { case 1: + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->principal_name); + } else { + $xfer += $input->skip($ftype); + } + break; + case 2: + if ($ftype == TType::I32) { + $xfer += $input->readI32($this->principal_type); + } else { + $xfer += $input->skip($ftype); + } + break; + case 3: if ($ftype == TType::STRUCT) { - $this->func = new \metastore\Function(); - $xfer += $this->func->read($input); + $this->hiveObject = new \metastore\HiveObjectRef(); + $xfer += $this->hiveObject->read($input); } else { $xfer += $input->skip($ftype); } @@ -23528,13 +26682,23 @@ class ThriftHiveMetastore_create_function_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_create_function_args'); - if ($this->func !== null) { - if (!is_object($this->func)) { + $xfer += $output->writeStructBegin('ThriftHiveMetastore_list_privileges_args'); + if ($this->principal_name !== null) { + $xfer += $output->writeFieldBegin('principal_name', TType::STRING, 1); + $xfer += $output->writeString($this->principal_name); + $xfer += $output->writeFieldEnd(); + } + if ($this->principal_type !== null) { + $xfer += $output->writeFieldBegin('principal_type', TType::I32, 2); + $xfer += $output->writeI32($this->principal_type); + $xfer += $output->writeFieldEnd(); + } + if ($this->hiveObject !== null) { + if (!is_object($this->hiveObject)) { throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); } - $xfer += $output->writeFieldBegin('func', TType::STRUCT, 1); - $xfer += $this->func->write($output); + $xfer += $output->writeFieldBegin('hiveObject', TType::STRUCT, 3); + $xfer += $this->hiveObject->write($output); $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -23544,57 +26708,43 @@ class ThriftHiveMetastore_create_function_args { } -class ThriftHiveMetastore_create_function_result { +class ThriftHiveMetastore_list_privileges_result { static $_TSPEC; + public $success = null; public $o1 = null; - public $o2 = null; - public $o3 = null; - public $o4 = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( + 0 => array( + 'var' => 'success', + 'type' => TType::LST, + 'etype' => TType::STRUCT, + 'elem' => array( + 'type' => TType::STRUCT, + 'class' => '\metastore\HiveObjectPrivilege', + ), + ), 1 => array( 'var' => 'o1', 'type' => TType::STRUCT, - 'class' => '\metastore\AlreadyExistsException', - ), - 2 => array( - 'var' => 'o2', - 'type' => TType::STRUCT, - 'class' => '\metastore\InvalidObjectException', - ), - 3 => array( - 'var' => 'o3', - 'type' => TType::STRUCT, 'class' => '\metastore\MetaException', ), - 4 => array( - 'var' => 'o4', - 'type' => TType::STRUCT, - 'class' => '\metastore\NoSuchObjectException', - ), ); } if (is_array($vals)) { + if (isset($vals['success'])) { + $this->success = $vals['success']; + } if (isset($vals['o1'])) { $this->o1 = $vals['o1']; } - if (isset($vals['o2'])) { - $this->o2 = $vals['o2']; - } - if (isset($vals['o3'])) { - $this->o3 = $vals['o3']; - } - if (isset($vals['o4'])) { - $this->o4 = $vals['o4']; - } } } public function getName() { - return 'ThriftHiveMetastore_create_function_result'; + return 'ThriftHiveMetastore_list_privileges_result'; } public function read($input) @@ -23612,34 +26762,28 @@ class ThriftHiveMetastore_create_function_result { } switch ($fid) { - case 1: - if ($ftype == TType::STRUCT) { - $this->o1 = new \metastore\AlreadyExistsException(); - $xfer += $this->o1->read($input); - } else { - $xfer += $input->skip($ftype); - } - break; - case 2: - if ($ftype == TType::STRUCT) { - $this->o2 = new \metastore\InvalidObjectException(); - $xfer += $this->o2->read($input); - } else { - $xfer += $input->skip($ftype); - } - break; - case 3: - if ($ftype == TType::STRUCT) { - $this->o3 = new \metastore\MetaException(); - $xfer += $this->o3->read($input); + case 0: + if ($ftype == TType::LST) { + $this->success = array(); + $_size695 = 0; + $_etype698 = 0; + $xfer += $input->readListBegin($_etype698, $_size695); + for ($_i699 = 0; $_i699 < $_size695; ++$_i699) + { + $elem700 = null; + $elem700 = new \metastore\HiveObjectPrivilege(); + $xfer += $elem700->read($input); + $this->success []= $elem700; + } + $xfer += $input->readListEnd(); } else { $xfer += $input->skip($ftype); } break; - case 4: + case 1: if ($ftype == TType::STRUCT) { - $this->o4 = new \metastore\NoSuchObjectException(); - $xfer += $this->o4->read($input); + $this->o1 = new \metastore\MetaException(); + $xfer += $this->o1->read($input); } else { $xfer += $input->skip($ftype); } @@ -23648,35 +26792,37 @@ class ThriftHiveMetastore_create_function_result { $xfer += $input->skip($ftype); break; } - $xfer += $input->readFieldEnd(); + $xfer += $input->readFieldEnd(); + } + $xfer += $input->readStructEnd(); + return $xfer; + } + + public function write($output) { + $xfer = 0; + $xfer += $output->writeStructBegin('ThriftHiveMetastore_list_privileges_result'); + if ($this->success !== null) { + if (!is_array($this->success)) { + throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); + } + $xfer += $output->writeFieldBegin('success', TType::LST, 0); + { + $output->writeListBegin(TType::STRUCT, count($this->success)); + { + foreach ($this->success as $iter701) + { + $xfer += $iter701->write($output); + } + } + $output->writeListEnd(); + } + $xfer += $output->writeFieldEnd(); } - $xfer += $input->readStructEnd(); - return $xfer; - } - - public function write($output) { - $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_create_function_result'); if ($this->o1 !== null) { $xfer += $output->writeFieldBegin('o1', TType::STRUCT, 1); $xfer += $this->o1->write($output); $xfer += $output->writeFieldEnd(); } - if ($this->o2 !== null) { - $xfer += $output->writeFieldBegin('o2', TType::STRUCT, 2); - $xfer += $this->o2->write($output); - $xfer += $output->writeFieldEnd(); - } - if ($this->o3 !== null) { - $xfer += $output->writeFieldBegin('o3', TType::STRUCT, 3); - $xfer += $this->o3->write($output); - $xfer += $output->writeFieldEnd(); - } - if ($this->o4 !== null) { - $xfer += $output->writeFieldBegin('o4', TType::STRUCT, 4); - $xfer += $this->o4->write($output); - $xfer += $output->writeFieldEnd(); - } $xfer += $output->writeFieldStop(); $xfer += $output->writeStructEnd(); return $xfer; @@ -23684,37 +26830,30 @@ class ThriftHiveMetastore_create_function_result { } -class ThriftHiveMetastore_drop_function_args { +class ThriftHiveMetastore_grant_privileges_args { static $_TSPEC; - public $dbName = null; - public $funcName = null; + public $privileges = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 1 => array( - 'var' => 'dbName', - 'type' => TType::STRING, - ), - 2 => array( - 'var' => 'funcName', - 'type' => TType::STRING, + 'var' => 'privileges', + 'type' => TType::STRUCT, + 'class' => '\metastore\PrivilegeBag', ), ); } if (is_array($vals)) { - if (isset($vals['dbName'])) { - $this->dbName = $vals['dbName']; - } - if (isset($vals['funcName'])) { - $this->funcName = $vals['funcName']; + if (isset($vals['privileges'])) { + $this->privileges = $vals['privileges']; } } } public function getName() { - return 'ThriftHiveMetastore_drop_function_args'; + return 'ThriftHiveMetastore_grant_privileges_args'; } public function read($input) @@ -23733,15 +26872,9 @@ class ThriftHiveMetastore_drop_function_args { switch ($fid) { case 1: - if ($ftype == TType::STRING) { - $xfer += $input->readString($this->dbName); - } else { - $xfer += $input->skip($ftype); - } - break; - case 2: - if ($ftype == TType::STRING) { - $xfer += $input->readString($this->funcName); + if ($ftype == TType::STRUCT) { + $this->privileges = new \metastore\PrivilegeBag(); + $xfer += $this->privileges->read($input); } else { $xfer += $input->skip($ftype); } @@ -23758,15 +26891,13 @@ class ThriftHiveMetastore_drop_function_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_drop_function_args'); - if ($this->dbName !== null) { - $xfer += $output->writeFieldBegin('dbName', TType::STRING, 1); - $xfer += $output->writeString($this->dbName); - $xfer += $output->writeFieldEnd(); - } - if ($this->funcName !== null) { - $xfer += $output->writeFieldBegin('funcName', TType::STRING, 2); - $xfer += $output->writeString($this->funcName); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_grant_privileges_args'); + if ($this->privileges !== null) { + if (!is_object($this->privileges)) { + throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); + } + $xfer += $output->writeFieldBegin('privileges', TType::STRUCT, 1); + $xfer += $this->privileges->write($output); $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -23776,39 +26907,38 @@ class ThriftHiveMetastore_drop_function_args { } -class ThriftHiveMetastore_drop_function_result { +class ThriftHiveMetastore_grant_privileges_result { static $_TSPEC; + public $success = null; public $o1 = null; - public $o3 = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( + 0 => array( + 'var' => 'success', + 'type' => TType::BOOL, + ), 1 => array( 'var' => 'o1', 'type' => TType::STRUCT, - 'class' => '\metastore\NoSuchObjectException', - ), - 2 => array( - 'var' => 'o3', - 'type' => TType::STRUCT, 'class' => '\metastore\MetaException', ), ); } if (is_array($vals)) { + if (isset($vals['success'])) { + $this->success = $vals['success']; + } if (isset($vals['o1'])) { $this->o1 = $vals['o1']; } - if (isset($vals['o3'])) { - $this->o3 = $vals['o3']; - } } } public function getName() { - return 'ThriftHiveMetastore_drop_function_result'; + return 'ThriftHiveMetastore_grant_privileges_result'; } public function read($input) @@ -23826,18 +26956,17 @@ class ThriftHiveMetastore_drop_function_result { } switch ($fid) { - case 1: - if ($ftype == TType::STRUCT) { - $this->o1 = new \metastore\NoSuchObjectException(); - $xfer += $this->o1->read($input); + case 0: + if ($ftype == TType::BOOL) { + $xfer += $input->readBool($this->success); } else { $xfer += $input->skip($ftype); } break; - case 2: + case 1: if ($ftype == TType::STRUCT) { - $this->o3 = new \metastore\MetaException(); - $xfer += $this->o3->read($input); + $this->o1 = new \metastore\MetaException(); + $xfer += $this->o1->read($input); } else { $xfer += $input->skip($ftype); } @@ -23854,17 +26983,17 @@ class ThriftHiveMetastore_drop_function_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_drop_function_result'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_grant_privileges_result'); + if ($this->success !== null) { + $xfer += $output->writeFieldBegin('success', TType::BOOL, 0); + $xfer += $output->writeBool($this->success); + $xfer += $output->writeFieldEnd(); + } if ($this->o1 !== null) { $xfer += $output->writeFieldBegin('o1', TType::STRUCT, 1); $xfer += $this->o1->write($output); $xfer += $output->writeFieldEnd(); } - if ($this->o3 !== null) { - $xfer += $output->writeFieldBegin('o3', TType::STRUCT, 2); - $xfer += $this->o3->write($output); - $xfer += $output->writeFieldEnd(); - } $xfer += $output->writeFieldStop(); $xfer += $output->writeStructEnd(); return $xfer; @@ -23872,46 +27001,30 @@ class ThriftHiveMetastore_drop_function_result { } -class ThriftHiveMetastore_alter_function_args { +class ThriftHiveMetastore_revoke_privileges_args { static $_TSPEC; - public $dbName = null; - public $funcName = null; - public $newFunc = null; + public $privileges = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 1 => array( - 'var' => 'dbName', - 'type' => TType::STRING, - ), - 2 => array( - 'var' => 'funcName', - 'type' => TType::STRING, - ), - 3 => array( - 'var' => 'newFunc', + 'var' => 'privileges', 'type' => TType::STRUCT, - 'class' => '\metastore\Function', + 'class' => '\metastore\PrivilegeBag', ), ); } if (is_array($vals)) { - if (isset($vals['dbName'])) { - $this->dbName = $vals['dbName']; - } - if (isset($vals['funcName'])) { - $this->funcName = $vals['funcName']; - } - if (isset($vals['newFunc'])) { - $this->newFunc = $vals['newFunc']; + if (isset($vals['privileges'])) { + $this->privileges = $vals['privileges']; } } } public function getName() { - return 'ThriftHiveMetastore_alter_function_args'; + return 'ThriftHiveMetastore_revoke_privileges_args'; } public function read($input) @@ -23930,23 +27043,9 @@ class ThriftHiveMetastore_alter_function_args { switch ($fid) { case 1: - if ($ftype == TType::STRING) { - $xfer += $input->readString($this->dbName); - } else { - $xfer += $input->skip($ftype); - } - break; - case 2: - if ($ftype == TType::STRING) { - $xfer += $input->readString($this->funcName); - } else { - $xfer += $input->skip($ftype); - } - break; - case 3: if ($ftype == TType::STRUCT) { - $this->newFunc = new \metastore\Function(); - $xfer += $this->newFunc->read($input); + $this->privileges = new \metastore\PrivilegeBag(); + $xfer += $this->privileges->read($input); } else { $xfer += $input->skip($ftype); } @@ -23963,23 +27062,13 @@ class ThriftHiveMetastore_alter_function_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_alter_function_args'); - if ($this->dbName !== null) { - $xfer += $output->writeFieldBegin('dbName', TType::STRING, 1); - $xfer += $output->writeString($this->dbName); - $xfer += $output->writeFieldEnd(); - } - if ($this->funcName !== null) { - $xfer += $output->writeFieldBegin('funcName', TType::STRING, 2); - $xfer += $output->writeString($this->funcName); - $xfer += $output->writeFieldEnd(); - } - if ($this->newFunc !== null) { - if (!is_object($this->newFunc)) { + $xfer += $output->writeStructBegin('ThriftHiveMetastore_revoke_privileges_args'); + if ($this->privileges !== null) { + if (!is_object($this->privileges)) { throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); } - $xfer += $output->writeFieldBegin('newFunc', TType::STRUCT, 3); - $xfer += $this->newFunc->write($output); + $xfer += $output->writeFieldBegin('privileges', TType::STRUCT, 1); + $xfer += $this->privileges->write($output); $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -23989,39 +27078,38 @@ class ThriftHiveMetastore_alter_function_args { } -class ThriftHiveMetastore_alter_function_result { +class ThriftHiveMetastore_revoke_privileges_result { static $_TSPEC; + public $success = null; public $o1 = null; - public $o2 = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( + 0 => array( + 'var' => 'success', + 'type' => TType::BOOL, + ), 1 => array( 'var' => 'o1', 'type' => TType::STRUCT, - 'class' => '\metastore\InvalidOperationException', - ), - 2 => array( - 'var' => 'o2', - 'type' => TType::STRUCT, 'class' => '\metastore\MetaException', ), ); } if (is_array($vals)) { + if (isset($vals['success'])) { + $this->success = $vals['success']; + } if (isset($vals['o1'])) { $this->o1 = $vals['o1']; } - if (isset($vals['o2'])) { - $this->o2 = $vals['o2']; - } } } public function getName() { - return 'ThriftHiveMetastore_alter_function_result'; + return 'ThriftHiveMetastore_revoke_privileges_result'; } public function read($input) @@ -24039,18 +27127,17 @@ class ThriftHiveMetastore_alter_function_result { } switch ($fid) { - case 1: - if ($ftype == TType::STRUCT) { - $this->o1 = new \metastore\InvalidOperationException(); - $xfer += $this->o1->read($input); + case 0: + if ($ftype == TType::BOOL) { + $xfer += $input->readBool($this->success); } else { $xfer += $input->skip($ftype); } break; - case 2: + case 1: if ($ftype == TType::STRUCT) { - $this->o2 = new \metastore\MetaException(); - $xfer += $this->o2->read($input); + $this->o1 = new \metastore\MetaException(); + $xfer += $this->o1->read($input); } else { $xfer += $input->skip($ftype); } @@ -24067,17 +27154,17 @@ class ThriftHiveMetastore_alter_function_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_alter_function_result'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_revoke_privileges_result'); + if ($this->success !== null) { + $xfer += $output->writeFieldBegin('success', TType::BOOL, 0); + $xfer += $output->writeBool($this->success); + $xfer += $output->writeFieldEnd(); + } if ($this->o1 !== null) { $xfer += $output->writeFieldBegin('o1', TType::STRUCT, 1); $xfer += $this->o1->write($output); $xfer += $output->writeFieldEnd(); } - if ($this->o2 !== null) { - $xfer += $output->writeFieldBegin('o2', TType::STRUCT, 2); - $xfer += $this->o2->write($output); - $xfer += $output->writeFieldEnd(); - } $xfer += $output->writeFieldStop(); $xfer += $output->writeStructEnd(); return $xfer; @@ -24085,37 +27172,41 @@ class ThriftHiveMetastore_alter_function_result { } -class ThriftHiveMetastore_get_functions_args { +class ThriftHiveMetastore_set_ugi_args { static $_TSPEC; - public $dbName = null; - public $pattern = null; + public $user_name = null; + public $group_names = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 1 => array( - 'var' => 'dbName', + 'var' => 'user_name', 'type' => TType::STRING, ), 2 => array( - 'var' => 'pattern', - 'type' => TType::STRING, + 'var' => 'group_names', + 'type' => TType::LST, + 'etype' => TType::STRING, + 'elem' => array( + 'type' => TType::STRING, + ), ), ); } - if (is_array($vals)) { - if (isset($vals['dbName'])) { - $this->dbName = $vals['dbName']; + if (is_array($vals)) { + if (isset($vals['user_name'])) { + $this->user_name = $vals['user_name']; } - if (isset($vals['pattern'])) { - $this->pattern = $vals['pattern']; + if (isset($vals['group_names'])) { + $this->group_names = $vals['group_names']; } } } public function getName() { - return 'ThriftHiveMetastore_get_functions_args'; + return 'ThriftHiveMetastore_set_ugi_args'; } public function read($input) @@ -24135,14 +27226,24 @@ class ThriftHiveMetastore_get_functions_args { { case 1: if ($ftype == TType::STRING) { - $xfer += $input->readString($this->dbName); + $xfer += $input->readString($this->user_name); } else { $xfer += $input->skip($ftype); } break; case 2: - if ($ftype == TType::STRING) { - $xfer += $input->readString($this->pattern); + if ($ftype == TType::LST) { + $this->group_names = array(); + $_size702 = 0; + $_etype705 = 0; + $xfer += $input->readListBegin($_etype705, $_size702); + for ($_i706 = 0; $_i706 < $_size702; ++$_i706) + { + $elem707 = null; + $xfer += $input->readString($elem707); + $this->group_names []= $elem707; + } + $xfer += $input->readListEnd(); } else { $xfer += $input->skip($ftype); } @@ -24159,15 +27260,27 @@ class ThriftHiveMetastore_get_functions_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_functions_args'); - if ($this->dbName !== null) { - $xfer += $output->writeFieldBegin('dbName', TType::STRING, 1); - $xfer += $output->writeString($this->dbName); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_set_ugi_args'); + if ($this->user_name !== null) { + $xfer += $output->writeFieldBegin('user_name', TType::STRING, 1); + $xfer += $output->writeString($this->user_name); $xfer += $output->writeFieldEnd(); } - if ($this->pattern !== null) { - $xfer += $output->writeFieldBegin('pattern', TType::STRING, 2); - $xfer += $output->writeString($this->pattern); + if ($this->group_names !== null) { + if (!is_array($this->group_names)) { + throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); + } + $xfer += $output->writeFieldBegin('group_names', TType::LST, 2); + { + $output->writeListBegin(TType::STRING, count($this->group_names)); + { + foreach ($this->group_names as $iter708) + { + $xfer += $output->writeString($iter708); + } + } + $output->writeListEnd(); + } $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -24177,7 +27290,7 @@ class ThriftHiveMetastore_get_functions_args { } -class ThriftHiveMetastore_get_functions_result { +class ThriftHiveMetastore_set_ugi_result { static $_TSPEC; public $success = null; @@ -24212,7 +27325,7 @@ class ThriftHiveMetastore_get_functions_result { } public function getName() { - return 'ThriftHiveMetastore_get_functions_result'; + return 'ThriftHiveMetastore_set_ugi_result'; } public function read($input) @@ -24233,14 +27346,14 @@ class ThriftHiveMetastore_get_functions_result { case 0: if ($ftype == TType::LST) { $this->success = array(); - $_size624 = 0; - $_etype627 = 0; - $xfer += $input->readListBegin($_etype627, $_size624); - for ($_i628 = 0; $_i628 < $_size624; ++$_i628) + $_size709 = 0; + $_etype712 = 0; + $xfer += $input->readListBegin($_etype712, $_size709); + for ($_i713 = 0; $_i713 < $_size709; ++$_i713) { - $elem629 = null; - $xfer += $input->readString($elem629); - $this->success []= $elem629; + $elem714 = null; + $xfer += $input->readString($elem714); + $this->success []= $elem714; } $xfer += $input->readListEnd(); } else { @@ -24267,7 +27380,7 @@ class ThriftHiveMetastore_get_functions_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_functions_result'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_set_ugi_result'); if ($this->success !== null) { if (!is_array($this->success)) { throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); @@ -24276,9 +27389,9 @@ class ThriftHiveMetastore_get_functions_result { { $output->writeListBegin(TType::STRING, count($this->success)); { - foreach ($this->success as $iter630) + foreach ($this->success as $iter715) { - $xfer += $output->writeString($iter630); + $xfer += $output->writeString($iter715); } } $output->writeListEnd(); @@ -24297,37 +27410,37 @@ class ThriftHiveMetastore_get_functions_result { } -class ThriftHiveMetastore_get_function_args { +class ThriftHiveMetastore_get_delegation_token_args { static $_TSPEC; - public $dbName = null; - public $funcName = null; + public $token_owner = null; + public $renewer_kerberos_principal_name = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 1 => array( - 'var' => 'dbName', + 'var' => 'token_owner', 'type' => TType::STRING, ), 2 => array( - 'var' => 'funcName', + 'var' => 'renewer_kerberos_principal_name', 'type' => TType::STRING, ), ); } if (is_array($vals)) { - if (isset($vals['dbName'])) { - $this->dbName = $vals['dbName']; + if (isset($vals['token_owner'])) { + $this->token_owner = $vals['token_owner']; } - if (isset($vals['funcName'])) { - $this->funcName = $vals['funcName']; + if (isset($vals['renewer_kerberos_principal_name'])) { + $this->renewer_kerberos_principal_name = $vals['renewer_kerberos_principal_name']; } } } public function getName() { - return 'ThriftHiveMetastore_get_function_args'; + return 'ThriftHiveMetastore_get_delegation_token_args'; } public function read($input) @@ -24347,14 +27460,14 @@ class ThriftHiveMetastore_get_function_args { { case 1: if ($ftype == TType::STRING) { - $xfer += $input->readString($this->dbName); + $xfer += $input->readString($this->token_owner); } else { $xfer += $input->skip($ftype); } break; case 2: if ($ftype == TType::STRING) { - $xfer += $input->readString($this->funcName); + $xfer += $input->readString($this->renewer_kerberos_principal_name); } else { $xfer += $input->skip($ftype); } @@ -24371,15 +27484,15 @@ class ThriftHiveMetastore_get_function_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_function_args'); - if ($this->dbName !== null) { - $xfer += $output->writeFieldBegin('dbName', TType::STRING, 1); - $xfer += $output->writeString($this->dbName); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_delegation_token_args'); + if ($this->token_owner !== null) { + $xfer += $output->writeFieldBegin('token_owner', TType::STRING, 1); + $xfer += $output->writeString($this->token_owner); $xfer += $output->writeFieldEnd(); } - if ($this->funcName !== null) { - $xfer += $output->writeFieldBegin('funcName', TType::STRING, 2); - $xfer += $output->writeString($this->funcName); + if ($this->renewer_kerberos_principal_name !== null) { + $xfer += $output->writeFieldBegin('renewer_kerberos_principal_name', TType::STRING, 2); + $xfer += $output->writeString($this->renewer_kerberos_principal_name); $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -24389,31 +27502,24 @@ class ThriftHiveMetastore_get_function_args { } -class ThriftHiveMetastore_get_function_result { +class ThriftHiveMetastore_get_delegation_token_result { static $_TSPEC; public $success = null; public $o1 = null; - public $o2 = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 0 => array( 'var' => 'success', - 'type' => TType::STRUCT, - 'class' => '\metastore\Function', + 'type' => TType::STRING, ), 1 => array( 'var' => 'o1', 'type' => TType::STRUCT, 'class' => '\metastore\MetaException', ), - 2 => array( - 'var' => 'o2', - 'type' => TType::STRUCT, - 'class' => '\metastore\NoSuchObjectException', - ), ); } if (is_array($vals)) { @@ -24423,14 +27529,11 @@ class ThriftHiveMetastore_get_function_result { if (isset($vals['o1'])) { $this->o1 = $vals['o1']; } - if (isset($vals['o2'])) { - $this->o2 = $vals['o2']; - } } } public function getName() { - return 'ThriftHiveMetastore_get_function_result'; + return 'ThriftHiveMetastore_get_delegation_token_result'; } public function read($input) @@ -24449,9 +27552,8 @@ class ThriftHiveMetastore_get_function_result { switch ($fid) { case 0: - if ($ftype == TType::STRUCT) { - $this->success = new \metastore\Function(); - $xfer += $this->success->read($input); + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->success); } else { $xfer += $input->skip($ftype); } @@ -24464,14 +27566,6 @@ class ThriftHiveMetastore_get_function_result { $xfer += $input->skip($ftype); } break; - case 2: - if ($ftype == TType::STRUCT) { - $this->o2 = new \metastore\NoSuchObjectException(); - $xfer += $this->o2->read($input); - } else { - $xfer += $input->skip($ftype); - } - break; default: $xfer += $input->skip($ftype); break; @@ -24484,13 +27578,10 @@ class ThriftHiveMetastore_get_function_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_function_result'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_delegation_token_result'); if ($this->success !== null) { - if (!is_object($this->success)) { - throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); - } - $xfer += $output->writeFieldBegin('success', TType::STRUCT, 0); - $xfer += $this->success->write($output); + $xfer += $output->writeFieldBegin('success', TType::STRING, 0); + $xfer += $output->writeString($this->success); $xfer += $output->writeFieldEnd(); } if ($this->o1 !== null) { @@ -24498,11 +27589,6 @@ class ThriftHiveMetastore_get_function_result { $xfer += $this->o1->write($output); $xfer += $output->writeFieldEnd(); } - if ($this->o2 !== null) { - $xfer += $output->writeFieldBegin('o2', TType::STRUCT, 2); - $xfer += $this->o2->write($output); - $xfer += $output->writeFieldEnd(); - } $xfer += $output->writeFieldStop(); $xfer += $output->writeStructEnd(); return $xfer; @@ -24510,30 +27596,29 @@ class ThriftHiveMetastore_get_function_result { } -class ThriftHiveMetastore_create_role_args { +class ThriftHiveMetastore_renew_delegation_token_args { static $_TSPEC; - public $role = null; + public $token_str_form = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 1 => array( - 'var' => 'role', - 'type' => TType::STRUCT, - 'class' => '\metastore\Role', + 'var' => 'token_str_form', + 'type' => TType::STRING, ), ); } if (is_array($vals)) { - if (isset($vals['role'])) { - $this->role = $vals['role']; + if (isset($vals['token_str_form'])) { + $this->token_str_form = $vals['token_str_form']; } } } public function getName() { - return 'ThriftHiveMetastore_create_role_args'; + return 'ThriftHiveMetastore_renew_delegation_token_args'; } public function read($input) @@ -24552,9 +27637,8 @@ class ThriftHiveMetastore_create_role_args { switch ($fid) { case 1: - if ($ftype == TType::STRUCT) { - $this->role = new \metastore\Role(); - $xfer += $this->role->read($input); + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->token_str_form); } else { $xfer += $input->skip($ftype); } @@ -24571,13 +27655,10 @@ class ThriftHiveMetastore_create_role_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_create_role_args'); - if ($this->role !== null) { - if (!is_object($this->role)) { - throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); - } - $xfer += $output->writeFieldBegin('role', TType::STRUCT, 1); - $xfer += $this->role->write($output); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_renew_delegation_token_args'); + if ($this->token_str_form !== null) { + $xfer += $output->writeFieldBegin('token_str_form', TType::STRING, 1); + $xfer += $output->writeString($this->token_str_form); $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -24587,7 +27668,7 @@ class ThriftHiveMetastore_create_role_args { } -class ThriftHiveMetastore_create_role_result { +class ThriftHiveMetastore_renew_delegation_token_result { static $_TSPEC; public $success = null; @@ -24598,7 +27679,7 @@ class ThriftHiveMetastore_create_role_result { self::$_TSPEC = array( 0 => array( 'var' => 'success', - 'type' => TType::BOOL, + 'type' => TType::I64, ), 1 => array( 'var' => 'o1', @@ -24618,7 +27699,7 @@ class ThriftHiveMetastore_create_role_result { } public function getName() { - return 'ThriftHiveMetastore_create_role_result'; + return 'ThriftHiveMetastore_renew_delegation_token_result'; } public function read($input) @@ -24637,8 +27718,8 @@ class ThriftHiveMetastore_create_role_result { switch ($fid) { case 0: - if ($ftype == TType::BOOL) { - $xfer += $input->readBool($this->success); + if ($ftype == TType::I64) { + $xfer += $input->readI64($this->success); } else { $xfer += $input->skip($ftype); } @@ -24663,10 +27744,10 @@ class ThriftHiveMetastore_create_role_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_create_role_result'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_renew_delegation_token_result'); if ($this->success !== null) { - $xfer += $output->writeFieldBegin('success', TType::BOOL, 0); - $xfer += $output->writeBool($this->success); + $xfer += $output->writeFieldBegin('success', TType::I64, 0); + $xfer += $output->writeI64($this->success); $xfer += $output->writeFieldEnd(); } if ($this->o1 !== null) { @@ -24681,29 +27762,29 @@ class ThriftHiveMetastore_create_role_result { } -class ThriftHiveMetastore_drop_role_args { +class ThriftHiveMetastore_cancel_delegation_token_args { static $_TSPEC; - public $role_name = null; + public $token_str_form = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 1 => array( - 'var' => 'role_name', + 'var' => 'token_str_form', 'type' => TType::STRING, ), ); } if (is_array($vals)) { - if (isset($vals['role_name'])) { - $this->role_name = $vals['role_name']; + if (isset($vals['token_str_form'])) { + $this->token_str_form = $vals['token_str_form']; } } } public function getName() { - return 'ThriftHiveMetastore_drop_role_args'; + return 'ThriftHiveMetastore_cancel_delegation_token_args'; } public function read($input) @@ -24723,7 +27804,7 @@ class ThriftHiveMetastore_drop_role_args { { case 1: if ($ftype == TType::STRING) { - $xfer += $input->readString($this->role_name); + $xfer += $input->readString($this->token_str_form); } else { $xfer += $input->skip($ftype); } @@ -24740,10 +27821,10 @@ class ThriftHiveMetastore_drop_role_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_drop_role_args'); - if ($this->role_name !== null) { - $xfer += $output->writeFieldBegin('role_name', TType::STRING, 1); - $xfer += $output->writeString($this->role_name); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_cancel_delegation_token_args'); + if ($this->token_str_form !== null) { + $xfer += $output->writeFieldBegin('token_str_form', TType::STRING, 1); + $xfer += $output->writeString($this->token_str_form); $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -24753,19 +27834,14 @@ class ThriftHiveMetastore_drop_role_args { } -class ThriftHiveMetastore_drop_role_result { +class ThriftHiveMetastore_cancel_delegation_token_result { static $_TSPEC; - public $success = null; public $o1 = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( - 0 => array( - 'var' => 'success', - 'type' => TType::BOOL, - ), 1 => array( 'var' => 'o1', 'type' => TType::STRUCT, @@ -24774,9 +27850,6 @@ class ThriftHiveMetastore_drop_role_result { ); } if (is_array($vals)) { - if (isset($vals['success'])) { - $this->success = $vals['success']; - } if (isset($vals['o1'])) { $this->o1 = $vals['o1']; } @@ -24784,7 +27857,7 @@ class ThriftHiveMetastore_drop_role_result { } public function getName() { - return 'ThriftHiveMetastore_drop_role_result'; + return 'ThriftHiveMetastore_cancel_delegation_token_result'; } public function read($input) @@ -24802,13 +27875,6 @@ class ThriftHiveMetastore_drop_role_result { } switch ($fid) { - case 0: - if ($ftype == TType::BOOL) { - $xfer += $input->readBool($this->success); - } else { - $xfer += $input->skip($ftype); - } - break; case 1: if ($ftype == TType::STRUCT) { $this->o1 = new \metastore\MetaException(); @@ -24829,12 +27895,7 @@ class ThriftHiveMetastore_drop_role_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_drop_role_result'); - if ($this->success !== null) { - $xfer += $output->writeFieldBegin('success', TType::BOOL, 0); - $xfer += $output->writeBool($this->success); - $xfer += $output->writeFieldEnd(); - } + $xfer += $output->writeStructBegin('ThriftHiveMetastore_cancel_delegation_token_result'); if ($this->o1 !== null) { $xfer += $output->writeFieldBegin('o1', TType::STRUCT, 1); $xfer += $this->o1->write($output); @@ -24847,7 +27908,7 @@ class ThriftHiveMetastore_drop_role_result { } -class ThriftHiveMetastore_get_role_names_args { +class ThriftHiveMetastore_get_open_txns_args { static $_TSPEC; @@ -24859,7 +27920,7 @@ class ThriftHiveMetastore_get_role_names_args { } public function getName() { - return 'ThriftHiveMetastore_get_role_names_args'; + return 'ThriftHiveMetastore_get_open_txns_args'; } public function read($input) @@ -24889,7 +27950,7 @@ class ThriftHiveMetastore_get_role_names_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_role_names_args'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_open_txns_args'); $xfer += $output->writeFieldStop(); $xfer += $output->writeStructEnd(); return $xfer; @@ -24897,27 +27958,18 @@ class ThriftHiveMetastore_get_role_names_args { } -class ThriftHiveMetastore_get_role_names_result { +class ThriftHiveMetastore_get_open_txns_result { static $_TSPEC; public $success = null; - public $o1 = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 0 => array( 'var' => 'success', - 'type' => TType::LST, - 'etype' => TType::STRING, - 'elem' => array( - 'type' => TType::STRING, - ), - ), - 1 => array( - 'var' => 'o1', 'type' => TType::STRUCT, - 'class' => '\metastore\MetaException', + 'class' => '\metastore\GetOpenTxnsResponse', ), ); } @@ -24925,14 +27977,11 @@ class ThriftHiveMetastore_get_role_names_result { if (isset($vals['success'])) { $this->success = $vals['success']; } - if (isset($vals['o1'])) { - $this->o1 = $vals['o1']; - } } } public function getName() { - return 'ThriftHiveMetastore_get_role_names_result'; + return 'ThriftHiveMetastore_get_open_txns_result'; } public function read($input) @@ -24951,26 +28000,9 @@ class ThriftHiveMetastore_get_role_names_result { switch ($fid) { case 0: - if ($ftype == TType::LST) { - $this->success = array(); - $_size631 = 0; - $_etype634 = 0; - $xfer += $input->readListBegin($_etype634, $_size631); - for ($_i635 = 0; $_i635 < $_size631; ++$_i635) - { - $elem636 = null; - $xfer += $input->readString($elem636); - $this->success []= $elem636; - } - $xfer += $input->readListEnd(); - } else { - $xfer += $input->skip($ftype); - } - break; - case 1: if ($ftype == TType::STRUCT) { - $this->o1 = new \metastore\MetaException(); - $xfer += $this->o1->read($input); + $this->success = new \metastore\GetOpenTxnsResponse(); + $xfer += $this->success->read($input); } else { $xfer += $input->skip($ftype); } @@ -24987,27 +28019,13 @@ class ThriftHiveMetastore_get_role_names_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_role_names_result'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_open_txns_result'); if ($this->success !== null) { - if (!is_array($this->success)) { + if (!is_object($this->success)) { throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); } - $xfer += $output->writeFieldBegin('success', TType::LST, 0); - { - $output->writeListBegin(TType::STRING, count($this->success)); - { - foreach ($this->success as $iter637) - { - $xfer += $output->writeString($iter637); - } - } - $output->writeListEnd(); - } - $xfer += $output->writeFieldEnd(); - } - if ($this->o1 !== null) { - $xfer += $output->writeFieldBegin('o1', TType::STRUCT, 1); - $xfer += $this->o1->write($output); + $xfer += $output->writeFieldBegin('success', TType::STRUCT, 0); + $xfer += $this->success->write($output); $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -25017,69 +28035,19 @@ class ThriftHiveMetastore_get_role_names_result { } -class ThriftHiveMetastore_grant_role_args { +class ThriftHiveMetastore_get_open_txns_info_args { static $_TSPEC; - public $role_name = null; - public $principal_name = null; - public $principal_type = null; - public $grantor = null; - public $grantorType = null; - public $grant_option = null; - public function __construct($vals=null) { + public function __construct() { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( - 1 => array( - 'var' => 'role_name', - 'type' => TType::STRING, - ), - 2 => array( - 'var' => 'principal_name', - 'type' => TType::STRING, - ), - 3 => array( - 'var' => 'principal_type', - 'type' => TType::I32, - ), - 4 => array( - 'var' => 'grantor', - 'type' => TType::STRING, - ), - 5 => array( - 'var' => 'grantorType', - 'type' => TType::I32, - ), - 6 => array( - 'var' => 'grant_option', - 'type' => TType::BOOL, - ), ); } - if (is_array($vals)) { - if (isset($vals['role_name'])) { - $this->role_name = $vals['role_name']; - } - if (isset($vals['principal_name'])) { - $this->principal_name = $vals['principal_name']; - } - if (isset($vals['principal_type'])) { - $this->principal_type = $vals['principal_type']; - } - if (isset($vals['grantor'])) { - $this->grantor = $vals['grantor']; - } - if (isset($vals['grantorType'])) { - $this->grantorType = $vals['grantorType']; - } - if (isset($vals['grant_option'])) { - $this->grant_option = $vals['grant_option']; - } - } } public function getName() { - return 'ThriftHiveMetastore_grant_role_args'; + return 'ThriftHiveMetastore_get_open_txns_info_args'; } public function read($input) @@ -25097,48 +28065,6 @@ class ThriftHiveMetastore_grant_role_args { } switch ($fid) { - case 1: - if ($ftype == TType::STRING) { - $xfer += $input->readString($this->role_name); - } else { - $xfer += $input->skip($ftype); - } - break; - case 2: - if ($ftype == TType::STRING) { - $xfer += $input->readString($this->principal_name); - } else { - $xfer += $input->skip($ftype); - } - break; - case 3: - if ($ftype == TType::I32) { - $xfer += $input->readI32($this->principal_type); - } else { - $xfer += $input->skip($ftype); - } - break; - case 4: - if ($ftype == TType::STRING) { - $xfer += $input->readString($this->grantor); - } else { - $xfer += $input->skip($ftype); - } - break; - case 5: - if ($ftype == TType::I32) { - $xfer += $input->readI32($this->grantorType); - } else { - $xfer += $input->skip($ftype); - } - break; - case 6: - if ($ftype == TType::BOOL) { - $xfer += $input->readBool($this->grant_option); - } else { - $xfer += $input->skip($ftype); - } - break; default: $xfer += $input->skip($ftype); break; @@ -25151,37 +28077,7 @@ class ThriftHiveMetastore_grant_role_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_grant_role_args'); - if ($this->role_name !== null) { - $xfer += $output->writeFieldBegin('role_name', TType::STRING, 1); - $xfer += $output->writeString($this->role_name); - $xfer += $output->writeFieldEnd(); - } - if ($this->principal_name !== null) { - $xfer += $output->writeFieldBegin('principal_name', TType::STRING, 2); - $xfer += $output->writeString($this->principal_name); - $xfer += $output->writeFieldEnd(); - } - if ($this->principal_type !== null) { - $xfer += $output->writeFieldBegin('principal_type', TType::I32, 3); - $xfer += $output->writeI32($this->principal_type); - $xfer += $output->writeFieldEnd(); - } - if ($this->grantor !== null) { - $xfer += $output->writeFieldBegin('grantor', TType::STRING, 4); - $xfer += $output->writeString($this->grantor); - $xfer += $output->writeFieldEnd(); - } - if ($this->grantorType !== null) { - $xfer += $output->writeFieldBegin('grantorType', TType::I32, 5); - $xfer += $output->writeI32($this->grantorType); - $xfer += $output->writeFieldEnd(); - } - if ($this->grant_option !== null) { - $xfer += $output->writeFieldBegin('grant_option', TType::BOOL, 6); - $xfer += $output->writeBool($this->grant_option); - $xfer += $output->writeFieldEnd(); - } + $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_open_txns_info_args'); $xfer += $output->writeFieldStop(); $xfer += $output->writeStructEnd(); return $xfer; @@ -25189,23 +28085,18 @@ class ThriftHiveMetastore_grant_role_args { } -class ThriftHiveMetastore_grant_role_result { +class ThriftHiveMetastore_get_open_txns_info_result { static $_TSPEC; public $success = null; - public $o1 = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 0 => array( 'var' => 'success', - 'type' => TType::BOOL, - ), - 1 => array( - 'var' => 'o1', 'type' => TType::STRUCT, - 'class' => '\metastore\MetaException', + 'class' => '\metastore\GetOpenTxnsInfoResponse', ), ); } @@ -25213,14 +28104,11 @@ class ThriftHiveMetastore_grant_role_result { if (isset($vals['success'])) { $this->success = $vals['success']; } - if (isset($vals['o1'])) { - $this->o1 = $vals['o1']; - } } } public function getName() { - return 'ThriftHiveMetastore_grant_role_result'; + return 'ThriftHiveMetastore_get_open_txns_info_result'; } public function read($input) @@ -25239,16 +28127,9 @@ class ThriftHiveMetastore_grant_role_result { switch ($fid) { case 0: - if ($ftype == TType::BOOL) { - $xfer += $input->readBool($this->success); - } else { - $xfer += $input->skip($ftype); - } - break; - case 1: if ($ftype == TType::STRUCT) { - $this->o1 = new \metastore\MetaException(); - $xfer += $this->o1->read($input); + $this->success = new \metastore\GetOpenTxnsInfoResponse(); + $xfer += $this->success->read($input); } else { $xfer += $input->skip($ftype); } @@ -25265,15 +28146,13 @@ class ThriftHiveMetastore_grant_role_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_grant_role_result'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_open_txns_info_result'); if ($this->success !== null) { - $xfer += $output->writeFieldBegin('success', TType::BOOL, 0); - $xfer += $output->writeBool($this->success); - $xfer += $output->writeFieldEnd(); - } - if ($this->o1 !== null) { - $xfer += $output->writeFieldBegin('o1', TType::STRUCT, 1); - $xfer += $this->o1->write($output); + if (!is_object($this->success)) { + throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); + } + $xfer += $output->writeFieldBegin('success', TType::STRUCT, 0); + $xfer += $this->success->write($output); $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -25283,45 +28162,30 @@ class ThriftHiveMetastore_grant_role_result { } -class ThriftHiveMetastore_revoke_role_args { +class ThriftHiveMetastore_open_txns_args { static $_TSPEC; - public $role_name = null; - public $principal_name = null; - public $principal_type = null; + public $rqst = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 1 => array( - 'var' => 'role_name', - 'type' => TType::STRING, - ), - 2 => array( - 'var' => 'principal_name', - 'type' => TType::STRING, - ), - 3 => array( - 'var' => 'principal_type', - 'type' => TType::I32, + 'var' => 'rqst', + 'type' => TType::STRUCT, + 'class' => '\metastore\OpenTxnRequest', ), ); } if (is_array($vals)) { - if (isset($vals['role_name'])) { - $this->role_name = $vals['role_name']; - } - if (isset($vals['principal_name'])) { - $this->principal_name = $vals['principal_name']; - } - if (isset($vals['principal_type'])) { - $this->principal_type = $vals['principal_type']; + if (isset($vals['rqst'])) { + $this->rqst = $vals['rqst']; } } } public function getName() { - return 'ThriftHiveMetastore_revoke_role_args'; + return 'ThriftHiveMetastore_open_txns_args'; } public function read($input) @@ -25340,22 +28204,9 @@ class ThriftHiveMetastore_revoke_role_args { switch ($fid) { case 1: - if ($ftype == TType::STRING) { - $xfer += $input->readString($this->role_name); - } else { - $xfer += $input->skip($ftype); - } - break; - case 2: - if ($ftype == TType::STRING) { - $xfer += $input->readString($this->principal_name); - } else { - $xfer += $input->skip($ftype); - } - break; - case 3: - if ($ftype == TType::I32) { - $xfer += $input->readI32($this->principal_type); + if ($ftype == TType::STRUCT) { + $this->rqst = new \metastore\OpenTxnRequest(); + $xfer += $this->rqst->read($input); } else { $xfer += $input->skip($ftype); } @@ -25372,20 +28223,13 @@ class ThriftHiveMetastore_revoke_role_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_revoke_role_args'); - if ($this->role_name !== null) { - $xfer += $output->writeFieldBegin('role_name', TType::STRING, 1); - $xfer += $output->writeString($this->role_name); - $xfer += $output->writeFieldEnd(); - } - if ($this->principal_name !== null) { - $xfer += $output->writeFieldBegin('principal_name', TType::STRING, 2); - $xfer += $output->writeString($this->principal_name); - $xfer += $output->writeFieldEnd(); - } - if ($this->principal_type !== null) { - $xfer += $output->writeFieldBegin('principal_type', TType::I32, 3); - $xfer += $output->writeI32($this->principal_type); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_open_txns_args'); + if ($this->rqst !== null) { + if (!is_object($this->rqst)) { + throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); + } + $xfer += $output->writeFieldBegin('rqst', TType::STRUCT, 1); + $xfer += $this->rqst->write($output); $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -25395,23 +28239,18 @@ class ThriftHiveMetastore_revoke_role_args { } -class ThriftHiveMetastore_revoke_role_result { +class ThriftHiveMetastore_open_txns_result { static $_TSPEC; public $success = null; - public $o1 = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 0 => array( 'var' => 'success', - 'type' => TType::BOOL, - ), - 1 => array( - 'var' => 'o1', 'type' => TType::STRUCT, - 'class' => '\metastore\MetaException', + 'class' => '\metastore\OpenTxnsResponse', ), ); } @@ -25419,14 +28258,11 @@ class ThriftHiveMetastore_revoke_role_result { if (isset($vals['success'])) { $this->success = $vals['success']; } - if (isset($vals['o1'])) { - $this->o1 = $vals['o1']; - } } } public function getName() { - return 'ThriftHiveMetastore_revoke_role_result'; + return 'ThriftHiveMetastore_open_txns_result'; } public function read($input) @@ -25445,16 +28281,9 @@ class ThriftHiveMetastore_revoke_role_result { switch ($fid) { case 0: - if ($ftype == TType::BOOL) { - $xfer += $input->readBool($this->success); - } else { - $xfer += $input->skip($ftype); - } - break; - case 1: if ($ftype == TType::STRUCT) { - $this->o1 = new \metastore\MetaException(); - $xfer += $this->o1->read($input); + $this->success = new \metastore\OpenTxnsResponse(); + $xfer += $this->success->read($input); } else { $xfer += $input->skip($ftype); } @@ -25471,15 +28300,13 @@ class ThriftHiveMetastore_revoke_role_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_revoke_role_result'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_open_txns_result'); if ($this->success !== null) { - $xfer += $output->writeFieldBegin('success', TType::BOOL, 0); - $xfer += $output->writeBool($this->success); - $xfer += $output->writeFieldEnd(); - } - if ($this->o1 !== null) { - $xfer += $output->writeFieldBegin('o1', TType::STRUCT, 1); - $xfer += $this->o1->write($output); + if (!is_object($this->success)) { + throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); + } + $xfer += $output->writeFieldBegin('success', TType::STRUCT, 0); + $xfer += $this->success->write($output); $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -25489,37 +28316,30 @@ class ThriftHiveMetastore_revoke_role_result { } -class ThriftHiveMetastore_list_roles_args { +class ThriftHiveMetastore_abort_txn_args { static $_TSPEC; - public $principal_name = null; - public $principal_type = null; + public $rqst = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 1 => array( - 'var' => 'principal_name', - 'type' => TType::STRING, - ), - 2 => array( - 'var' => 'principal_type', - 'type' => TType::I32, + 'var' => 'rqst', + 'type' => TType::STRUCT, + 'class' => '\metastore\AbortTxnRequest', ), ); } if (is_array($vals)) { - if (isset($vals['principal_name'])) { - $this->principal_name = $vals['principal_name']; - } - if (isset($vals['principal_type'])) { - $this->principal_type = $vals['principal_type']; + if (isset($vals['rqst'])) { + $this->rqst = $vals['rqst']; } } } public function getName() { - return 'ThriftHiveMetastore_list_roles_args'; + return 'ThriftHiveMetastore_abort_txn_args'; } public function read($input) @@ -25538,15 +28358,9 @@ class ThriftHiveMetastore_list_roles_args { switch ($fid) { case 1: - if ($ftype == TType::STRING) { - $xfer += $input->readString($this->principal_name); - } else { - $xfer += $input->skip($ftype); - } - break; - case 2: - if ($ftype == TType::I32) { - $xfer += $input->readI32($this->principal_type); + if ($ftype == TType::STRUCT) { + $this->rqst = new \metastore\AbortTxnRequest(); + $xfer += $this->rqst->read($input); } else { $xfer += $input->skip($ftype); } @@ -25563,15 +28377,13 @@ class ThriftHiveMetastore_list_roles_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_list_roles_args'); - if ($this->principal_name !== null) { - $xfer += $output->writeFieldBegin('principal_name', TType::STRING, 1); - $xfer += $output->writeString($this->principal_name); - $xfer += $output->writeFieldEnd(); - } - if ($this->principal_type !== null) { - $xfer += $output->writeFieldBegin('principal_type', TType::I32, 2); - $xfer += $output->writeI32($this->principal_type); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_abort_txn_args'); + if ($this->rqst !== null) { + if (!is_object($this->rqst)) { + throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); + } + $xfer += $output->writeFieldBegin('rqst', TType::STRUCT, 1); + $xfer += $this->rqst->write($output); $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -25581,35 +28393,22 @@ class ThriftHiveMetastore_list_roles_args { } -class ThriftHiveMetastore_list_roles_result { +class ThriftHiveMetastore_abort_txn_result { static $_TSPEC; - public $success = null; public $o1 = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( - 0 => array( - 'var' => 'success', - 'type' => TType::LST, - 'etype' => TType::STRUCT, - 'elem' => array( - 'type' => TType::STRUCT, - 'class' => '\metastore\Role', - ), - ), 1 => array( 'var' => 'o1', 'type' => TType::STRUCT, - 'class' => '\metastore\MetaException', + 'class' => '\metastore\NoSuchTxnException', ), ); } if (is_array($vals)) { - if (isset($vals['success'])) { - $this->success = $vals['success']; - } if (isset($vals['o1'])) { $this->o1 = $vals['o1']; } @@ -25617,7 +28416,7 @@ class ThriftHiveMetastore_list_roles_result { } public function getName() { - return 'ThriftHiveMetastore_list_roles_result'; + return 'ThriftHiveMetastore_abort_txn_result'; } public function read($input) @@ -25635,27 +28434,9 @@ class ThriftHiveMetastore_list_roles_result { } switch ($fid) { - case 0: - if ($ftype == TType::LST) { - $this->success = array(); - $_size638 = 0; - $_etype641 = 0; - $xfer += $input->readListBegin($_etype641, $_size638); - for ($_i642 = 0; $_i642 < $_size638; ++$_i642) - { - $elem643 = null; - $elem643 = new \metastore\Role(); - $xfer += $elem643->read($input); - $this->success []= $elem643; - } - $xfer += $input->readListEnd(); - } else { - $xfer += $input->skip($ftype); - } - break; case 1: if ($ftype == TType::STRUCT) { - $this->o1 = new \metastore\MetaException(); + $this->o1 = new \metastore\NoSuchTxnException(); $xfer += $this->o1->read($input); } else { $xfer += $input->skip($ftype); @@ -25673,24 +28454,7 @@ class ThriftHiveMetastore_list_roles_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_list_roles_result'); - if ($this->success !== null) { - if (!is_array($this->success)) { - throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); - } - $xfer += $output->writeFieldBegin('success', TType::LST, 0); - { - $output->writeListBegin(TType::STRUCT, count($this->success)); - { - foreach ($this->success as $iter644) - { - $xfer += $iter644->write($output); - } - } - $output->writeListEnd(); - } - $xfer += $output->writeFieldEnd(); - } + $xfer += $output->writeStructBegin('ThriftHiveMetastore_abort_txn_result'); if ($this->o1 !== null) { $xfer += $output->writeFieldBegin('o1', TType::STRUCT, 1); $xfer += $this->o1->write($output); @@ -25703,50 +28467,30 @@ class ThriftHiveMetastore_list_roles_result { } -class ThriftHiveMetastore_get_privilege_set_args { +class ThriftHiveMetastore_commit_txn_args { static $_TSPEC; - public $hiveObject = null; - public $user_name = null; - public $group_names = null; + public $rqst = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 1 => array( - 'var' => 'hiveObject', + 'var' => 'rqst', 'type' => TType::STRUCT, - 'class' => '\metastore\HiveObjectRef', - ), - 2 => array( - 'var' => 'user_name', - 'type' => TType::STRING, - ), - 3 => array( - 'var' => 'group_names', - 'type' => TType::LST, - 'etype' => TType::STRING, - 'elem' => array( - 'type' => TType::STRING, - ), + 'class' => '\metastore\CommitTxnRequest', ), ); } if (is_array($vals)) { - if (isset($vals['hiveObject'])) { - $this->hiveObject = $vals['hiveObject']; - } - if (isset($vals['user_name'])) { - $this->user_name = $vals['user_name']; - } - if (isset($vals['group_names'])) { - $this->group_names = $vals['group_names']; + if (isset($vals['rqst'])) { + $this->rqst = $vals['rqst']; } } } public function getName() { - return 'ThriftHiveMetastore_get_privilege_set_args'; + return 'ThriftHiveMetastore_commit_txn_args'; } public function read($input) @@ -25766,32 +28510,8 @@ class ThriftHiveMetastore_get_privilege_set_args { { case 1: if ($ftype == TType::STRUCT) { - $this->hiveObject = new \metastore\HiveObjectRef(); - $xfer += $this->hiveObject->read($input); - } else { - $xfer += $input->skip($ftype); - } - break; - case 2: - if ($ftype == TType::STRING) { - $xfer += $input->readString($this->user_name); - } else { - $xfer += $input->skip($ftype); - } - break; - case 3: - if ($ftype == TType::LST) { - $this->group_names = array(); - $_size645 = 0; - $_etype648 = 0; - $xfer += $input->readListBegin($_etype648, $_size645); - for ($_i649 = 0; $_i649 < $_size645; ++$_i649) - { - $elem650 = null; - $xfer += $input->readString($elem650); - $this->group_names []= $elem650; - } - $xfer += $input->readListEnd(); + $this->rqst = new \metastore\CommitTxnRequest(); + $xfer += $this->rqst->read($input); } else { $xfer += $input->skip($ftype); } @@ -25808,35 +28528,13 @@ class ThriftHiveMetastore_get_privilege_set_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_privilege_set_args'); - if ($this->hiveObject !== null) { - if (!is_object($this->hiveObject)) { - throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); - } - $xfer += $output->writeFieldBegin('hiveObject', TType::STRUCT, 1); - $xfer += $this->hiveObject->write($output); - $xfer += $output->writeFieldEnd(); - } - if ($this->user_name !== null) { - $xfer += $output->writeFieldBegin('user_name', TType::STRING, 2); - $xfer += $output->writeString($this->user_name); - $xfer += $output->writeFieldEnd(); - } - if ($this->group_names !== null) { - if (!is_array($this->group_names)) { + $xfer += $output->writeStructBegin('ThriftHiveMetastore_commit_txn_args'); + if ($this->rqst !== null) { + if (!is_object($this->rqst)) { throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); } - $xfer += $output->writeFieldBegin('group_names', TType::LST, 3); - { - $output->writeListBegin(TType::STRING, count($this->group_names)); - { - foreach ($this->group_names as $iter651) - { - $xfer += $output->writeString($iter651); - } - } - $output->writeListEnd(); - } + $xfer += $output->writeFieldBegin('rqst', TType::STRUCT, 1); + $xfer += $this->rqst->write($output); $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -25846,39 +28544,39 @@ class ThriftHiveMetastore_get_privilege_set_args { } -class ThriftHiveMetastore_get_privilege_set_result { +class ThriftHiveMetastore_commit_txn_result { static $_TSPEC; - public $success = null; public $o1 = null; + public $o2 = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( - 0 => array( - 'var' => 'success', - 'type' => TType::STRUCT, - 'class' => '\metastore\PrincipalPrivilegeSet', - ), 1 => array( 'var' => 'o1', 'type' => TType::STRUCT, - 'class' => '\metastore\MetaException', + 'class' => '\metastore\NoSuchTxnException', + ), + 2 => array( + 'var' => 'o2', + 'type' => TType::STRUCT, + 'class' => '\metastore\TxnAbortedException', ), ); } if (is_array($vals)) { - if (isset($vals['success'])) { - $this->success = $vals['success']; - } if (isset($vals['o1'])) { $this->o1 = $vals['o1']; } + if (isset($vals['o2'])) { + $this->o2 = $vals['o2']; + } } } public function getName() { - return 'ThriftHiveMetastore_get_privilege_set_result'; + return 'ThriftHiveMetastore_commit_txn_result'; } public function read($input) @@ -25896,18 +28594,18 @@ class ThriftHiveMetastore_get_privilege_set_result { } switch ($fid) { - case 0: + case 1: if ($ftype == TType::STRUCT) { - $this->success = new \metastore\PrincipalPrivilegeSet(); - $xfer += $this->success->read($input); + $this->o1 = new \metastore\NoSuchTxnException(); + $xfer += $this->o1->read($input); } else { $xfer += $input->skip($ftype); } break; - case 1: + case 2: if ($ftype == TType::STRUCT) { - $this->o1 = new \metastore\MetaException(); - $xfer += $this->o1->read($input); + $this->o2 = new \metastore\TxnAbortedException(); + $xfer += $this->o2->read($input); } else { $xfer += $input->skip($ftype); } @@ -25924,20 +28622,17 @@ class ThriftHiveMetastore_get_privilege_set_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_privilege_set_result'); - if ($this->success !== null) { - if (!is_object($this->success)) { - throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); - } - $xfer += $output->writeFieldBegin('success', TType::STRUCT, 0); - $xfer += $this->success->write($output); - $xfer += $output->writeFieldEnd(); - } + $xfer += $output->writeStructBegin('ThriftHiveMetastore_commit_txn_result'); if ($this->o1 !== null) { $xfer += $output->writeFieldBegin('o1', TType::STRUCT, 1); $xfer += $this->o1->write($output); $xfer += $output->writeFieldEnd(); } + if ($this->o2 !== null) { + $xfer += $output->writeFieldBegin('o2', TType::STRUCT, 2); + $xfer += $this->o2->write($output); + $xfer += $output->writeFieldEnd(); + } $xfer += $output->writeFieldStop(); $xfer += $output->writeStructEnd(); return $xfer; @@ -25945,46 +28640,30 @@ class ThriftHiveMetastore_get_privilege_set_result { } -class ThriftHiveMetastore_list_privileges_args { +class ThriftHiveMetastore_lock_args { static $_TSPEC; - public $principal_name = null; - public $principal_type = null; - public $hiveObject = null; + public $rqst = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 1 => array( - 'var' => 'principal_name', - 'type' => TType::STRING, - ), - 2 => array( - 'var' => 'principal_type', - 'type' => TType::I32, - ), - 3 => array( - 'var' => 'hiveObject', + 'var' => 'rqst', 'type' => TType::STRUCT, - 'class' => '\metastore\HiveObjectRef', + 'class' => '\metastore\LockRequest', ), ); } if (is_array($vals)) { - if (isset($vals['principal_name'])) { - $this->principal_name = $vals['principal_name']; - } - if (isset($vals['principal_type'])) { - $this->principal_type = $vals['principal_type']; - } - if (isset($vals['hiveObject'])) { - $this->hiveObject = $vals['hiveObject']; + if (isset($vals['rqst'])) { + $this->rqst = $vals['rqst']; } } } public function getName() { - return 'ThriftHiveMetastore_list_privileges_args'; + return 'ThriftHiveMetastore_lock_args'; } public function read($input) @@ -25993,33 +28672,19 @@ class ThriftHiveMetastore_list_privileges_args { $fname = null; $ftype = 0; $fid = 0; - $xfer += $input->readStructBegin($fname); - while (true) - { - $xfer += $input->readFieldBegin($fname, $ftype, $fid); - if ($ftype == TType::STOP) { - break; - } - switch ($fid) - { - case 1: - if ($ftype == TType::STRING) { - $xfer += $input->readString($this->principal_name); - } else { - $xfer += $input->skip($ftype); - } - break; - case 2: - if ($ftype == TType::I32) { - $xfer += $input->readI32($this->principal_type); - } else { - $xfer += $input->skip($ftype); - } - break; - case 3: + $xfer += $input->readStructBegin($fname); + while (true) + { + $xfer += $input->readFieldBegin($fname, $ftype, $fid); + if ($ftype == TType::STOP) { + break; + } + switch ($fid) + { + case 1: if ($ftype == TType::STRUCT) { - $this->hiveObject = new \metastore\HiveObjectRef(); - $xfer += $this->hiveObject->read($input); + $this->rqst = new \metastore\LockRequest(); + $xfer += $this->rqst->read($input); } else { $xfer += $input->skip($ftype); } @@ -26036,23 +28701,13 @@ class ThriftHiveMetastore_list_privileges_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_list_privileges_args'); - if ($this->principal_name !== null) { - $xfer += $output->writeFieldBegin('principal_name', TType::STRING, 1); - $xfer += $output->writeString($this->principal_name); - $xfer += $output->writeFieldEnd(); - } - if ($this->principal_type !== null) { - $xfer += $output->writeFieldBegin('principal_type', TType::I32, 2); - $xfer += $output->writeI32($this->principal_type); - $xfer += $output->writeFieldEnd(); - } - if ($this->hiveObject !== null) { - if (!is_object($this->hiveObject)) { + $xfer += $output->writeStructBegin('ThriftHiveMetastore_lock_args'); + if ($this->rqst !== null) { + if (!is_object($this->rqst)) { throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); } - $xfer += $output->writeFieldBegin('hiveObject', TType::STRUCT, 3); - $xfer += $this->hiveObject->write($output); + $xfer += $output->writeFieldBegin('rqst', TType::STRUCT, 1); + $xfer += $this->rqst->write($output); $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -26062,28 +28717,30 @@ class ThriftHiveMetastore_list_privileges_args { } -class ThriftHiveMetastore_list_privileges_result { +class ThriftHiveMetastore_lock_result { static $_TSPEC; public $success = null; public $o1 = null; + public $o2 = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 0 => array( 'var' => 'success', - 'type' => TType::LST, - 'etype' => TType::STRUCT, - 'elem' => array( - 'type' => TType::STRUCT, - 'class' => '\metastore\HiveObjectPrivilege', - ), + 'type' => TType::STRUCT, + 'class' => '\metastore\LockResponse', ), 1 => array( 'var' => 'o1', 'type' => TType::STRUCT, - 'class' => '\metastore\MetaException', + 'class' => '\metastore\NoSuchTxnException', + ), + 2 => array( + 'var' => 'o2', + 'type' => TType::STRUCT, + 'class' => '\metastore\TxnAbortedException', ), ); } @@ -26094,11 +28751,14 @@ class ThriftHiveMetastore_list_privileges_result { if (isset($vals['o1'])) { $this->o1 = $vals['o1']; } + if (isset($vals['o2'])) { + $this->o2 = $vals['o2']; + } } } public function getName() { - return 'ThriftHiveMetastore_list_privileges_result'; + return 'ThriftHiveMetastore_lock_result'; } public function read($input) @@ -26117,31 +28777,29 @@ class ThriftHiveMetastore_list_privileges_result { switch ($fid) { case 0: - if ($ftype == TType::LST) { - $this->success = array(); - $_size652 = 0; - $_etype655 = 0; - $xfer += $input->readListBegin($_etype655, $_size652); - for ($_i656 = 0; $_i656 < $_size652; ++$_i656) - { - $elem657 = null; - $elem657 = new \metastore\HiveObjectPrivilege(); - $xfer += $elem657->read($input); - $this->success []= $elem657; - } - $xfer += $input->readListEnd(); + if ($ftype == TType::STRUCT) { + $this->success = new \metastore\LockResponse(); + $xfer += $this->success->read($input); } else { $xfer += $input->skip($ftype); } break; case 1: if ($ftype == TType::STRUCT) { - $this->o1 = new \metastore\MetaException(); + $this->o1 = new \metastore\NoSuchTxnException(); $xfer += $this->o1->read($input); } else { $xfer += $input->skip($ftype); } break; + case 2: + if ($ftype == TType::STRUCT) { + $this->o2 = new \metastore\TxnAbortedException(); + $xfer += $this->o2->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; default: $xfer += $input->skip($ftype); break; @@ -26154,22 +28812,13 @@ class ThriftHiveMetastore_list_privileges_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_list_privileges_result'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_lock_result'); if ($this->success !== null) { - if (!is_array($this->success)) { + if (!is_object($this->success)) { throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); } - $xfer += $output->writeFieldBegin('success', TType::LST, 0); - { - $output->writeListBegin(TType::STRUCT, count($this->success)); - { - foreach ($this->success as $iter658) - { - $xfer += $iter658->write($output); - } - } - $output->writeListEnd(); - } + $xfer += $output->writeFieldBegin('success', TType::STRUCT, 0); + $xfer += $this->success->write($output); $xfer += $output->writeFieldEnd(); } if ($this->o1 !== null) { @@ -26177,6 +28826,11 @@ class ThriftHiveMetastore_list_privileges_result { $xfer += $this->o1->write($output); $xfer += $output->writeFieldEnd(); } + if ($this->o2 !== null) { + $xfer += $output->writeFieldBegin('o2', TType::STRUCT, 2); + $xfer += $this->o2->write($output); + $xfer += $output->writeFieldEnd(); + } $xfer += $output->writeFieldStop(); $xfer += $output->writeStructEnd(); return $xfer; @@ -26184,30 +28838,30 @@ class ThriftHiveMetastore_list_privileges_result { } -class ThriftHiveMetastore_grant_privileges_args { +class ThriftHiveMetastore_check_lock_args { static $_TSPEC; - public $privileges = null; + public $rqst = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 1 => array( - 'var' => 'privileges', + 'var' => 'rqst', 'type' => TType::STRUCT, - 'class' => '\metastore\PrivilegeBag', + 'class' => '\metastore\CheckLockRequest', ), ); } if (is_array($vals)) { - if (isset($vals['privileges'])) { - $this->privileges = $vals['privileges']; + if (isset($vals['rqst'])) { + $this->rqst = $vals['rqst']; } } } public function getName() { - return 'ThriftHiveMetastore_grant_privileges_args'; + return 'ThriftHiveMetastore_check_lock_args'; } public function read($input) @@ -26227,8 +28881,8 @@ class ThriftHiveMetastore_grant_privileges_args { { case 1: if ($ftype == TType::STRUCT) { - $this->privileges = new \metastore\PrivilegeBag(); - $xfer += $this->privileges->read($input); + $this->rqst = new \metastore\CheckLockRequest(); + $xfer += $this->rqst->read($input); } else { $xfer += $input->skip($ftype); } @@ -26245,13 +28899,13 @@ class ThriftHiveMetastore_grant_privileges_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_grant_privileges_args'); - if ($this->privileges !== null) { - if (!is_object($this->privileges)) { + $xfer += $output->writeStructBegin('ThriftHiveMetastore_check_lock_args'); + if ($this->rqst !== null) { + if (!is_object($this->rqst)) { throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); } - $xfer += $output->writeFieldBegin('privileges', TType::STRUCT, 1); - $xfer += $this->privileges->write($output); + $xfer += $output->writeFieldBegin('rqst', TType::STRUCT, 1); + $xfer += $this->rqst->write($output); $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -26261,23 +28915,36 @@ class ThriftHiveMetastore_grant_privileges_args { } -class ThriftHiveMetastore_grant_privileges_result { +class ThriftHiveMetastore_check_lock_result { static $_TSPEC; public $success = null; public $o1 = null; + public $o2 = null; + public $o3 = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 0 => array( 'var' => 'success', - 'type' => TType::BOOL, + 'type' => TType::STRUCT, + 'class' => '\metastore\LockResponse', ), 1 => array( 'var' => 'o1', 'type' => TType::STRUCT, - 'class' => '\metastore\MetaException', + 'class' => '\metastore\NoSuchTxnException', + ), + 2 => array( + 'var' => 'o2', + 'type' => TType::STRUCT, + 'class' => '\metastore\TxnAbortedException', + ), + 3 => array( + 'var' => 'o3', + 'type' => TType::STRUCT, + 'class' => '\metastore\NoSuchLockException', ), ); } @@ -26288,11 +28955,17 @@ class ThriftHiveMetastore_grant_privileges_result { if (isset($vals['o1'])) { $this->o1 = $vals['o1']; } + if (isset($vals['o2'])) { + $this->o2 = $vals['o2']; + } + if (isset($vals['o3'])) { + $this->o3 = $vals['o3']; + } } } public function getName() { - return 'ThriftHiveMetastore_grant_privileges_result'; + return 'ThriftHiveMetastore_check_lock_result'; } public function read($input) @@ -26311,20 +28984,37 @@ class ThriftHiveMetastore_grant_privileges_result { switch ($fid) { case 0: - if ($ftype == TType::BOOL) { - $xfer += $input->readBool($this->success); + if ($ftype == TType::STRUCT) { + $this->success = new \metastore\LockResponse(); + $xfer += $this->success->read($input); } else { $xfer += $input->skip($ftype); } break; case 1: if ($ftype == TType::STRUCT) { - $this->o1 = new \metastore\MetaException(); + $this->o1 = new \metastore\NoSuchTxnException(); $xfer += $this->o1->read($input); } else { $xfer += $input->skip($ftype); } break; + case 2: + if ($ftype == TType::STRUCT) { + $this->o2 = new \metastore\TxnAbortedException(); + $xfer += $this->o2->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; + case 3: + if ($ftype == TType::STRUCT) { + $this->o3 = new \metastore\NoSuchLockException(); + $xfer += $this->o3->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; default: $xfer += $input->skip($ftype); break; @@ -26337,10 +29027,13 @@ class ThriftHiveMetastore_grant_privileges_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_grant_privileges_result'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_check_lock_result'); if ($this->success !== null) { - $xfer += $output->writeFieldBegin('success', TType::BOOL, 0); - $xfer += $output->writeBool($this->success); + if (!is_object($this->success)) { + throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); + } + $xfer += $output->writeFieldBegin('success', TType::STRUCT, 0); + $xfer += $this->success->write($output); $xfer += $output->writeFieldEnd(); } if ($this->o1 !== null) { @@ -26348,6 +29041,16 @@ class ThriftHiveMetastore_grant_privileges_result { $xfer += $this->o1->write($output); $xfer += $output->writeFieldEnd(); } + if ($this->o2 !== null) { + $xfer += $output->writeFieldBegin('o2', TType::STRUCT, 2); + $xfer += $this->o2->write($output); + $xfer += $output->writeFieldEnd(); + } + if ($this->o3 !== null) { + $xfer += $output->writeFieldBegin('o3', TType::STRUCT, 3); + $xfer += $this->o3->write($output); + $xfer += $output->writeFieldEnd(); + } $xfer += $output->writeFieldStop(); $xfer += $output->writeStructEnd(); return $xfer; @@ -26355,30 +29058,30 @@ class ThriftHiveMetastore_grant_privileges_result { } -class ThriftHiveMetastore_revoke_privileges_args { +class ThriftHiveMetastore_unlock_args { static $_TSPEC; - public $privileges = null; + public $rqst = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 1 => array( - 'var' => 'privileges', + 'var' => 'rqst', 'type' => TType::STRUCT, - 'class' => '\metastore\PrivilegeBag', + 'class' => '\metastore\UnlockRequest', ), ); } if (is_array($vals)) { - if (isset($vals['privileges'])) { - $this->privileges = $vals['privileges']; + if (isset($vals['rqst'])) { + $this->rqst = $vals['rqst']; } } } public function getName() { - return 'ThriftHiveMetastore_revoke_privileges_args'; + return 'ThriftHiveMetastore_unlock_args'; } public function read($input) @@ -26398,8 +29101,8 @@ class ThriftHiveMetastore_revoke_privileges_args { { case 1: if ($ftype == TType::STRUCT) { - $this->privileges = new \metastore\PrivilegeBag(); - $xfer += $this->privileges->read($input); + $this->rqst = new \metastore\UnlockRequest(); + $xfer += $this->rqst->read($input); } else { $xfer += $input->skip($ftype); } @@ -26416,13 +29119,13 @@ class ThriftHiveMetastore_revoke_privileges_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_revoke_privileges_args'); - if ($this->privileges !== null) { - if (!is_object($this->privileges)) { + $xfer += $output->writeStructBegin('ThriftHiveMetastore_unlock_args'); + if ($this->rqst !== null) { + if (!is_object($this->rqst)) { throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); } - $xfer += $output->writeFieldBegin('privileges', TType::STRUCT, 1); - $xfer += $this->privileges->write($output); + $xfer += $output->writeFieldBegin('rqst', TType::STRUCT, 1); + $xfer += $this->rqst->write($output); $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -26432,38 +29135,39 @@ class ThriftHiveMetastore_revoke_privileges_args { } -class ThriftHiveMetastore_revoke_privileges_result { +class ThriftHiveMetastore_unlock_result { static $_TSPEC; - public $success = null; public $o1 = null; + public $o2 = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( - 0 => array( - 'var' => 'success', - 'type' => TType::BOOL, - ), 1 => array( 'var' => 'o1', 'type' => TType::STRUCT, - 'class' => '\metastore\MetaException', + 'class' => '\metastore\NoSuchLockException', + ), + 2 => array( + 'var' => 'o2', + 'type' => TType::STRUCT, + 'class' => '\metastore\TxnOpenException', ), ); } if (is_array($vals)) { - if (isset($vals['success'])) { - $this->success = $vals['success']; - } if (isset($vals['o1'])) { $this->o1 = $vals['o1']; } + if (isset($vals['o2'])) { + $this->o2 = $vals['o2']; + } } } public function getName() { - return 'ThriftHiveMetastore_revoke_privileges_result'; + return 'ThriftHiveMetastore_unlock_result'; } public function read($input) @@ -26481,17 +29185,18 @@ class ThriftHiveMetastore_revoke_privileges_result { } switch ($fid) { - case 0: - if ($ftype == TType::BOOL) { - $xfer += $input->readBool($this->success); + case 1: + if ($ftype == TType::STRUCT) { + $this->o1 = new \metastore\NoSuchLockException(); + $xfer += $this->o1->read($input); } else { $xfer += $input->skip($ftype); } break; - case 1: + case 2: if ($ftype == TType::STRUCT) { - $this->o1 = new \metastore\MetaException(); - $xfer += $this->o1->read($input); + $this->o2 = new \metastore\TxnOpenException(); + $xfer += $this->o2->read($input); } else { $xfer += $input->skip($ftype); } @@ -26508,17 +29213,17 @@ class ThriftHiveMetastore_revoke_privileges_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_revoke_privileges_result'); - if ($this->success !== null) { - $xfer += $output->writeFieldBegin('success', TType::BOOL, 0); - $xfer += $output->writeBool($this->success); - $xfer += $output->writeFieldEnd(); - } + $xfer += $output->writeStructBegin('ThriftHiveMetastore_unlock_result'); if ($this->o1 !== null) { $xfer += $output->writeFieldBegin('o1', TType::STRUCT, 1); $xfer += $this->o1->write($output); $xfer += $output->writeFieldEnd(); } + if ($this->o2 !== null) { + $xfer += $output->writeFieldBegin('o2', TType::STRUCT, 2); + $xfer += $this->o2->write($output); + $xfer += $output->writeFieldEnd(); + } $xfer += $output->writeFieldStop(); $xfer += $output->writeStructEnd(); return $xfer; @@ -26526,41 +29231,30 @@ class ThriftHiveMetastore_revoke_privileges_result { } -class ThriftHiveMetastore_set_ugi_args { +class ThriftHiveMetastore_show_locks_args { static $_TSPEC; - public $user_name = null; - public $group_names = null; + public $rqst = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 1 => array( - 'var' => 'user_name', - 'type' => TType::STRING, - ), - 2 => array( - 'var' => 'group_names', - 'type' => TType::LST, - 'etype' => TType::STRING, - 'elem' => array( - 'type' => TType::STRING, - ), + 'var' => 'rqst', + 'type' => TType::STRUCT, + 'class' => '\metastore\ShowLocksRequest', ), ); } if (is_array($vals)) { - if (isset($vals['user_name'])) { - $this->user_name = $vals['user_name']; - } - if (isset($vals['group_names'])) { - $this->group_names = $vals['group_names']; + if (isset($vals['rqst'])) { + $this->rqst = $vals['rqst']; } } } public function getName() { - return 'ThriftHiveMetastore_set_ugi_args'; + return 'ThriftHiveMetastore_show_locks_args'; } public function read($input) @@ -26579,25 +29273,9 @@ class ThriftHiveMetastore_set_ugi_args { switch ($fid) { case 1: - if ($ftype == TType::STRING) { - $xfer += $input->readString($this->user_name); - } else { - $xfer += $input->skip($ftype); - } - break; - case 2: - if ($ftype == TType::LST) { - $this->group_names = array(); - $_size659 = 0; - $_etype662 = 0; - $xfer += $input->readListBegin($_etype662, $_size659); - for ($_i663 = 0; $_i663 < $_size659; ++$_i663) - { - $elem664 = null; - $xfer += $input->readString($elem664); - $this->group_names []= $elem664; - } - $xfer += $input->readListEnd(); + if ($ftype == TType::STRUCT) { + $this->rqst = new \metastore\ShowLocksRequest(); + $xfer += $this->rqst->read($input); } else { $xfer += $input->skip($ftype); } @@ -26614,27 +29292,13 @@ class ThriftHiveMetastore_set_ugi_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_set_ugi_args'); - if ($this->user_name !== null) { - $xfer += $output->writeFieldBegin('user_name', TType::STRING, 1); - $xfer += $output->writeString($this->user_name); - $xfer += $output->writeFieldEnd(); - } - if ($this->group_names !== null) { - if (!is_array($this->group_names)) { + $xfer += $output->writeStructBegin('ThriftHiveMetastore_show_locks_args'); + if ($this->rqst !== null) { + if (!is_object($this->rqst)) { throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); } - $xfer += $output->writeFieldBegin('group_names', TType::LST, 2); - { - $output->writeListBegin(TType::STRING, count($this->group_names)); - { - foreach ($this->group_names as $iter665) - { - $xfer += $output->writeString($iter665); - } - } - $output->writeListEnd(); - } + $xfer += $output->writeFieldBegin('rqst', TType::STRUCT, 1); + $xfer += $this->rqst->write($output); $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -26644,27 +29308,18 @@ class ThriftHiveMetastore_set_ugi_args { } -class ThriftHiveMetastore_set_ugi_result { +class ThriftHiveMetastore_show_locks_result { static $_TSPEC; public $success = null; - public $o1 = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 0 => array( 'var' => 'success', - 'type' => TType::LST, - 'etype' => TType::STRING, - 'elem' => array( - 'type' => TType::STRING, - ), - ), - 1 => array( - 'var' => 'o1', 'type' => TType::STRUCT, - 'class' => '\metastore\MetaException', + 'class' => '\metastore\ShowLocksResponse', ), ); } @@ -26672,14 +29327,11 @@ class ThriftHiveMetastore_set_ugi_result { if (isset($vals['success'])) { $this->success = $vals['success']; } - if (isset($vals['o1'])) { - $this->o1 = $vals['o1']; - } } } public function getName() { - return 'ThriftHiveMetastore_set_ugi_result'; + return 'ThriftHiveMetastore_show_locks_result'; } public function read($input) @@ -26698,26 +29350,9 @@ class ThriftHiveMetastore_set_ugi_result { switch ($fid) { case 0: - if ($ftype == TType::LST) { - $this->success = array(); - $_size666 = 0; - $_etype669 = 0; - $xfer += $input->readListBegin($_etype669, $_size666); - for ($_i670 = 0; $_i670 < $_size666; ++$_i670) - { - $elem671 = null; - $xfer += $input->readString($elem671); - $this->success []= $elem671; - } - $xfer += $input->readListEnd(); - } else { - $xfer += $input->skip($ftype); - } - break; - case 1: if ($ftype == TType::STRUCT) { - $this->o1 = new \metastore\MetaException(); - $xfer += $this->o1->read($input); + $this->success = new \metastore\ShowLocksResponse(); + $xfer += $this->success->read($input); } else { $xfer += $input->skip($ftype); } @@ -26734,27 +29369,13 @@ class ThriftHiveMetastore_set_ugi_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_set_ugi_result'); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_show_locks_result'); if ($this->success !== null) { - if (!is_array($this->success)) { + if (!is_object($this->success)) { throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); } - $xfer += $output->writeFieldBegin('success', TType::LST, 0); - { - $output->writeListBegin(TType::STRING, count($this->success)); - { - foreach ($this->success as $iter672) - { - $xfer += $output->writeString($iter672); - } - } - $output->writeListEnd(); - } - $xfer += $output->writeFieldEnd(); - } - if ($this->o1 !== null) { - $xfer += $output->writeFieldBegin('o1', TType::STRUCT, 1); - $xfer += $this->o1->write($output); + $xfer += $output->writeFieldBegin('success', TType::STRUCT, 0); + $xfer += $this->success->write($output); $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -26764,37 +29385,30 @@ class ThriftHiveMetastore_set_ugi_result { } -class ThriftHiveMetastore_get_delegation_token_args { +class ThriftHiveMetastore_heartbeat_args { static $_TSPEC; - public $token_owner = null; - public $renewer_kerberos_principal_name = null; + public $ids = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 1 => array( - 'var' => 'token_owner', - 'type' => TType::STRING, - ), - 2 => array( - 'var' => 'renewer_kerberos_principal_name', - 'type' => TType::STRING, + 'var' => 'ids', + 'type' => TType::STRUCT, + 'class' => '\metastore\HeartbeatRequest', ), ); } if (is_array($vals)) { - if (isset($vals['token_owner'])) { - $this->token_owner = $vals['token_owner']; - } - if (isset($vals['renewer_kerberos_principal_name'])) { - $this->renewer_kerberos_principal_name = $vals['renewer_kerberos_principal_name']; + if (isset($vals['ids'])) { + $this->ids = $vals['ids']; } } } public function getName() { - return 'ThriftHiveMetastore_get_delegation_token_args'; + return 'ThriftHiveMetastore_heartbeat_args'; } public function read($input) @@ -26813,15 +29427,9 @@ class ThriftHiveMetastore_get_delegation_token_args { switch ($fid) { case 1: - if ($ftype == TType::STRING) { - $xfer += $input->readString($this->token_owner); - } else { - $xfer += $input->skip($ftype); - } - break; - case 2: - if ($ftype == TType::STRING) { - $xfer += $input->readString($this->renewer_kerberos_principal_name); + if ($ftype == TType::STRUCT) { + $this->ids = new \metastore\HeartbeatRequest(); + $xfer += $this->ids->read($input); } else { $xfer += $input->skip($ftype); } @@ -26838,15 +29446,13 @@ class ThriftHiveMetastore_get_delegation_token_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_delegation_token_args'); - if ($this->token_owner !== null) { - $xfer += $output->writeFieldBegin('token_owner', TType::STRING, 1); - $xfer += $output->writeString($this->token_owner); - $xfer += $output->writeFieldEnd(); - } - if ($this->renewer_kerberos_principal_name !== null) { - $xfer += $output->writeFieldBegin('renewer_kerberos_principal_name', TType::STRING, 2); - $xfer += $output->writeString($this->renewer_kerberos_principal_name); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_heartbeat_args'); + if ($this->ids !== null) { + if (!is_object($this->ids)) { + throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); + } + $xfer += $output->writeFieldBegin('ids', TType::STRUCT, 1); + $xfer += $this->ids->write($output); $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -26856,38 +29462,48 @@ class ThriftHiveMetastore_get_delegation_token_args { } -class ThriftHiveMetastore_get_delegation_token_result { +class ThriftHiveMetastore_heartbeat_result { static $_TSPEC; - public $success = null; public $o1 = null; + public $o2 = null; + public $o3 = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( - 0 => array( - 'var' => 'success', - 'type' => TType::STRING, - ), 1 => array( 'var' => 'o1', 'type' => TType::STRUCT, - 'class' => '\metastore\MetaException', + 'class' => '\metastore\NoSuchLockException', + ), + 2 => array( + 'var' => 'o2', + 'type' => TType::STRUCT, + 'class' => '\metastore\NoSuchTxnException', + ), + 3 => array( + 'var' => 'o3', + 'type' => TType::STRUCT, + 'class' => '\metastore\TxnAbortedException', ), ); } if (is_array($vals)) { - if (isset($vals['success'])) { - $this->success = $vals['success']; - } if (isset($vals['o1'])) { $this->o1 = $vals['o1']; } + if (isset($vals['o2'])) { + $this->o2 = $vals['o2']; + } + if (isset($vals['o3'])) { + $this->o3 = $vals['o3']; + } } } public function getName() { - return 'ThriftHiveMetastore_get_delegation_token_result'; + return 'ThriftHiveMetastore_heartbeat_result'; } public function read($input) @@ -26905,17 +29521,26 @@ class ThriftHiveMetastore_get_delegation_token_result { } switch ($fid) { - case 0: - if ($ftype == TType::STRING) { - $xfer += $input->readString($this->success); + case 1: + if ($ftype == TType::STRUCT) { + $this->o1 = new \metastore\NoSuchLockException(); + $xfer += $this->o1->read($input); } else { $xfer += $input->skip($ftype); } break; - case 1: + case 2: if ($ftype == TType::STRUCT) { - $this->o1 = new \metastore\MetaException(); - $xfer += $this->o1->read($input); + $this->o2 = new \metastore\NoSuchTxnException(); + $xfer += $this->o2->read($input); + } else { + $xfer += $input->skip($ftype); + } + break; + case 3: + if ($ftype == TType::STRUCT) { + $this->o3 = new \metastore\TxnAbortedException(); + $xfer += $this->o3->read($input); } else { $xfer += $input->skip($ftype); } @@ -26932,17 +29557,22 @@ class ThriftHiveMetastore_get_delegation_token_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_delegation_token_result'); - if ($this->success !== null) { - $xfer += $output->writeFieldBegin('success', TType::STRING, 0); - $xfer += $output->writeString($this->success); - $xfer += $output->writeFieldEnd(); - } + $xfer += $output->writeStructBegin('ThriftHiveMetastore_heartbeat_result'); if ($this->o1 !== null) { $xfer += $output->writeFieldBegin('o1', TType::STRUCT, 1); $xfer += $this->o1->write($output); $xfer += $output->writeFieldEnd(); } + if ($this->o2 !== null) { + $xfer += $output->writeFieldBegin('o2', TType::STRUCT, 2); + $xfer += $this->o2->write($output); + $xfer += $output->writeFieldEnd(); + } + if ($this->o3 !== null) { + $xfer += $output->writeFieldBegin('o3', TType::STRUCT, 3); + $xfer += $this->o3->write($output); + $xfer += $output->writeFieldEnd(); + } $xfer += $output->writeFieldStop(); $xfer += $output->writeStructEnd(); return $xfer; @@ -26950,29 +29580,30 @@ class ThriftHiveMetastore_get_delegation_token_result { } -class ThriftHiveMetastore_renew_delegation_token_args { +class ThriftHiveMetastore_compact_args { static $_TSPEC; - public $token_str_form = null; + public $rqst = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 1 => array( - 'var' => 'token_str_form', - 'type' => TType::STRING, + 'var' => 'rqst', + 'type' => TType::STRUCT, + 'class' => '\metastore\CompactionRequest', ), ); } if (is_array($vals)) { - if (isset($vals['token_str_form'])) { - $this->token_str_form = $vals['token_str_form']; + if (isset($vals['rqst'])) { + $this->rqst = $vals['rqst']; } } } public function getName() { - return 'ThriftHiveMetastore_renew_delegation_token_args'; + return 'ThriftHiveMetastore_compact_args'; } public function read($input) @@ -26991,8 +29622,9 @@ class ThriftHiveMetastore_renew_delegation_token_args { switch ($fid) { case 1: - if ($ftype == TType::STRING) { - $xfer += $input->readString($this->token_str_form); + if ($ftype == TType::STRUCT) { + $this->rqst = new \metastore\CompactionRequest(); + $xfer += $this->rqst->read($input); } else { $xfer += $input->skip($ftype); } @@ -27009,10 +29641,13 @@ class ThriftHiveMetastore_renew_delegation_token_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_renew_delegation_token_args'); - if ($this->token_str_form !== null) { - $xfer += $output->writeFieldBegin('token_str_form', TType::STRING, 1); - $xfer += $output->writeString($this->token_str_form); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_compact_args'); + if ($this->rqst !== null) { + if (!is_object($this->rqst)) { + throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); + } + $xfer += $output->writeFieldBegin('rqst', TType::STRUCT, 1); + $xfer += $this->rqst->write($output); $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -27022,38 +29657,19 @@ class ThriftHiveMetastore_renew_delegation_token_args { } -class ThriftHiveMetastore_renew_delegation_token_result { +class ThriftHiveMetastore_compact_result { static $_TSPEC; - public $success = null; - public $o1 = null; - public function __construct($vals=null) { + public function __construct() { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( - 0 => array( - 'var' => 'success', - 'type' => TType::I64, - ), - 1 => array( - 'var' => 'o1', - 'type' => TType::STRUCT, - 'class' => '\metastore\MetaException', - ), ); } - if (is_array($vals)) { - if (isset($vals['success'])) { - $this->success = $vals['success']; - } - if (isset($vals['o1'])) { - $this->o1 = $vals['o1']; - } - } } public function getName() { - return 'ThriftHiveMetastore_renew_delegation_token_result'; + return 'ThriftHiveMetastore_compact_result'; } public function read($input) @@ -27071,21 +29687,6 @@ class ThriftHiveMetastore_renew_delegation_token_result { } switch ($fid) { - case 0: - if ($ftype == TType::I64) { - $xfer += $input->readI64($this->success); - } else { - $xfer += $input->skip($ftype); - } - break; - case 1: - if ($ftype == TType::STRUCT) { - $this->o1 = new \metastore\MetaException(); - $xfer += $this->o1->read($input); - } else { - $xfer += $input->skip($ftype); - } - break; default: $xfer += $input->skip($ftype); break; @@ -27098,17 +29699,7 @@ class ThriftHiveMetastore_renew_delegation_token_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_renew_delegation_token_result'); - if ($this->success !== null) { - $xfer += $output->writeFieldBegin('success', TType::I64, 0); - $xfer += $output->writeI64($this->success); - $xfer += $output->writeFieldEnd(); - } - if ($this->o1 !== null) { - $xfer += $output->writeFieldBegin('o1', TType::STRUCT, 1); - $xfer += $this->o1->write($output); - $xfer += $output->writeFieldEnd(); - } + $xfer += $output->writeStructBegin('ThriftHiveMetastore_compact_result'); $xfer += $output->writeFieldStop(); $xfer += $output->writeStructEnd(); return $xfer; @@ -27116,29 +29707,30 @@ class ThriftHiveMetastore_renew_delegation_token_result { } -class ThriftHiveMetastore_cancel_delegation_token_args { +class ThriftHiveMetastore_show_compact_args { static $_TSPEC; - public $token_str_form = null; + public $rqst = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 1 => array( - 'var' => 'token_str_form', - 'type' => TType::STRING, + 'var' => 'rqst', + 'type' => TType::STRUCT, + 'class' => '\metastore\ShowCompactRequest', ), ); } if (is_array($vals)) { - if (isset($vals['token_str_form'])) { - $this->token_str_form = $vals['token_str_form']; + if (isset($vals['rqst'])) { + $this->rqst = $vals['rqst']; } } } public function getName() { - return 'ThriftHiveMetastore_cancel_delegation_token_args'; + return 'ThriftHiveMetastore_show_compact_args'; } public function read($input) @@ -27157,8 +29749,9 @@ class ThriftHiveMetastore_cancel_delegation_token_args { switch ($fid) { case 1: - if ($ftype == TType::STRING) { - $xfer += $input->readString($this->token_str_form); + if ($ftype == TType::STRUCT) { + $this->rqst = new \metastore\ShowCompactRequest(); + $xfer += $this->rqst->read($input); } else { $xfer += $input->skip($ftype); } @@ -27175,10 +29768,13 @@ class ThriftHiveMetastore_cancel_delegation_token_args { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_cancel_delegation_token_args'); - if ($this->token_str_form !== null) { - $xfer += $output->writeFieldBegin('token_str_form', TType::STRING, 1); - $xfer += $output->writeString($this->token_str_form); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_show_compact_args'); + if ($this->rqst !== null) { + if (!is_object($this->rqst)) { + throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); + } + $xfer += $output->writeFieldBegin('rqst', TType::STRUCT, 1); + $xfer += $this->rqst->write($output); $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -27188,30 +29784,30 @@ class ThriftHiveMetastore_cancel_delegation_token_args { } -class ThriftHiveMetastore_cancel_delegation_token_result { +class ThriftHiveMetastore_show_compact_result { static $_TSPEC; - public $o1 = null; + public $success = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( - 1 => array( - 'var' => 'o1', + 0 => array( + 'var' => 'success', 'type' => TType::STRUCT, - 'class' => '\metastore\MetaException', + 'class' => '\metastore\ShowCompactResponse', ), ); } if (is_array($vals)) { - if (isset($vals['o1'])) { - $this->o1 = $vals['o1']; + if (isset($vals['success'])) { + $this->success = $vals['success']; } } } public function getName() { - return 'ThriftHiveMetastore_cancel_delegation_token_result'; + return 'ThriftHiveMetastore_show_compact_result'; } public function read($input) @@ -27229,10 +29825,10 @@ class ThriftHiveMetastore_cancel_delegation_token_result { } switch ($fid) { - case 1: + case 0: if ($ftype == TType::STRUCT) { - $this->o1 = new \metastore\MetaException(); - $xfer += $this->o1->read($input); + $this->success = new \metastore\ShowCompactResponse(); + $xfer += $this->success->read($input); } else { $xfer += $input->skip($ftype); } @@ -27249,10 +29845,13 @@ class ThriftHiveMetastore_cancel_delegation_token_result { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ThriftHiveMetastore_cancel_delegation_token_result'); - if ($this->o1 !== null) { - $xfer += $output->writeFieldBegin('o1', TType::STRUCT, 1); - $xfer += $this->o1->write($output); + $xfer += $output->writeStructBegin('ThriftHiveMetastore_show_compact_result'); + if ($this->success !== null) { + if (!is_object($this->success)) { + throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); + } + $xfer += $output->writeFieldBegin('success', TType::STRUCT, 0); + $xfer += $this->success->write($output); $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); diff --git metastore/src/gen/thrift/gen-php/metastore/Types.php metastore/src/gen/thrift/gen-php/metastore/Types.php index eda2bde..6e1e3bb 100644 --- metastore/src/gen/thrift/gen-php/metastore/Types.php +++ metastore/src/gen/thrift/gen-php/metastore/Types.php @@ -49,6 +49,61 @@ final class PartitionEventType { ); } +final class TxnState { + const COMMITTED = 1; + const ABORTED = 2; + const OPEN = 3; + static public $__names = array( + 1 => 'COMMITTED', + 2 => 'ABORTED', + 3 => 'OPEN', + ); +} + +final class LockLevel { + const DB = 1; + const TABLE = 2; + const PARTITION = 3; + static public $__names = array( + 1 => 'DB', + 2 => 'TABLE', + 3 => 'PARTITION', + ); +} + +final class LockState { + const ACQUIRED = 1; + const WAITING = 2; + const ABORT = 3; + const NOT_ACQUIRED = 4; + static public $__names = array( + 1 => 'ACQUIRED', + 2 => 'WAITING', + 3 => 'ABORT', + 4 => 'NOT_ACQUIRED', + ); +} + +final class LockType { + const SHARED_READ = 1; + const SHARED_WRITE = 2; + const EXCLUSIVE = 3; + static public $__names = array( + 1 => 'SHARED_READ', + 2 => 'SHARED_WRITE', + 3 => 'EXCLUSIVE', + ); +} + +final class CompactionType { + const MINOR = 1; + const MAJOR = 2; + static public $__names = array( + 1 => 'MINOR', + 2 => 'MAJOR', + ); +} + final class FunctionType { const JAVA = 1; static public $__names = array( @@ -7040,29 +7095,53 @@ class Function { } -class MetaException extends TException { +class TxnInfo { static $_TSPEC; - public $message = null; + public $id = null; + public $state = null; + public $user = null; + public $hostname = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 1 => array( - 'var' => 'message', + 'var' => 'id', + 'type' => TType::I64, + ), + 2 => array( + 'var' => 'state', + 'type' => TType::I32, + ), + 3 => array( + 'var' => 'user', + 'type' => TType::STRING, + ), + 4 => array( + 'var' => 'hostname', 'type' => TType::STRING, ), ); } if (is_array($vals)) { - if (isset($vals['message'])) { - $this->message = $vals['message']; + if (isset($vals['id'])) { + $this->id = $vals['id']; + } + if (isset($vals['state'])) { + $this->state = $vals['state']; + } + if (isset($vals['user'])) { + $this->user = $vals['user']; + } + if (isset($vals['hostname'])) { + $this->hostname = $vals['hostname']; } } } public function getName() { - return 'MetaException'; + return 'TxnInfo'; } public function read($input) @@ -7081,8 +7160,29 @@ class MetaException extends TException { switch ($fid) { case 1: + if ($ftype == TType::I64) { + $xfer += $input->readI64($this->id); + } else { + $xfer += $input->skip($ftype); + } + break; + case 2: + if ($ftype == TType::I32) { + $xfer += $input->readI32($this->state); + } else { + $xfer += $input->skip($ftype); + } + break; + case 3: if ($ftype == TType::STRING) { - $xfer += $input->readString($this->message); + $xfer += $input->readString($this->user); + } else { + $xfer += $input->skip($ftype); + } + break; + case 4: + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->hostname); } else { $xfer += $input->skip($ftype); } @@ -7099,10 +7199,25 @@ class MetaException extends TException { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('MetaException'); - if ($this->message !== null) { - $xfer += $output->writeFieldBegin('message', TType::STRING, 1); - $xfer += $output->writeString($this->message); + $xfer += $output->writeStructBegin('TxnInfo'); + if ($this->id !== null) { + $xfer += $output->writeFieldBegin('id', TType::I64, 1); + $xfer += $output->writeI64($this->id); + $xfer += $output->writeFieldEnd(); + } + if ($this->state !== null) { + $xfer += $output->writeFieldBegin('state', TType::I32, 2); + $xfer += $output->writeI32($this->state); + $xfer += $output->writeFieldEnd(); + } + if ($this->user !== null) { + $xfer += $output->writeFieldBegin('user', TType::STRING, 3); + $xfer += $output->writeString($this->user); + $xfer += $output->writeFieldEnd(); + } + if ($this->hostname !== null) { + $xfer += $output->writeFieldBegin('hostname', TType::STRING, 4); + $xfer += $output->writeString($this->hostname); $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -7112,29 +7227,42 @@ class MetaException extends TException { } -class UnknownTableException extends TException { +class GetOpenTxnsInfoResponse { static $_TSPEC; - public $message = null; + public $txn_high_water_mark = null; + public $open_txns = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 1 => array( - 'var' => 'message', - 'type' => TType::STRING, + 'var' => 'txn_high_water_mark', + 'type' => TType::I64, + ), + 2 => array( + 'var' => 'open_txns', + 'type' => TType::LST, + 'etype' => TType::STRUCT, + 'elem' => array( + 'type' => TType::STRUCT, + 'class' => '\metastore\TxnInfo', + ), ), ); } if (is_array($vals)) { - if (isset($vals['message'])) { - $this->message = $vals['message']; + if (isset($vals['txn_high_water_mark'])) { + $this->txn_high_water_mark = $vals['txn_high_water_mark']; + } + if (isset($vals['open_txns'])) { + $this->open_txns = $vals['open_txns']; } } } public function getName() { - return 'UnknownTableException'; + return 'GetOpenTxnsInfoResponse'; } public function read($input) @@ -7153,8 +7281,26 @@ class UnknownTableException extends TException { switch ($fid) { case 1: - if ($ftype == TType::STRING) { - $xfer += $input->readString($this->message); + if ($ftype == TType::I64) { + $xfer += $input->readI64($this->txn_high_water_mark); + } else { + $xfer += $input->skip($ftype); + } + break; + case 2: + if ($ftype == TType::LST) { + $this->open_txns = array(); + $_size320 = 0; + $_etype323 = 0; + $xfer += $input->readListBegin($_etype323, $_size320); + for ($_i324 = 0; $_i324 < $_size320; ++$_i324) + { + $elem325 = null; + $elem325 = new \metastore\TxnInfo(); + $xfer += $elem325->read($input); + $this->open_txns []= $elem325; + } + $xfer += $input->readListEnd(); } else { $xfer += $input->skip($ftype); } @@ -7171,10 +7317,27 @@ class UnknownTableException extends TException { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('UnknownTableException'); - if ($this->message !== null) { - $xfer += $output->writeFieldBegin('message', TType::STRING, 1); - $xfer += $output->writeString($this->message); + $xfer += $output->writeStructBegin('GetOpenTxnsInfoResponse'); + if ($this->txn_high_water_mark !== null) { + $xfer += $output->writeFieldBegin('txn_high_water_mark', TType::I64, 1); + $xfer += $output->writeI64($this->txn_high_water_mark); + $xfer += $output->writeFieldEnd(); + } + if ($this->open_txns !== null) { + if (!is_array($this->open_txns)) { + throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); + } + $xfer += $output->writeFieldBegin('open_txns', TType::LST, 2); + { + $output->writeListBegin(TType::STRUCT, count($this->open_txns)); + { + foreach ($this->open_txns as $iter326) + { + $xfer += $iter326->write($output); + } + } + $output->writeListEnd(); + } $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -7184,29 +7347,41 @@ class UnknownTableException extends TException { } -class UnknownDBException extends TException { +class GetOpenTxnsResponse { static $_TSPEC; - public $message = null; + public $txn_high_water_mark = null; + public $open_txns = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 1 => array( - 'var' => 'message', - 'type' => TType::STRING, + 'var' => 'txn_high_water_mark', + 'type' => TType::I64, + ), + 2 => array( + 'var' => 'open_txns', + 'type' => TType::SET, + 'etype' => TType::I64, + 'elem' => array( + 'type' => TType::I64, + ), ), ); } if (is_array($vals)) { - if (isset($vals['message'])) { - $this->message = $vals['message']; + if (isset($vals['txn_high_water_mark'])) { + $this->txn_high_water_mark = $vals['txn_high_water_mark']; + } + if (isset($vals['open_txns'])) { + $this->open_txns = $vals['open_txns']; } } } public function getName() { - return 'UnknownDBException'; + return 'GetOpenTxnsResponse'; } public function read($input) @@ -7225,8 +7400,29 @@ class UnknownDBException extends TException { switch ($fid) { case 1: - if ($ftype == TType::STRING) { - $xfer += $input->readString($this->message); + if ($ftype == TType::I64) { + $xfer += $input->readI64($this->txn_high_water_mark); + } else { + $xfer += $input->skip($ftype); + } + break; + case 2: + if ($ftype == TType::SET) { + $this->open_txns = array(); + $_size327 = 0; + $_etype330 = 0; + $xfer += $input->readSetBegin($_etype330, $_size327); + for ($_i331 = 0; $_i331 < $_size327; ++$_i331) + { + $elem332 = null; + $xfer += $input->readI64($elem332); + if (is_scalar($elem332)) { + $this->open_txns[$elem332] = true; + } else { + $this->open_txns []= $elem332; + } + } + $xfer += $input->readSetEnd(); } else { $xfer += $input->skip($ftype); } @@ -7243,10 +7439,31 @@ class UnknownDBException extends TException { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('UnknownDBException'); - if ($this->message !== null) { - $xfer += $output->writeFieldBegin('message', TType::STRING, 1); - $xfer += $output->writeString($this->message); + $xfer += $output->writeStructBegin('GetOpenTxnsResponse'); + if ($this->txn_high_water_mark !== null) { + $xfer += $output->writeFieldBegin('txn_high_water_mark', TType::I64, 1); + $xfer += $output->writeI64($this->txn_high_water_mark); + $xfer += $output->writeFieldEnd(); + } + if ($this->open_txns !== null) { + if (!is_array($this->open_txns)) { + throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); + } + $xfer += $output->writeFieldBegin('open_txns', TType::SET, 2); + { + $output->writeSetBegin(TType::I64, count($this->open_txns)); + { + foreach ($this->open_txns as $iter333 => $iter334) + { + if (is_scalar($iter334)) { + $xfer += $output->writeI64($iter333); + } else { + $xfer += $output->writeI64($iter334); + } + } + } + $output->writeSetEnd(); + } $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -7256,29 +7473,45 @@ class UnknownDBException extends TException { } -class AlreadyExistsException extends TException { +class OpenTxnRequest { static $_TSPEC; - public $message = null; + public $num_txns = null; + public $user = null; + public $hostname = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 1 => array( - 'var' => 'message', + 'var' => 'num_txns', + 'type' => TType::I32, + ), + 2 => array( + 'var' => 'user', + 'type' => TType::STRING, + ), + 3 => array( + 'var' => 'hostname', 'type' => TType::STRING, ), ); } if (is_array($vals)) { - if (isset($vals['message'])) { - $this->message = $vals['message']; + if (isset($vals['num_txns'])) { + $this->num_txns = $vals['num_txns']; + } + if (isset($vals['user'])) { + $this->user = $vals['user']; + } + if (isset($vals['hostname'])) { + $this->hostname = $vals['hostname']; } } } public function getName() { - return 'AlreadyExistsException'; + return 'OpenTxnRequest'; } public function read($input) @@ -7297,8 +7530,22 @@ class AlreadyExistsException extends TException { switch ($fid) { case 1: + if ($ftype == TType::I32) { + $xfer += $input->readI32($this->num_txns); + } else { + $xfer += $input->skip($ftype); + } + break; + case 2: if ($ftype == TType::STRING) { - $xfer += $input->readString($this->message); + $xfer += $input->readString($this->user); + } else { + $xfer += $input->skip($ftype); + } + break; + case 3: + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->hostname); } else { $xfer += $input->skip($ftype); } @@ -7315,10 +7562,20 @@ class AlreadyExistsException extends TException { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('AlreadyExistsException'); - if ($this->message !== null) { - $xfer += $output->writeFieldBegin('message', TType::STRING, 1); - $xfer += $output->writeString($this->message); + $xfer += $output->writeStructBegin('OpenTxnRequest'); + if ($this->num_txns !== null) { + $xfer += $output->writeFieldBegin('num_txns', TType::I32, 1); + $xfer += $output->writeI32($this->num_txns); + $xfer += $output->writeFieldEnd(); + } + if ($this->user !== null) { + $xfer += $output->writeFieldBegin('user', TType::STRING, 2); + $xfer += $output->writeString($this->user); + $xfer += $output->writeFieldEnd(); + } + if ($this->hostname !== null) { + $xfer += $output->writeFieldBegin('hostname', TType::STRING, 3); + $xfer += $output->writeString($this->hostname); $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -7328,29 +7585,33 @@ class AlreadyExistsException extends TException { } -class InvalidPartitionException extends TException { +class OpenTxnsResponse { static $_TSPEC; - public $message = null; + public $txn_ids = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 1 => array( - 'var' => 'message', - 'type' => TType::STRING, + 'var' => 'txn_ids', + 'type' => TType::LST, + 'etype' => TType::I64, + 'elem' => array( + 'type' => TType::I64, + ), ), ); } if (is_array($vals)) { - if (isset($vals['message'])) { - $this->message = $vals['message']; + if (isset($vals['txn_ids'])) { + $this->txn_ids = $vals['txn_ids']; } } } public function getName() { - return 'InvalidPartitionException'; + return 'OpenTxnsResponse'; } public function read($input) @@ -7369,8 +7630,18 @@ class InvalidPartitionException extends TException { switch ($fid) { case 1: - if ($ftype == TType::STRING) { - $xfer += $input->readString($this->message); + if ($ftype == TType::LST) { + $this->txn_ids = array(); + $_size335 = 0; + $_etype338 = 0; + $xfer += $input->readListBegin($_etype338, $_size335); + for ($_i339 = 0; $_i339 < $_size335; ++$_i339) + { + $elem340 = null; + $xfer += $input->readI64($elem340); + $this->txn_ids []= $elem340; + } + $xfer += $input->readListEnd(); } else { $xfer += $input->skip($ftype); } @@ -7387,10 +7658,22 @@ class InvalidPartitionException extends TException { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('InvalidPartitionException'); - if ($this->message !== null) { - $xfer += $output->writeFieldBegin('message', TType::STRING, 1); - $xfer += $output->writeString($this->message); + $xfer += $output->writeStructBegin('OpenTxnsResponse'); + if ($this->txn_ids !== null) { + if (!is_array($this->txn_ids)) { + throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); + } + $xfer += $output->writeFieldBegin('txn_ids', TType::LST, 1); + { + $output->writeListBegin(TType::I64, count($this->txn_ids)); + { + foreach ($this->txn_ids as $iter341) + { + $xfer += $output->writeI64($iter341); + } + } + $output->writeListEnd(); + } $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -7400,29 +7683,29 @@ class InvalidPartitionException extends TException { } -class UnknownPartitionException extends TException { +class AbortTxnRequest { static $_TSPEC; - public $message = null; + public $txnid = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 1 => array( - 'var' => 'message', - 'type' => TType::STRING, + 'var' => 'txnid', + 'type' => TType::I64, ), ); } if (is_array($vals)) { - if (isset($vals['message'])) { - $this->message = $vals['message']; + if (isset($vals['txnid'])) { + $this->txnid = $vals['txnid']; } } } public function getName() { - return 'UnknownPartitionException'; + return 'AbortTxnRequest'; } public function read($input) @@ -7441,8 +7724,8 @@ class UnknownPartitionException extends TException { switch ($fid) { case 1: - if ($ftype == TType::STRING) { - $xfer += $input->readString($this->message); + if ($ftype == TType::I64) { + $xfer += $input->readI64($this->txnid); } else { $xfer += $input->skip($ftype); } @@ -7459,10 +7742,10 @@ class UnknownPartitionException extends TException { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('UnknownPartitionException'); - if ($this->message !== null) { - $xfer += $output->writeFieldBegin('message', TType::STRING, 1); - $xfer += $output->writeString($this->message); + $xfer += $output->writeStructBegin('AbortTxnRequest'); + if ($this->txnid !== null) { + $xfer += $output->writeFieldBegin('txnid', TType::I64, 1); + $xfer += $output->writeI64($this->txnid); $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -7472,29 +7755,29 @@ class UnknownPartitionException extends TException { } -class InvalidObjectException extends TException { +class CommitTxnRequest { static $_TSPEC; - public $message = null; + public $txnid = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 1 => array( - 'var' => 'message', - 'type' => TType::STRING, + 'var' => 'txnid', + 'type' => TType::I64, ), ); } if (is_array($vals)) { - if (isset($vals['message'])) { - $this->message = $vals['message']; + if (isset($vals['txnid'])) { + $this->txnid = $vals['txnid']; } } } public function getName() { - return 'InvalidObjectException'; + return 'CommitTxnRequest'; } public function read($input) @@ -7513,8 +7796,8 @@ class InvalidObjectException extends TException { switch ($fid) { case 1: - if ($ftype == TType::STRING) { - $xfer += $input->readString($this->message); + if ($ftype == TType::I64) { + $xfer += $input->readI64($this->txnid); } else { $xfer += $input->skip($ftype); } @@ -7531,10 +7814,10 @@ class InvalidObjectException extends TException { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('InvalidObjectException'); - if ($this->message !== null) { - $xfer += $output->writeFieldBegin('message', TType::STRING, 1); - $xfer += $output->writeString($this->message); + $xfer += $output->writeStructBegin('CommitTxnRequest'); + if ($this->txnid !== null) { + $xfer += $output->writeFieldBegin('txnid', TType::I64, 1); + $xfer += $output->writeI64($this->txnid); $xfer += $output->writeFieldEnd(); } $xfer += $output->writeFieldStop(); @@ -7544,20 +7827,2172 @@ class InvalidObjectException extends TException { } -class NoSuchObjectException extends TException { +class LockComponent { static $_TSPEC; - public $message = null; + public $type = null; + public $level = null; + public $dbname = null; + public $tablename = null; + public $partitionname = null; public function __construct($vals=null) { if (!isset(self::$_TSPEC)) { self::$_TSPEC = array( 1 => array( - 'var' => 'message', + 'var' => 'type', + 'type' => TType::I32, + ), + 2 => array( + 'var' => 'level', + 'type' => TType::I32, + ), + 3 => array( + 'var' => 'dbname', 'type' => TType::STRING, ), - ); - } + 4 => array( + 'var' => 'tablename', + 'type' => TType::STRING, + ), + 5 => array( + 'var' => 'partitionname', + 'type' => TType::STRING, + ), + ); + } + if (is_array($vals)) { + if (isset($vals['type'])) { + $this->type = $vals['type']; + } + if (isset($vals['level'])) { + $this->level = $vals['level']; + } + if (isset($vals['dbname'])) { + $this->dbname = $vals['dbname']; + } + if (isset($vals['tablename'])) { + $this->tablename = $vals['tablename']; + } + if (isset($vals['partitionname'])) { + $this->partitionname = $vals['partitionname']; + } + } + } + + public function getName() { + return 'LockComponent'; + } + + public function read($input) + { + $xfer = 0; + $fname = null; + $ftype = 0; + $fid = 0; + $xfer += $input->readStructBegin($fname); + while (true) + { + $xfer += $input->readFieldBegin($fname, $ftype, $fid); + if ($ftype == TType::STOP) { + break; + } + switch ($fid) + { + case 1: + if ($ftype == TType::I32) { + $xfer += $input->readI32($this->type); + } else { + $xfer += $input->skip($ftype); + } + break; + case 2: + if ($ftype == TType::I32) { + $xfer += $input->readI32($this->level); + } else { + $xfer += $input->skip($ftype); + } + break; + case 3: + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->dbname); + } else { + $xfer += $input->skip($ftype); + } + break; + case 4: + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->tablename); + } else { + $xfer += $input->skip($ftype); + } + break; + case 5: + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->partitionname); + } else { + $xfer += $input->skip($ftype); + } + break; + default: + $xfer += $input->skip($ftype); + break; + } + $xfer += $input->readFieldEnd(); + } + $xfer += $input->readStructEnd(); + return $xfer; + } + + public function write($output) { + $xfer = 0; + $xfer += $output->writeStructBegin('LockComponent'); + if ($this->type !== null) { + $xfer += $output->writeFieldBegin('type', TType::I32, 1); + $xfer += $output->writeI32($this->type); + $xfer += $output->writeFieldEnd(); + } + if ($this->level !== null) { + $xfer += $output->writeFieldBegin('level', TType::I32, 2); + $xfer += $output->writeI32($this->level); + $xfer += $output->writeFieldEnd(); + } + if ($this->dbname !== null) { + $xfer += $output->writeFieldBegin('dbname', TType::STRING, 3); + $xfer += $output->writeString($this->dbname); + $xfer += $output->writeFieldEnd(); + } + if ($this->tablename !== null) { + $xfer += $output->writeFieldBegin('tablename', TType::STRING, 4); + $xfer += $output->writeString($this->tablename); + $xfer += $output->writeFieldEnd(); + } + if ($this->partitionname !== null) { + $xfer += $output->writeFieldBegin('partitionname', TType::STRING, 5); + $xfer += $output->writeString($this->partitionname); + $xfer += $output->writeFieldEnd(); + } + $xfer += $output->writeFieldStop(); + $xfer += $output->writeStructEnd(); + return $xfer; + } + +} + +class LockRequest { + static $_TSPEC; + + public $component = null; + public $txnid = null; + public $user = null; + public $hostname = null; + + public function __construct($vals=null) { + if (!isset(self::$_TSPEC)) { + self::$_TSPEC = array( + 1 => array( + 'var' => 'component', + 'type' => TType::LST, + 'etype' => TType::STRUCT, + 'elem' => array( + 'type' => TType::STRUCT, + 'class' => '\metastore\LockComponent', + ), + ), + 2 => array( + 'var' => 'txnid', + 'type' => TType::I64, + ), + 3 => array( + 'var' => 'user', + 'type' => TType::STRING, + ), + 4 => array( + 'var' => 'hostname', + 'type' => TType::STRING, + ), + ); + } + if (is_array($vals)) { + if (isset($vals['component'])) { + $this->component = $vals['component']; + } + if (isset($vals['txnid'])) { + $this->txnid = $vals['txnid']; + } + if (isset($vals['user'])) { + $this->user = $vals['user']; + } + if (isset($vals['hostname'])) { + $this->hostname = $vals['hostname']; + } + } + } + + public function getName() { + return 'LockRequest'; + } + + public function read($input) + { + $xfer = 0; + $fname = null; + $ftype = 0; + $fid = 0; + $xfer += $input->readStructBegin($fname); + while (true) + { + $xfer += $input->readFieldBegin($fname, $ftype, $fid); + if ($ftype == TType::STOP) { + break; + } + switch ($fid) + { + case 1: + if ($ftype == TType::LST) { + $this->component = array(); + $_size342 = 0; + $_etype345 = 0; + $xfer += $input->readListBegin($_etype345, $_size342); + for ($_i346 = 0; $_i346 < $_size342; ++$_i346) + { + $elem347 = null; + $elem347 = new \metastore\LockComponent(); + $xfer += $elem347->read($input); + $this->component []= $elem347; + } + $xfer += $input->readListEnd(); + } else { + $xfer += $input->skip($ftype); + } + break; + case 2: + if ($ftype == TType::I64) { + $xfer += $input->readI64($this->txnid); + } else { + $xfer += $input->skip($ftype); + } + break; + case 3: + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->user); + } else { + $xfer += $input->skip($ftype); + } + break; + case 4: + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->hostname); + } else { + $xfer += $input->skip($ftype); + } + break; + default: + $xfer += $input->skip($ftype); + break; + } + $xfer += $input->readFieldEnd(); + } + $xfer += $input->readStructEnd(); + return $xfer; + } + + public function write($output) { + $xfer = 0; + $xfer += $output->writeStructBegin('LockRequest'); + if ($this->component !== null) { + if (!is_array($this->component)) { + throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); + } + $xfer += $output->writeFieldBegin('component', TType::LST, 1); + { + $output->writeListBegin(TType::STRUCT, count($this->component)); + { + foreach ($this->component as $iter348) + { + $xfer += $iter348->write($output); + } + } + $output->writeListEnd(); + } + $xfer += $output->writeFieldEnd(); + } + if ($this->txnid !== null) { + $xfer += $output->writeFieldBegin('txnid', TType::I64, 2); + $xfer += $output->writeI64($this->txnid); + $xfer += $output->writeFieldEnd(); + } + if ($this->user !== null) { + $xfer += $output->writeFieldBegin('user', TType::STRING, 3); + $xfer += $output->writeString($this->user); + $xfer += $output->writeFieldEnd(); + } + if ($this->hostname !== null) { + $xfer += $output->writeFieldBegin('hostname', TType::STRING, 4); + $xfer += $output->writeString($this->hostname); + $xfer += $output->writeFieldEnd(); + } + $xfer += $output->writeFieldStop(); + $xfer += $output->writeStructEnd(); + return $xfer; + } + +} + +class LockResponse { + static $_TSPEC; + + public $lockid = null; + public $state = null; + + public function __construct($vals=null) { + if (!isset(self::$_TSPEC)) { + self::$_TSPEC = array( + 1 => array( + 'var' => 'lockid', + 'type' => TType::I64, + ), + 2 => array( + 'var' => 'state', + 'type' => TType::I32, + ), + ); + } + if (is_array($vals)) { + if (isset($vals['lockid'])) { + $this->lockid = $vals['lockid']; + } + if (isset($vals['state'])) { + $this->state = $vals['state']; + } + } + } + + public function getName() { + return 'LockResponse'; + } + + public function read($input) + { + $xfer = 0; + $fname = null; + $ftype = 0; + $fid = 0; + $xfer += $input->readStructBegin($fname); + while (true) + { + $xfer += $input->readFieldBegin($fname, $ftype, $fid); + if ($ftype == TType::STOP) { + break; + } + switch ($fid) + { + case 1: + if ($ftype == TType::I64) { + $xfer += $input->readI64($this->lockid); + } else { + $xfer += $input->skip($ftype); + } + break; + case 2: + if ($ftype == TType::I32) { + $xfer += $input->readI32($this->state); + } else { + $xfer += $input->skip($ftype); + } + break; + default: + $xfer += $input->skip($ftype); + break; + } + $xfer += $input->readFieldEnd(); + } + $xfer += $input->readStructEnd(); + return $xfer; + } + + public function write($output) { + $xfer = 0; + $xfer += $output->writeStructBegin('LockResponse'); + if ($this->lockid !== null) { + $xfer += $output->writeFieldBegin('lockid', TType::I64, 1); + $xfer += $output->writeI64($this->lockid); + $xfer += $output->writeFieldEnd(); + } + if ($this->state !== null) { + $xfer += $output->writeFieldBegin('state', TType::I32, 2); + $xfer += $output->writeI32($this->state); + $xfer += $output->writeFieldEnd(); + } + $xfer += $output->writeFieldStop(); + $xfer += $output->writeStructEnd(); + return $xfer; + } + +} + +class CheckLockRequest { + static $_TSPEC; + + public $lockid = null; + + public function __construct($vals=null) { + if (!isset(self::$_TSPEC)) { + self::$_TSPEC = array( + 1 => array( + 'var' => 'lockid', + 'type' => TType::I64, + ), + ); + } + if (is_array($vals)) { + if (isset($vals['lockid'])) { + $this->lockid = $vals['lockid']; + } + } + } + + public function getName() { + return 'CheckLockRequest'; + } + + public function read($input) + { + $xfer = 0; + $fname = null; + $ftype = 0; + $fid = 0; + $xfer += $input->readStructBegin($fname); + while (true) + { + $xfer += $input->readFieldBegin($fname, $ftype, $fid); + if ($ftype == TType::STOP) { + break; + } + switch ($fid) + { + case 1: + if ($ftype == TType::I64) { + $xfer += $input->readI64($this->lockid); + } else { + $xfer += $input->skip($ftype); + } + break; + default: + $xfer += $input->skip($ftype); + break; + } + $xfer += $input->readFieldEnd(); + } + $xfer += $input->readStructEnd(); + return $xfer; + } + + public function write($output) { + $xfer = 0; + $xfer += $output->writeStructBegin('CheckLockRequest'); + if ($this->lockid !== null) { + $xfer += $output->writeFieldBegin('lockid', TType::I64, 1); + $xfer += $output->writeI64($this->lockid); + $xfer += $output->writeFieldEnd(); + } + $xfer += $output->writeFieldStop(); + $xfer += $output->writeStructEnd(); + return $xfer; + } + +} + +class UnlockRequest { + static $_TSPEC; + + public $lockid = null; + + public function __construct($vals=null) { + if (!isset(self::$_TSPEC)) { + self::$_TSPEC = array( + 1 => array( + 'var' => 'lockid', + 'type' => TType::I64, + ), + ); + } + if (is_array($vals)) { + if (isset($vals['lockid'])) { + $this->lockid = $vals['lockid']; + } + } + } + + public function getName() { + return 'UnlockRequest'; + } + + public function read($input) + { + $xfer = 0; + $fname = null; + $ftype = 0; + $fid = 0; + $xfer += $input->readStructBegin($fname); + while (true) + { + $xfer += $input->readFieldBegin($fname, $ftype, $fid); + if ($ftype == TType::STOP) { + break; + } + switch ($fid) + { + case 1: + if ($ftype == TType::I64) { + $xfer += $input->readI64($this->lockid); + } else { + $xfer += $input->skip($ftype); + } + break; + default: + $xfer += $input->skip($ftype); + break; + } + $xfer += $input->readFieldEnd(); + } + $xfer += $input->readStructEnd(); + return $xfer; + } + + public function write($output) { + $xfer = 0; + $xfer += $output->writeStructBegin('UnlockRequest'); + if ($this->lockid !== null) { + $xfer += $output->writeFieldBegin('lockid', TType::I64, 1); + $xfer += $output->writeI64($this->lockid); + $xfer += $output->writeFieldEnd(); + } + $xfer += $output->writeFieldStop(); + $xfer += $output->writeStructEnd(); + return $xfer; + } + +} + +class ShowLocksRequest { + static $_TSPEC; + + + public function __construct() { + if (!isset(self::$_TSPEC)) { + self::$_TSPEC = array( + ); + } + } + + public function getName() { + return 'ShowLocksRequest'; + } + + public function read($input) + { + $xfer = 0; + $fname = null; + $ftype = 0; + $fid = 0; + $xfer += $input->readStructBegin($fname); + while (true) + { + $xfer += $input->readFieldBegin($fname, $ftype, $fid); + if ($ftype == TType::STOP) { + break; + } + switch ($fid) + { + default: + $xfer += $input->skip($ftype); + break; + } + $xfer += $input->readFieldEnd(); + } + $xfer += $input->readStructEnd(); + return $xfer; + } + + public function write($output) { + $xfer = 0; + $xfer += $output->writeStructBegin('ShowLocksRequest'); + $xfer += $output->writeFieldStop(); + $xfer += $output->writeStructEnd(); + return $xfer; + } + +} + +class ShowLocksResponseElement { + static $_TSPEC; + + public $lockid = null; + public $dbname = null; + public $tablename = null; + public $partname = null; + public $state = null; + public $type = null; + public $txnid = null; + public $lastheartbeat = null; + public $acquiredat = null; + public $user = null; + public $hostname = null; + + public function __construct($vals=null) { + if (!isset(self::$_TSPEC)) { + self::$_TSPEC = array( + 1 => array( + 'var' => 'lockid', + 'type' => TType::I64, + ), + 2 => array( + 'var' => 'dbname', + 'type' => TType::STRING, + ), + 3 => array( + 'var' => 'tablename', + 'type' => TType::STRING, + ), + 4 => array( + 'var' => 'partname', + 'type' => TType::STRING, + ), + 5 => array( + 'var' => 'state', + 'type' => TType::I32, + ), + 6 => array( + 'var' => 'type', + 'type' => TType::I32, + ), + 7 => array( + 'var' => 'txnid', + 'type' => TType::I64, + ), + 8 => array( + 'var' => 'lastheartbeat', + 'type' => TType::I64, + ), + 9 => array( + 'var' => 'acquiredat', + 'type' => TType::I64, + ), + 10 => array( + 'var' => 'user', + 'type' => TType::STRING, + ), + 11 => array( + 'var' => 'hostname', + 'type' => TType::STRING, + ), + ); + } + if (is_array($vals)) { + if (isset($vals['lockid'])) { + $this->lockid = $vals['lockid']; + } + if (isset($vals['dbname'])) { + $this->dbname = $vals['dbname']; + } + if (isset($vals['tablename'])) { + $this->tablename = $vals['tablename']; + } + if (isset($vals['partname'])) { + $this->partname = $vals['partname']; + } + if (isset($vals['state'])) { + $this->state = $vals['state']; + } + if (isset($vals['type'])) { + $this->type = $vals['type']; + } + if (isset($vals['txnid'])) { + $this->txnid = $vals['txnid']; + } + if (isset($vals['lastheartbeat'])) { + $this->lastheartbeat = $vals['lastheartbeat']; + } + if (isset($vals['acquiredat'])) { + $this->acquiredat = $vals['acquiredat']; + } + if (isset($vals['user'])) { + $this->user = $vals['user']; + } + if (isset($vals['hostname'])) { + $this->hostname = $vals['hostname']; + } + } + } + + public function getName() { + return 'ShowLocksResponseElement'; + } + + public function read($input) + { + $xfer = 0; + $fname = null; + $ftype = 0; + $fid = 0; + $xfer += $input->readStructBegin($fname); + while (true) + { + $xfer += $input->readFieldBegin($fname, $ftype, $fid); + if ($ftype == TType::STOP) { + break; + } + switch ($fid) + { + case 1: + if ($ftype == TType::I64) { + $xfer += $input->readI64($this->lockid); + } else { + $xfer += $input->skip($ftype); + } + break; + case 2: + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->dbname); + } else { + $xfer += $input->skip($ftype); + } + break; + case 3: + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->tablename); + } else { + $xfer += $input->skip($ftype); + } + break; + case 4: + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->partname); + } else { + $xfer += $input->skip($ftype); + } + break; + case 5: + if ($ftype == TType::I32) { + $xfer += $input->readI32($this->state); + } else { + $xfer += $input->skip($ftype); + } + break; + case 6: + if ($ftype == TType::I32) { + $xfer += $input->readI32($this->type); + } else { + $xfer += $input->skip($ftype); + } + break; + case 7: + if ($ftype == TType::I64) { + $xfer += $input->readI64($this->txnid); + } else { + $xfer += $input->skip($ftype); + } + break; + case 8: + if ($ftype == TType::I64) { + $xfer += $input->readI64($this->lastheartbeat); + } else { + $xfer += $input->skip($ftype); + } + break; + case 9: + if ($ftype == TType::I64) { + $xfer += $input->readI64($this->acquiredat); + } else { + $xfer += $input->skip($ftype); + } + break; + case 10: + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->user); + } else { + $xfer += $input->skip($ftype); + } + break; + case 11: + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->hostname); + } else { + $xfer += $input->skip($ftype); + } + break; + default: + $xfer += $input->skip($ftype); + break; + } + $xfer += $input->readFieldEnd(); + } + $xfer += $input->readStructEnd(); + return $xfer; + } + + public function write($output) { + $xfer = 0; + $xfer += $output->writeStructBegin('ShowLocksResponseElement'); + if ($this->lockid !== null) { + $xfer += $output->writeFieldBegin('lockid', TType::I64, 1); + $xfer += $output->writeI64($this->lockid); + $xfer += $output->writeFieldEnd(); + } + if ($this->dbname !== null) { + $xfer += $output->writeFieldBegin('dbname', TType::STRING, 2); + $xfer += $output->writeString($this->dbname); + $xfer += $output->writeFieldEnd(); + } + if ($this->tablename !== null) { + $xfer += $output->writeFieldBegin('tablename', TType::STRING, 3); + $xfer += $output->writeString($this->tablename); + $xfer += $output->writeFieldEnd(); + } + if ($this->partname !== null) { + $xfer += $output->writeFieldBegin('partname', TType::STRING, 4); + $xfer += $output->writeString($this->partname); + $xfer += $output->writeFieldEnd(); + } + if ($this->state !== null) { + $xfer += $output->writeFieldBegin('state', TType::I32, 5); + $xfer += $output->writeI32($this->state); + $xfer += $output->writeFieldEnd(); + } + if ($this->type !== null) { + $xfer += $output->writeFieldBegin('type', TType::I32, 6); + $xfer += $output->writeI32($this->type); + $xfer += $output->writeFieldEnd(); + } + if ($this->txnid !== null) { + $xfer += $output->writeFieldBegin('txnid', TType::I64, 7); + $xfer += $output->writeI64($this->txnid); + $xfer += $output->writeFieldEnd(); + } + if ($this->lastheartbeat !== null) { + $xfer += $output->writeFieldBegin('lastheartbeat', TType::I64, 8); + $xfer += $output->writeI64($this->lastheartbeat); + $xfer += $output->writeFieldEnd(); + } + if ($this->acquiredat !== null) { + $xfer += $output->writeFieldBegin('acquiredat', TType::I64, 9); + $xfer += $output->writeI64($this->acquiredat); + $xfer += $output->writeFieldEnd(); + } + if ($this->user !== null) { + $xfer += $output->writeFieldBegin('user', TType::STRING, 10); + $xfer += $output->writeString($this->user); + $xfer += $output->writeFieldEnd(); + } + if ($this->hostname !== null) { + $xfer += $output->writeFieldBegin('hostname', TType::STRING, 11); + $xfer += $output->writeString($this->hostname); + $xfer += $output->writeFieldEnd(); + } + $xfer += $output->writeFieldStop(); + $xfer += $output->writeStructEnd(); + return $xfer; + } + +} + +class ShowLocksResponse { + static $_TSPEC; + + public $locks = null; + + public function __construct($vals=null) { + if (!isset(self::$_TSPEC)) { + self::$_TSPEC = array( + 1 => array( + 'var' => 'locks', + 'type' => TType::LST, + 'etype' => TType::STRUCT, + 'elem' => array( + 'type' => TType::STRUCT, + 'class' => '\metastore\ShowLocksResponseElement', + ), + ), + ); + } + if (is_array($vals)) { + if (isset($vals['locks'])) { + $this->locks = $vals['locks']; + } + } + } + + public function getName() { + return 'ShowLocksResponse'; + } + + public function read($input) + { + $xfer = 0; + $fname = null; + $ftype = 0; + $fid = 0; + $xfer += $input->readStructBegin($fname); + while (true) + { + $xfer += $input->readFieldBegin($fname, $ftype, $fid); + if ($ftype == TType::STOP) { + break; + } + switch ($fid) + { + case 1: + if ($ftype == TType::LST) { + $this->locks = array(); + $_size349 = 0; + $_etype352 = 0; + $xfer += $input->readListBegin($_etype352, $_size349); + for ($_i353 = 0; $_i353 < $_size349; ++$_i353) + { + $elem354 = null; + $elem354 = new \metastore\ShowLocksResponseElement(); + $xfer += $elem354->read($input); + $this->locks []= $elem354; + } + $xfer += $input->readListEnd(); + } else { + $xfer += $input->skip($ftype); + } + break; + default: + $xfer += $input->skip($ftype); + break; + } + $xfer += $input->readFieldEnd(); + } + $xfer += $input->readStructEnd(); + return $xfer; + } + + public function write($output) { + $xfer = 0; + $xfer += $output->writeStructBegin('ShowLocksResponse'); + if ($this->locks !== null) { + if (!is_array($this->locks)) { + throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); + } + $xfer += $output->writeFieldBegin('locks', TType::LST, 1); + { + $output->writeListBegin(TType::STRUCT, count($this->locks)); + { + foreach ($this->locks as $iter355) + { + $xfer += $iter355->write($output); + } + } + $output->writeListEnd(); + } + $xfer += $output->writeFieldEnd(); + } + $xfer += $output->writeFieldStop(); + $xfer += $output->writeStructEnd(); + return $xfer; + } + +} + +class HeartbeatRequest { + static $_TSPEC; + + public $lockid = null; + public $txnid = null; + + public function __construct($vals=null) { + if (!isset(self::$_TSPEC)) { + self::$_TSPEC = array( + 1 => array( + 'var' => 'lockid', + 'type' => TType::I64, + ), + 2 => array( + 'var' => 'txnid', + 'type' => TType::I64, + ), + ); + } + if (is_array($vals)) { + if (isset($vals['lockid'])) { + $this->lockid = $vals['lockid']; + } + if (isset($vals['txnid'])) { + $this->txnid = $vals['txnid']; + } + } + } + + public function getName() { + return 'HeartbeatRequest'; + } + + public function read($input) + { + $xfer = 0; + $fname = null; + $ftype = 0; + $fid = 0; + $xfer += $input->readStructBegin($fname); + while (true) + { + $xfer += $input->readFieldBegin($fname, $ftype, $fid); + if ($ftype == TType::STOP) { + break; + } + switch ($fid) + { + case 1: + if ($ftype == TType::I64) { + $xfer += $input->readI64($this->lockid); + } else { + $xfer += $input->skip($ftype); + } + break; + case 2: + if ($ftype == TType::I64) { + $xfer += $input->readI64($this->txnid); + } else { + $xfer += $input->skip($ftype); + } + break; + default: + $xfer += $input->skip($ftype); + break; + } + $xfer += $input->readFieldEnd(); + } + $xfer += $input->readStructEnd(); + return $xfer; + } + + public function write($output) { + $xfer = 0; + $xfer += $output->writeStructBegin('HeartbeatRequest'); + if ($this->lockid !== null) { + $xfer += $output->writeFieldBegin('lockid', TType::I64, 1); + $xfer += $output->writeI64($this->lockid); + $xfer += $output->writeFieldEnd(); + } + if ($this->txnid !== null) { + $xfer += $output->writeFieldBegin('txnid', TType::I64, 2); + $xfer += $output->writeI64($this->txnid); + $xfer += $output->writeFieldEnd(); + } + $xfer += $output->writeFieldStop(); + $xfer += $output->writeStructEnd(); + return $xfer; + } + +} + +class CompactionRequest { + static $_TSPEC; + + public $dbname = null; + public $tablename = null; + public $partitionname = null; + public $type = null; + public $runas = null; + + public function __construct($vals=null) { + if (!isset(self::$_TSPEC)) { + self::$_TSPEC = array( + 1 => array( + 'var' => 'dbname', + 'type' => TType::STRING, + ), + 2 => array( + 'var' => 'tablename', + 'type' => TType::STRING, + ), + 3 => array( + 'var' => 'partitionname', + 'type' => TType::STRING, + ), + 4 => array( + 'var' => 'type', + 'type' => TType::I32, + ), + 5 => array( + 'var' => 'runas', + 'type' => TType::STRING, + ), + ); + } + if (is_array($vals)) { + if (isset($vals['dbname'])) { + $this->dbname = $vals['dbname']; + } + if (isset($vals['tablename'])) { + $this->tablename = $vals['tablename']; + } + if (isset($vals['partitionname'])) { + $this->partitionname = $vals['partitionname']; + } + if (isset($vals['type'])) { + $this->type = $vals['type']; + } + if (isset($vals['runas'])) { + $this->runas = $vals['runas']; + } + } + } + + public function getName() { + return 'CompactionRequest'; + } + + public function read($input) + { + $xfer = 0; + $fname = null; + $ftype = 0; + $fid = 0; + $xfer += $input->readStructBegin($fname); + while (true) + { + $xfer += $input->readFieldBegin($fname, $ftype, $fid); + if ($ftype == TType::STOP) { + break; + } + switch ($fid) + { + case 1: + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->dbname); + } else { + $xfer += $input->skip($ftype); + } + break; + case 2: + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->tablename); + } else { + $xfer += $input->skip($ftype); + } + break; + case 3: + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->partitionname); + } else { + $xfer += $input->skip($ftype); + } + break; + case 4: + if ($ftype == TType::I32) { + $xfer += $input->readI32($this->type); + } else { + $xfer += $input->skip($ftype); + } + break; + case 5: + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->runas); + } else { + $xfer += $input->skip($ftype); + } + break; + default: + $xfer += $input->skip($ftype); + break; + } + $xfer += $input->readFieldEnd(); + } + $xfer += $input->readStructEnd(); + return $xfer; + } + + public function write($output) { + $xfer = 0; + $xfer += $output->writeStructBegin('CompactionRequest'); + if ($this->dbname !== null) { + $xfer += $output->writeFieldBegin('dbname', TType::STRING, 1); + $xfer += $output->writeString($this->dbname); + $xfer += $output->writeFieldEnd(); + } + if ($this->tablename !== null) { + $xfer += $output->writeFieldBegin('tablename', TType::STRING, 2); + $xfer += $output->writeString($this->tablename); + $xfer += $output->writeFieldEnd(); + } + if ($this->partitionname !== null) { + $xfer += $output->writeFieldBegin('partitionname', TType::STRING, 3); + $xfer += $output->writeString($this->partitionname); + $xfer += $output->writeFieldEnd(); + } + if ($this->type !== null) { + $xfer += $output->writeFieldBegin('type', TType::I32, 4); + $xfer += $output->writeI32($this->type); + $xfer += $output->writeFieldEnd(); + } + if ($this->runas !== null) { + $xfer += $output->writeFieldBegin('runas', TType::STRING, 5); + $xfer += $output->writeString($this->runas); + $xfer += $output->writeFieldEnd(); + } + $xfer += $output->writeFieldStop(); + $xfer += $output->writeStructEnd(); + return $xfer; + } + +} + +class ShowCompactRequest { + static $_TSPEC; + + + public function __construct() { + if (!isset(self::$_TSPEC)) { + self::$_TSPEC = array( + ); + } + } + + public function getName() { + return 'ShowCompactRequest'; + } + + public function read($input) + { + $xfer = 0; + $fname = null; + $ftype = 0; + $fid = 0; + $xfer += $input->readStructBegin($fname); + while (true) + { + $xfer += $input->readFieldBegin($fname, $ftype, $fid); + if ($ftype == TType::STOP) { + break; + } + switch ($fid) + { + default: + $xfer += $input->skip($ftype); + break; + } + $xfer += $input->readFieldEnd(); + } + $xfer += $input->readStructEnd(); + return $xfer; + } + + public function write($output) { + $xfer = 0; + $xfer += $output->writeStructBegin('ShowCompactRequest'); + $xfer += $output->writeFieldStop(); + $xfer += $output->writeStructEnd(); + return $xfer; + } + +} + +class ShowCompactResponseElement { + static $_TSPEC; + + public $dbname = null; + public $tablename = null; + public $partitionname = null; + public $type = null; + public $state = null; + public $workerid = null; + public $start = null; + public $runAs = null; + + public function __construct($vals=null) { + if (!isset(self::$_TSPEC)) { + self::$_TSPEC = array( + 1 => array( + 'var' => 'dbname', + 'type' => TType::STRING, + ), + 2 => array( + 'var' => 'tablename', + 'type' => TType::STRING, + ), + 3 => array( + 'var' => 'partitionname', + 'type' => TType::STRING, + ), + 4 => array( + 'var' => 'type', + 'type' => TType::I32, + ), + 5 => array( + 'var' => 'state', + 'type' => TType::STRING, + ), + 6 => array( + 'var' => 'workerid', + 'type' => TType::STRING, + ), + 7 => array( + 'var' => 'start', + 'type' => TType::I64, + ), + 8 => array( + 'var' => 'runAs', + 'type' => TType::STRING, + ), + ); + } + if (is_array($vals)) { + if (isset($vals['dbname'])) { + $this->dbname = $vals['dbname']; + } + if (isset($vals['tablename'])) { + $this->tablename = $vals['tablename']; + } + if (isset($vals['partitionname'])) { + $this->partitionname = $vals['partitionname']; + } + if (isset($vals['type'])) { + $this->type = $vals['type']; + } + if (isset($vals['state'])) { + $this->state = $vals['state']; + } + if (isset($vals['workerid'])) { + $this->workerid = $vals['workerid']; + } + if (isset($vals['start'])) { + $this->start = $vals['start']; + } + if (isset($vals['runAs'])) { + $this->runAs = $vals['runAs']; + } + } + } + + public function getName() { + return 'ShowCompactResponseElement'; + } + + public function read($input) + { + $xfer = 0; + $fname = null; + $ftype = 0; + $fid = 0; + $xfer += $input->readStructBegin($fname); + while (true) + { + $xfer += $input->readFieldBegin($fname, $ftype, $fid); + if ($ftype == TType::STOP) { + break; + } + switch ($fid) + { + case 1: + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->dbname); + } else { + $xfer += $input->skip($ftype); + } + break; + case 2: + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->tablename); + } else { + $xfer += $input->skip($ftype); + } + break; + case 3: + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->partitionname); + } else { + $xfer += $input->skip($ftype); + } + break; + case 4: + if ($ftype == TType::I32) { + $xfer += $input->readI32($this->type); + } else { + $xfer += $input->skip($ftype); + } + break; + case 5: + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->state); + } else { + $xfer += $input->skip($ftype); + } + break; + case 6: + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->workerid); + } else { + $xfer += $input->skip($ftype); + } + break; + case 7: + if ($ftype == TType::I64) { + $xfer += $input->readI64($this->start); + } else { + $xfer += $input->skip($ftype); + } + break; + case 8: + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->runAs); + } else { + $xfer += $input->skip($ftype); + } + break; + default: + $xfer += $input->skip($ftype); + break; + } + $xfer += $input->readFieldEnd(); + } + $xfer += $input->readStructEnd(); + return $xfer; + } + + public function write($output) { + $xfer = 0; + $xfer += $output->writeStructBegin('ShowCompactResponseElement'); + if ($this->dbname !== null) { + $xfer += $output->writeFieldBegin('dbname', TType::STRING, 1); + $xfer += $output->writeString($this->dbname); + $xfer += $output->writeFieldEnd(); + } + if ($this->tablename !== null) { + $xfer += $output->writeFieldBegin('tablename', TType::STRING, 2); + $xfer += $output->writeString($this->tablename); + $xfer += $output->writeFieldEnd(); + } + if ($this->partitionname !== null) { + $xfer += $output->writeFieldBegin('partitionname', TType::STRING, 3); + $xfer += $output->writeString($this->partitionname); + $xfer += $output->writeFieldEnd(); + } + if ($this->type !== null) { + $xfer += $output->writeFieldBegin('type', TType::I32, 4); + $xfer += $output->writeI32($this->type); + $xfer += $output->writeFieldEnd(); + } + if ($this->state !== null) { + $xfer += $output->writeFieldBegin('state', TType::STRING, 5); + $xfer += $output->writeString($this->state); + $xfer += $output->writeFieldEnd(); + } + if ($this->workerid !== null) { + $xfer += $output->writeFieldBegin('workerid', TType::STRING, 6); + $xfer += $output->writeString($this->workerid); + $xfer += $output->writeFieldEnd(); + } + if ($this->start !== null) { + $xfer += $output->writeFieldBegin('start', TType::I64, 7); + $xfer += $output->writeI64($this->start); + $xfer += $output->writeFieldEnd(); + } + if ($this->runAs !== null) { + $xfer += $output->writeFieldBegin('runAs', TType::STRING, 8); + $xfer += $output->writeString($this->runAs); + $xfer += $output->writeFieldEnd(); + } + $xfer += $output->writeFieldStop(); + $xfer += $output->writeStructEnd(); + return $xfer; + } + +} + +class ShowCompactResponse { + static $_TSPEC; + + public $compacts = null; + + public function __construct($vals=null) { + if (!isset(self::$_TSPEC)) { + self::$_TSPEC = array( + 1 => array( + 'var' => 'compacts', + 'type' => TType::LST, + 'etype' => TType::STRUCT, + 'elem' => array( + 'type' => TType::STRUCT, + 'class' => '\metastore\ShowCompactResponseElement', + ), + ), + ); + } + if (is_array($vals)) { + if (isset($vals['compacts'])) { + $this->compacts = $vals['compacts']; + } + } + } + + public function getName() { + return 'ShowCompactResponse'; + } + + public function read($input) + { + $xfer = 0; + $fname = null; + $ftype = 0; + $fid = 0; + $xfer += $input->readStructBegin($fname); + while (true) + { + $xfer += $input->readFieldBegin($fname, $ftype, $fid); + if ($ftype == TType::STOP) { + break; + } + switch ($fid) + { + case 1: + if ($ftype == TType::LST) { + $this->compacts = array(); + $_size356 = 0; + $_etype359 = 0; + $xfer += $input->readListBegin($_etype359, $_size356); + for ($_i360 = 0; $_i360 < $_size356; ++$_i360) + { + $elem361 = null; + $elem361 = new \metastore\ShowCompactResponseElement(); + $xfer += $elem361->read($input); + $this->compacts []= $elem361; + } + $xfer += $input->readListEnd(); + } else { + $xfer += $input->skip($ftype); + } + break; + default: + $xfer += $input->skip($ftype); + break; + } + $xfer += $input->readFieldEnd(); + } + $xfer += $input->readStructEnd(); + return $xfer; + } + + public function write($output) { + $xfer = 0; + $xfer += $output->writeStructBegin('ShowCompactResponse'); + if ($this->compacts !== null) { + if (!is_array($this->compacts)) { + throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA); + } + $xfer += $output->writeFieldBegin('compacts', TType::LST, 1); + { + $output->writeListBegin(TType::STRUCT, count($this->compacts)); + { + foreach ($this->compacts as $iter362) + { + $xfer += $iter362->write($output); + } + } + $output->writeListEnd(); + } + $xfer += $output->writeFieldEnd(); + } + $xfer += $output->writeFieldStop(); + $xfer += $output->writeStructEnd(); + return $xfer; + } + +} + +class MetaException extends TException { + static $_TSPEC; + + public $message = null; + + public function __construct($vals=null) { + if (!isset(self::$_TSPEC)) { + self::$_TSPEC = array( + 1 => array( + 'var' => 'message', + 'type' => TType::STRING, + ), + ); + } + if (is_array($vals)) { + if (isset($vals['message'])) { + $this->message = $vals['message']; + } + } + } + + public function getName() { + return 'MetaException'; + } + + public function read($input) + { + $xfer = 0; + $fname = null; + $ftype = 0; + $fid = 0; + $xfer += $input->readStructBegin($fname); + while (true) + { + $xfer += $input->readFieldBegin($fname, $ftype, $fid); + if ($ftype == TType::STOP) { + break; + } + switch ($fid) + { + case 1: + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->message); + } else { + $xfer += $input->skip($ftype); + } + break; + default: + $xfer += $input->skip($ftype); + break; + } + $xfer += $input->readFieldEnd(); + } + $xfer += $input->readStructEnd(); + return $xfer; + } + + public function write($output) { + $xfer = 0; + $xfer += $output->writeStructBegin('MetaException'); + if ($this->message !== null) { + $xfer += $output->writeFieldBegin('message', TType::STRING, 1); + $xfer += $output->writeString($this->message); + $xfer += $output->writeFieldEnd(); + } + $xfer += $output->writeFieldStop(); + $xfer += $output->writeStructEnd(); + return $xfer; + } + +} + +class UnknownTableException extends TException { + static $_TSPEC; + + public $message = null; + + public function __construct($vals=null) { + if (!isset(self::$_TSPEC)) { + self::$_TSPEC = array( + 1 => array( + 'var' => 'message', + 'type' => TType::STRING, + ), + ); + } + if (is_array($vals)) { + if (isset($vals['message'])) { + $this->message = $vals['message']; + } + } + } + + public function getName() { + return 'UnknownTableException'; + } + + public function read($input) + { + $xfer = 0; + $fname = null; + $ftype = 0; + $fid = 0; + $xfer += $input->readStructBegin($fname); + while (true) + { + $xfer += $input->readFieldBegin($fname, $ftype, $fid); + if ($ftype == TType::STOP) { + break; + } + switch ($fid) + { + case 1: + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->message); + } else { + $xfer += $input->skip($ftype); + } + break; + default: + $xfer += $input->skip($ftype); + break; + } + $xfer += $input->readFieldEnd(); + } + $xfer += $input->readStructEnd(); + return $xfer; + } + + public function write($output) { + $xfer = 0; + $xfer += $output->writeStructBegin('UnknownTableException'); + if ($this->message !== null) { + $xfer += $output->writeFieldBegin('message', TType::STRING, 1); + $xfer += $output->writeString($this->message); + $xfer += $output->writeFieldEnd(); + } + $xfer += $output->writeFieldStop(); + $xfer += $output->writeStructEnd(); + return $xfer; + } + +} + +class UnknownDBException extends TException { + static $_TSPEC; + + public $message = null; + + public function __construct($vals=null) { + if (!isset(self::$_TSPEC)) { + self::$_TSPEC = array( + 1 => array( + 'var' => 'message', + 'type' => TType::STRING, + ), + ); + } + if (is_array($vals)) { + if (isset($vals['message'])) { + $this->message = $vals['message']; + } + } + } + + public function getName() { + return 'UnknownDBException'; + } + + public function read($input) + { + $xfer = 0; + $fname = null; + $ftype = 0; + $fid = 0; + $xfer += $input->readStructBegin($fname); + while (true) + { + $xfer += $input->readFieldBegin($fname, $ftype, $fid); + if ($ftype == TType::STOP) { + break; + } + switch ($fid) + { + case 1: + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->message); + } else { + $xfer += $input->skip($ftype); + } + break; + default: + $xfer += $input->skip($ftype); + break; + } + $xfer += $input->readFieldEnd(); + } + $xfer += $input->readStructEnd(); + return $xfer; + } + + public function write($output) { + $xfer = 0; + $xfer += $output->writeStructBegin('UnknownDBException'); + if ($this->message !== null) { + $xfer += $output->writeFieldBegin('message', TType::STRING, 1); + $xfer += $output->writeString($this->message); + $xfer += $output->writeFieldEnd(); + } + $xfer += $output->writeFieldStop(); + $xfer += $output->writeStructEnd(); + return $xfer; + } + +} + +class AlreadyExistsException extends TException { + static $_TSPEC; + + public $message = null; + + public function __construct($vals=null) { + if (!isset(self::$_TSPEC)) { + self::$_TSPEC = array( + 1 => array( + 'var' => 'message', + 'type' => TType::STRING, + ), + ); + } + if (is_array($vals)) { + if (isset($vals['message'])) { + $this->message = $vals['message']; + } + } + } + + public function getName() { + return 'AlreadyExistsException'; + } + + public function read($input) + { + $xfer = 0; + $fname = null; + $ftype = 0; + $fid = 0; + $xfer += $input->readStructBegin($fname); + while (true) + { + $xfer += $input->readFieldBegin($fname, $ftype, $fid); + if ($ftype == TType::STOP) { + break; + } + switch ($fid) + { + case 1: + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->message); + } else { + $xfer += $input->skip($ftype); + } + break; + default: + $xfer += $input->skip($ftype); + break; + } + $xfer += $input->readFieldEnd(); + } + $xfer += $input->readStructEnd(); + return $xfer; + } + + public function write($output) { + $xfer = 0; + $xfer += $output->writeStructBegin('AlreadyExistsException'); + if ($this->message !== null) { + $xfer += $output->writeFieldBegin('message', TType::STRING, 1); + $xfer += $output->writeString($this->message); + $xfer += $output->writeFieldEnd(); + } + $xfer += $output->writeFieldStop(); + $xfer += $output->writeStructEnd(); + return $xfer; + } + +} + +class InvalidPartitionException extends TException { + static $_TSPEC; + + public $message = null; + + public function __construct($vals=null) { + if (!isset(self::$_TSPEC)) { + self::$_TSPEC = array( + 1 => array( + 'var' => 'message', + 'type' => TType::STRING, + ), + ); + } + if (is_array($vals)) { + if (isset($vals['message'])) { + $this->message = $vals['message']; + } + } + } + + public function getName() { + return 'InvalidPartitionException'; + } + + public function read($input) + { + $xfer = 0; + $fname = null; + $ftype = 0; + $fid = 0; + $xfer += $input->readStructBegin($fname); + while (true) + { + $xfer += $input->readFieldBegin($fname, $ftype, $fid); + if ($ftype == TType::STOP) { + break; + } + switch ($fid) + { + case 1: + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->message); + } else { + $xfer += $input->skip($ftype); + } + break; + default: + $xfer += $input->skip($ftype); + break; + } + $xfer += $input->readFieldEnd(); + } + $xfer += $input->readStructEnd(); + return $xfer; + } + + public function write($output) { + $xfer = 0; + $xfer += $output->writeStructBegin('InvalidPartitionException'); + if ($this->message !== null) { + $xfer += $output->writeFieldBegin('message', TType::STRING, 1); + $xfer += $output->writeString($this->message); + $xfer += $output->writeFieldEnd(); + } + $xfer += $output->writeFieldStop(); + $xfer += $output->writeStructEnd(); + return $xfer; + } + +} + +class UnknownPartitionException extends TException { + static $_TSPEC; + + public $message = null; + + public function __construct($vals=null) { + if (!isset(self::$_TSPEC)) { + self::$_TSPEC = array( + 1 => array( + 'var' => 'message', + 'type' => TType::STRING, + ), + ); + } + if (is_array($vals)) { + if (isset($vals['message'])) { + $this->message = $vals['message']; + } + } + } + + public function getName() { + return 'UnknownPartitionException'; + } + + public function read($input) + { + $xfer = 0; + $fname = null; + $ftype = 0; + $fid = 0; + $xfer += $input->readStructBegin($fname); + while (true) + { + $xfer += $input->readFieldBegin($fname, $ftype, $fid); + if ($ftype == TType::STOP) { + break; + } + switch ($fid) + { + case 1: + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->message); + } else { + $xfer += $input->skip($ftype); + } + break; + default: + $xfer += $input->skip($ftype); + break; + } + $xfer += $input->readFieldEnd(); + } + $xfer += $input->readStructEnd(); + return $xfer; + } + + public function write($output) { + $xfer = 0; + $xfer += $output->writeStructBegin('UnknownPartitionException'); + if ($this->message !== null) { + $xfer += $output->writeFieldBegin('message', TType::STRING, 1); + $xfer += $output->writeString($this->message); + $xfer += $output->writeFieldEnd(); + } + $xfer += $output->writeFieldStop(); + $xfer += $output->writeStructEnd(); + return $xfer; + } + +} + +class InvalidObjectException extends TException { + static $_TSPEC; + + public $message = null; + + public function __construct($vals=null) { + if (!isset(self::$_TSPEC)) { + self::$_TSPEC = array( + 1 => array( + 'var' => 'message', + 'type' => TType::STRING, + ), + ); + } + if (is_array($vals)) { + if (isset($vals['message'])) { + $this->message = $vals['message']; + } + } + } + + public function getName() { + return 'InvalidObjectException'; + } + + public function read($input) + { + $xfer = 0; + $fname = null; + $ftype = 0; + $fid = 0; + $xfer += $input->readStructBegin($fname); + while (true) + { + $xfer += $input->readFieldBegin($fname, $ftype, $fid); + if ($ftype == TType::STOP) { + break; + } + switch ($fid) + { + case 1: + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->message); + } else { + $xfer += $input->skip($ftype); + } + break; + default: + $xfer += $input->skip($ftype); + break; + } + $xfer += $input->readFieldEnd(); + } + $xfer += $input->readStructEnd(); + return $xfer; + } + + public function write($output) { + $xfer = 0; + $xfer += $output->writeStructBegin('InvalidObjectException'); + if ($this->message !== null) { + $xfer += $output->writeFieldBegin('message', TType::STRING, 1); + $xfer += $output->writeString($this->message); + $xfer += $output->writeFieldEnd(); + } + $xfer += $output->writeFieldStop(); + $xfer += $output->writeStructEnd(); + return $xfer; + } + +} + +class NoSuchObjectException extends TException { + static $_TSPEC; + + public $message = null; + + public function __construct($vals=null) { + if (!isset(self::$_TSPEC)) { + self::$_TSPEC = array( + 1 => array( + 'var' => 'message', + 'type' => TType::STRING, + ), + ); + } + if (is_array($vals)) { + if (isset($vals['message'])) { + $this->message = $vals['message']; + } + } + } + + public function getName() { + return 'NoSuchObjectException'; + } + + public function read($input) + { + $xfer = 0; + $fname = null; + $ftype = 0; + $fid = 0; + $xfer += $input->readStructBegin($fname); + while (true) + { + $xfer += $input->readFieldBegin($fname, $ftype, $fid); + if ($ftype == TType::STOP) { + break; + } + switch ($fid) + { + case 1: + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->message); + } else { + $xfer += $input->skip($ftype); + } + break; + default: + $xfer += $input->skip($ftype); + break; + } + $xfer += $input->readFieldEnd(); + } + $xfer += $input->readStructEnd(); + return $xfer; + } + + public function write($output) { + $xfer = 0; + $xfer += $output->writeStructBegin('NoSuchObjectException'); + if ($this->message !== null) { + $xfer += $output->writeFieldBegin('message', TType::STRING, 1); + $xfer += $output->writeString($this->message); + $xfer += $output->writeFieldEnd(); + } + $xfer += $output->writeFieldStop(); + $xfer += $output->writeStructEnd(); + return $xfer; + } + +} + +class IndexAlreadyExistsException extends TException { + static $_TSPEC; + + public $message = null; + + public function __construct($vals=null) { + if (!isset(self::$_TSPEC)) { + self::$_TSPEC = array( + 1 => array( + 'var' => 'message', + 'type' => TType::STRING, + ), + ); + } if (is_array($vals)) { if (isset($vals['message'])) { $this->message = $vals['message']; @@ -7566,7 +10001,7 @@ class NoSuchObjectException extends TException { } public function getName() { - return 'NoSuchObjectException'; + return 'IndexAlreadyExistsException'; } public function read($input) @@ -7603,7 +10038,7 @@ class NoSuchObjectException extends TException { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('NoSuchObjectException'); + $xfer += $output->writeStructBegin('IndexAlreadyExistsException'); if ($this->message !== null) { $xfer += $output->writeFieldBegin('message', TType::STRING, 1); $xfer += $output->writeString($this->message); @@ -7616,7 +10051,7 @@ class NoSuchObjectException extends TException { } -class IndexAlreadyExistsException extends TException { +class InvalidOperationException extends TException { static $_TSPEC; public $message = null; @@ -7638,7 +10073,7 @@ class IndexAlreadyExistsException extends TException { } public function getName() { - return 'IndexAlreadyExistsException'; + return 'InvalidOperationException'; } public function read($input) @@ -7675,7 +10110,7 @@ class IndexAlreadyExistsException extends TException { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('IndexAlreadyExistsException'); + $xfer += $output->writeStructBegin('InvalidOperationException'); if ($this->message !== null) { $xfer += $output->writeFieldBegin('message', TType::STRING, 1); $xfer += $output->writeString($this->message); @@ -7688,7 +10123,7 @@ class IndexAlreadyExistsException extends TException { } -class InvalidOperationException extends TException { +class ConfigValSecurityException extends TException { static $_TSPEC; public $message = null; @@ -7710,7 +10145,7 @@ class InvalidOperationException extends TException { } public function getName() { - return 'InvalidOperationException'; + return 'ConfigValSecurityException'; } public function read($input) @@ -7747,7 +10182,7 @@ class InvalidOperationException extends TException { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('InvalidOperationException'); + $xfer += $output->writeStructBegin('ConfigValSecurityException'); if ($this->message !== null) { $xfer += $output->writeFieldBegin('message', TType::STRING, 1); $xfer += $output->writeString($this->message); @@ -7760,7 +10195,7 @@ class InvalidOperationException extends TException { } -class ConfigValSecurityException extends TException { +class InvalidInputException extends TException { static $_TSPEC; public $message = null; @@ -7782,7 +10217,7 @@ class ConfigValSecurityException extends TException { } public function getName() { - return 'ConfigValSecurityException'; + return 'InvalidInputException'; } public function read($input) @@ -7819,7 +10254,7 @@ class ConfigValSecurityException extends TException { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('ConfigValSecurityException'); + $xfer += $output->writeStructBegin('InvalidInputException'); if ($this->message !== null) { $xfer += $output->writeFieldBegin('message', TType::STRING, 1); $xfer += $output->writeString($this->message); @@ -7832,7 +10267,7 @@ class ConfigValSecurityException extends TException { } -class InvalidInputException extends TException { +class NoSuchTxnException extends TException { static $_TSPEC; public $message = null; @@ -7854,7 +10289,7 @@ class InvalidInputException extends TException { } public function getName() { - return 'InvalidInputException'; + return 'NoSuchTxnException'; } public function read($input) @@ -7891,7 +10326,223 @@ class InvalidInputException extends TException { public function write($output) { $xfer = 0; - $xfer += $output->writeStructBegin('InvalidInputException'); + $xfer += $output->writeStructBegin('NoSuchTxnException'); + if ($this->message !== null) { + $xfer += $output->writeFieldBegin('message', TType::STRING, 1); + $xfer += $output->writeString($this->message); + $xfer += $output->writeFieldEnd(); + } + $xfer += $output->writeFieldStop(); + $xfer += $output->writeStructEnd(); + return $xfer; + } + +} + +class TxnAbortedException extends TException { + static $_TSPEC; + + public $message = null; + + public function __construct($vals=null) { + if (!isset(self::$_TSPEC)) { + self::$_TSPEC = array( + 1 => array( + 'var' => 'message', + 'type' => TType::STRING, + ), + ); + } + if (is_array($vals)) { + if (isset($vals['message'])) { + $this->message = $vals['message']; + } + } + } + + public function getName() { + return 'TxnAbortedException'; + } + + public function read($input) + { + $xfer = 0; + $fname = null; + $ftype = 0; + $fid = 0; + $xfer += $input->readStructBegin($fname); + while (true) + { + $xfer += $input->readFieldBegin($fname, $ftype, $fid); + if ($ftype == TType::STOP) { + break; + } + switch ($fid) + { + case 1: + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->message); + } else { + $xfer += $input->skip($ftype); + } + break; + default: + $xfer += $input->skip($ftype); + break; + } + $xfer += $input->readFieldEnd(); + } + $xfer += $input->readStructEnd(); + return $xfer; + } + + public function write($output) { + $xfer = 0; + $xfer += $output->writeStructBegin('TxnAbortedException'); + if ($this->message !== null) { + $xfer += $output->writeFieldBegin('message', TType::STRING, 1); + $xfer += $output->writeString($this->message); + $xfer += $output->writeFieldEnd(); + } + $xfer += $output->writeFieldStop(); + $xfer += $output->writeStructEnd(); + return $xfer; + } + +} + +class TxnOpenException extends TException { + static $_TSPEC; + + public $message = null; + + public function __construct($vals=null) { + if (!isset(self::$_TSPEC)) { + self::$_TSPEC = array( + 1 => array( + 'var' => 'message', + 'type' => TType::STRING, + ), + ); + } + if (is_array($vals)) { + if (isset($vals['message'])) { + $this->message = $vals['message']; + } + } + } + + public function getName() { + return 'TxnOpenException'; + } + + public function read($input) + { + $xfer = 0; + $fname = null; + $ftype = 0; + $fid = 0; + $xfer += $input->readStructBegin($fname); + while (true) + { + $xfer += $input->readFieldBegin($fname, $ftype, $fid); + if ($ftype == TType::STOP) { + break; + } + switch ($fid) + { + case 1: + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->message); + } else { + $xfer += $input->skip($ftype); + } + break; + default: + $xfer += $input->skip($ftype); + break; + } + $xfer += $input->readFieldEnd(); + } + $xfer += $input->readStructEnd(); + return $xfer; + } + + public function write($output) { + $xfer = 0; + $xfer += $output->writeStructBegin('TxnOpenException'); + if ($this->message !== null) { + $xfer += $output->writeFieldBegin('message', TType::STRING, 1); + $xfer += $output->writeString($this->message); + $xfer += $output->writeFieldEnd(); + } + $xfer += $output->writeFieldStop(); + $xfer += $output->writeStructEnd(); + return $xfer; + } + +} + +class NoSuchLockException extends TException { + static $_TSPEC; + + public $message = null; + + public function __construct($vals=null) { + if (!isset(self::$_TSPEC)) { + self::$_TSPEC = array( + 1 => array( + 'var' => 'message', + 'type' => TType::STRING, + ), + ); + } + if (is_array($vals)) { + if (isset($vals['message'])) { + $this->message = $vals['message']; + } + } + } + + public function getName() { + return 'NoSuchLockException'; + } + + public function read($input) + { + $xfer = 0; + $fname = null; + $ftype = 0; + $fid = 0; + $xfer += $input->readStructBegin($fname); + while (true) + { + $xfer += $input->readFieldBegin($fname, $ftype, $fid); + if ($ftype == TType::STOP) { + break; + } + switch ($fid) + { + case 1: + if ($ftype == TType::STRING) { + $xfer += $input->readString($this->message); + } else { + $xfer += $input->skip($ftype); + } + break; + default: + $xfer += $input->skip($ftype); + break; + } + $xfer += $input->readFieldEnd(); + } + $xfer += $input->readStructEnd(); + return $xfer; + } + + public function write($output) { + $xfer = 0; + $xfer += $output->writeStructBegin('NoSuchLockException'); if ($this->message !== null) { $xfer += $output->writeFieldBegin('message', TType::STRING, 1); $xfer += $output->writeString($this->message); diff --git metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore-remote metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore-remote index 8988133..69ef1b5 100644 --- metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore-remote +++ metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore-remote @@ -115,6 +115,18 @@ if len(sys.argv) <= 1 or sys.argv[1] == '--help': print ' string get_delegation_token(string token_owner, string renewer_kerberos_principal_name)' print ' i64 renew_delegation_token(string token_str_form)' print ' void cancel_delegation_token(string token_str_form)' + print ' GetOpenTxnsResponse get_open_txns()' + print ' GetOpenTxnsInfoResponse get_open_txns_info()' + print ' OpenTxnsResponse open_txns(OpenTxnRequest rqst)' + print ' void abort_txn(AbortTxnRequest rqst)' + print ' void commit_txn(CommitTxnRequest rqst)' + print ' LockResponse lock(LockRequest rqst)' + print ' LockResponse check_lock(CheckLockRequest rqst)' + print ' void unlock(UnlockRequest rqst)' + print ' ShowLocksResponse show_locks(ShowLocksRequest rqst)' + print ' void heartbeat(HeartbeatRequest ids)' + print ' void compact(CompactionRequest rqst)' + print ' ShowCompactResponse show_compact(ShowCompactRequest rqst)' print '' sys.exit(0) @@ -718,6 +730,78 @@ elif cmd == 'cancel_delegation_token': sys.exit(1) pp.pprint(client.cancel_delegation_token(args[0],)) +elif cmd == 'get_open_txns': + if len(args) != 0: + print 'get_open_txns requires 0 args' + sys.exit(1) + pp.pprint(client.get_open_txns()) + +elif cmd == 'get_open_txns_info': + if len(args) != 0: + print 'get_open_txns_info requires 0 args' + sys.exit(1) + pp.pprint(client.get_open_txns_info()) + +elif cmd == 'open_txns': + if len(args) != 1: + print 'open_txns requires 1 args' + sys.exit(1) + pp.pprint(client.open_txns(eval(args[0]),)) + +elif cmd == 'abort_txn': + if len(args) != 1: + print 'abort_txn requires 1 args' + sys.exit(1) + pp.pprint(client.abort_txn(eval(args[0]),)) + +elif cmd == 'commit_txn': + if len(args) != 1: + print 'commit_txn requires 1 args' + sys.exit(1) + pp.pprint(client.commit_txn(eval(args[0]),)) + +elif cmd == 'lock': + if len(args) != 1: + print 'lock requires 1 args' + sys.exit(1) + pp.pprint(client.lock(eval(args[0]),)) + +elif cmd == 'check_lock': + if len(args) != 1: + print 'check_lock requires 1 args' + sys.exit(1) + pp.pprint(client.check_lock(eval(args[0]),)) + +elif cmd == 'unlock': + if len(args) != 1: + print 'unlock requires 1 args' + sys.exit(1) + pp.pprint(client.unlock(eval(args[0]),)) + +elif cmd == 'show_locks': + if len(args) != 1: + print 'show_locks requires 1 args' + sys.exit(1) + pp.pprint(client.show_locks(eval(args[0]),)) + +elif cmd == 'heartbeat': + if len(args) != 1: + print 'heartbeat requires 1 args' + sys.exit(1) + pp.pprint(client.heartbeat(eval(args[0]),)) + +elif cmd == 'compact': + if len(args) != 1: + print 'compact requires 1 args' + sys.exit(1) + pp.pprint(client.compact(eval(args[0]),)) + +elif cmd == 'show_compact': + if len(args) != 1: + print 'show_compact requires 1 args' + sys.exit(1) + pp.pprint(client.show_compact(eval(args[0]),)) + else: print 'Unrecognized method %s' % cmd sys.exit(1) diff --git metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py index eb0736a..b27c432 100644 --- metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py +++ metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py @@ -800,6 +800,82 @@ def cancel_delegation_token(self, token_str_form): """ pass + def get_open_txns(self, ): + pass + + def get_open_txns_info(self, ): + pass + + def open_txns(self, rqst): + """ + Parameters: + - rqst + """ + pass + + def abort_txn(self, rqst): + """ + Parameters: + - rqst + """ + pass + + def commit_txn(self, rqst): + """ + Parameters: + - rqst + """ + pass + + def lock(self, rqst): + """ + Parameters: + - rqst + """ + pass + + def check_lock(self, rqst): + """ + Parameters: + - rqst + """ + pass + + def unlock(self, rqst): + """ + Parameters: + - rqst + """ + pass + + def show_locks(self, rqst): + """ + Parameters: + - rqst + """ + pass + + def heartbeat(self, ids): + """ + Parameters: + - ids + """ + pass + + def compact(self, rqst): + """ + Parameters: + - rqst + """ + pass + + def show_compact(self, rqst): + """ + Parameters: + - rqst + """ + pass + class Client(fb303.FacebookService.Client, Iface): """ @@ -4208,1642 +4284,4009 @@ def recv_cancel_delegation_token(self, ): raise result.o1 return + def get_open_txns(self, ): + self.send_get_open_txns() + return self.recv_get_open_txns() -class Processor(fb303.FacebookService.Processor, Iface, TProcessor): - def __init__(self, handler): - fb303.FacebookService.Processor.__init__(self, handler) - self._processMap["create_database"] = Processor.process_create_database - self._processMap["get_database"] = Processor.process_get_database - self._processMap["drop_database"] = Processor.process_drop_database - self._processMap["get_databases"] = Processor.process_get_databases - self._processMap["get_all_databases"] = Processor.process_get_all_databases - self._processMap["alter_database"] = Processor.process_alter_database - self._processMap["get_type"] = Processor.process_get_type - self._processMap["create_type"] = Processor.process_create_type - self._processMap["drop_type"] = Processor.process_drop_type - self._processMap["get_type_all"] = Processor.process_get_type_all - self._processMap["get_fields"] = Processor.process_get_fields - self._processMap["get_schema"] = Processor.process_get_schema - self._processMap["create_table"] = Processor.process_create_table - self._processMap["create_table_with_environment_context"] = Processor.process_create_table_with_environment_context - self._processMap["drop_table"] = Processor.process_drop_table - self._processMap["drop_table_with_environment_context"] = Processor.process_drop_table_with_environment_context - self._processMap["get_tables"] = Processor.process_get_tables - self._processMap["get_all_tables"] = Processor.process_get_all_tables - self._processMap["get_table"] = Processor.process_get_table - self._processMap["get_table_objects_by_name"] = Processor.process_get_table_objects_by_name - self._processMap["get_table_names_by_filter"] = Processor.process_get_table_names_by_filter - self._processMap["alter_table"] = Processor.process_alter_table - self._processMap["alter_table_with_environment_context"] = Processor.process_alter_table_with_environment_context - self._processMap["add_partition"] = Processor.process_add_partition - self._processMap["add_partition_with_environment_context"] = Processor.process_add_partition_with_environment_context - self._processMap["add_partitions"] = Processor.process_add_partitions - self._processMap["append_partition"] = Processor.process_append_partition - self._processMap["add_partitions_req"] = Processor.process_add_partitions_req - self._processMap["append_partition_with_environment_context"] = Processor.process_append_partition_with_environment_context - self._processMap["append_partition_by_name"] = Processor.process_append_partition_by_name - self._processMap["append_partition_by_name_with_environment_context"] = Processor.process_append_partition_by_name_with_environment_context - self._processMap["drop_partition"] = Processor.process_drop_partition - self._processMap["drop_partition_with_environment_context"] = Processor.process_drop_partition_with_environment_context - self._processMap["drop_partition_by_name"] = Processor.process_drop_partition_by_name - self._processMap["drop_partition_by_name_with_environment_context"] = Processor.process_drop_partition_by_name_with_environment_context - self._processMap["drop_partitions_req"] = Processor.process_drop_partitions_req - self._processMap["get_partition"] = Processor.process_get_partition - self._processMap["exchange_partition"] = Processor.process_exchange_partition - self._processMap["get_partition_with_auth"] = Processor.process_get_partition_with_auth - self._processMap["get_partition_by_name"] = Processor.process_get_partition_by_name - self._processMap["get_partitions"] = Processor.process_get_partitions - self._processMap["get_partitions_with_auth"] = Processor.process_get_partitions_with_auth - self._processMap["get_partition_names"] = Processor.process_get_partition_names - self._processMap["get_partitions_ps"] = Processor.process_get_partitions_ps - self._processMap["get_partitions_ps_with_auth"] = Processor.process_get_partitions_ps_with_auth - self._processMap["get_partition_names_ps"] = Processor.process_get_partition_names_ps - self._processMap["get_partitions_by_filter"] = Processor.process_get_partitions_by_filter - self._processMap["get_partitions_by_expr"] = Processor.process_get_partitions_by_expr - self._processMap["get_partitions_by_names"] = Processor.process_get_partitions_by_names - self._processMap["alter_partition"] = Processor.process_alter_partition - self._processMap["alter_partitions"] = Processor.process_alter_partitions - self._processMap["alter_partition_with_environment_context"] = Processor.process_alter_partition_with_environment_context - self._processMap["rename_partition"] = Processor.process_rename_partition - self._processMap["partition_name_has_valid_characters"] = Processor.process_partition_name_has_valid_characters - self._processMap["get_config_value"] = Processor.process_get_config_value - self._processMap["partition_name_to_vals"] = Processor.process_partition_name_to_vals - self._processMap["partition_name_to_spec"] = Processor.process_partition_name_to_spec - self._processMap["markPartitionForEvent"] = Processor.process_markPartitionForEvent - self._processMap["isPartitionMarkedForEvent"] = Processor.process_isPartitionMarkedForEvent - self._processMap["add_index"] = Processor.process_add_index - self._processMap["alter_index"] = Processor.process_alter_index - self._processMap["drop_index_by_name"] = Processor.process_drop_index_by_name - self._processMap["get_index_by_name"] = Processor.process_get_index_by_name - self._processMap["get_indexes"] = Processor.process_get_indexes - self._processMap["get_index_names"] = Processor.process_get_index_names - self._processMap["update_table_column_statistics"] = Processor.process_update_table_column_statistics - self._processMap["update_partition_column_statistics"] = Processor.process_update_partition_column_statistics - self._processMap["get_table_column_statistics"] = Processor.process_get_table_column_statistics - self._processMap["get_partition_column_statistics"] = Processor.process_get_partition_column_statistics - self._processMap["get_table_statistics_req"] = Processor.process_get_table_statistics_req - self._processMap["get_partitions_statistics_req"] = Processor.process_get_partitions_statistics_req - self._processMap["delete_partition_column_statistics"] = Processor.process_delete_partition_column_statistics - self._processMap["delete_table_column_statistics"] = Processor.process_delete_table_column_statistics - self._processMap["create_function"] = Processor.process_create_function - self._processMap["drop_function"] = Processor.process_drop_function - self._processMap["alter_function"] = Processor.process_alter_function - self._processMap["get_functions"] = Processor.process_get_functions - self._processMap["get_function"] = Processor.process_get_function - self._processMap["create_role"] = Processor.process_create_role - self._processMap["drop_role"] = Processor.process_drop_role - self._processMap["get_role_names"] = Processor.process_get_role_names - self._processMap["grant_role"] = Processor.process_grant_role - self._processMap["revoke_role"] = Processor.process_revoke_role - self._processMap["list_roles"] = Processor.process_list_roles - self._processMap["get_privilege_set"] = Processor.process_get_privilege_set - self._processMap["list_privileges"] = Processor.process_list_privileges - self._processMap["grant_privileges"] = Processor.process_grant_privileges - self._processMap["revoke_privileges"] = Processor.process_revoke_privileges - self._processMap["set_ugi"] = Processor.process_set_ugi - self._processMap["get_delegation_token"] = Processor.process_get_delegation_token - self._processMap["renew_delegation_token"] = Processor.process_renew_delegation_token - self._processMap["cancel_delegation_token"] = Processor.process_cancel_delegation_token + def send_get_open_txns(self, ): + self._oprot.writeMessageBegin('get_open_txns', TMessageType.CALL, self._seqid) + args = get_open_txns_args() + args.write(self._oprot) + self._oprot.writeMessageEnd() + self._oprot.trans.flush() - def process(self, iprot, oprot): - (name, type, seqid) = iprot.readMessageBegin() - if name not in self._processMap: - iprot.skip(TType.STRUCT) - iprot.readMessageEnd() - x = TApplicationException(TApplicationException.UNKNOWN_METHOD, 'Unknown function %s' % (name)) - oprot.writeMessageBegin(name, TMessageType.EXCEPTION, seqid) - x.write(oprot) - oprot.writeMessageEnd() - oprot.trans.flush() - return - else: - self._processMap[name](self, seqid, iprot, oprot) - return True + def recv_get_open_txns(self, ): + (fname, mtype, rseqid) = self._iprot.readMessageBegin() + if mtype == TMessageType.EXCEPTION: + x = TApplicationException() + x.read(self._iprot) + self._iprot.readMessageEnd() + raise x + result = get_open_txns_result() + result.read(self._iprot) + self._iprot.readMessageEnd() + if result.success is not None: + return result.success + raise TApplicationException(TApplicationException.MISSING_RESULT, "get_open_txns failed: unknown result"); - def process_create_database(self, seqid, iprot, oprot): - args = create_database_args() - args.read(iprot) - iprot.readMessageEnd() - result = create_database_result() - try: - self._handler.create_database(args.database) - except AlreadyExistsException as o1: - result.o1 = o1 - except InvalidObjectException as o2: - result.o2 = o2 - except MetaException as o3: - result.o3 = o3 - oprot.writeMessageBegin("create_database", TMessageType.REPLY, seqid) - result.write(oprot) - oprot.writeMessageEnd() - oprot.trans.flush() + def get_open_txns_info(self, ): + self.send_get_open_txns_info() + return self.recv_get_open_txns_info() - def process_get_database(self, seqid, iprot, oprot): - args = get_database_args() - args.read(iprot) - iprot.readMessageEnd() - result = get_database_result() - try: - result.success = self._handler.get_database(args.name) - except NoSuchObjectException as o1: - result.o1 = o1 - except MetaException as o2: - result.o2 = o2 - oprot.writeMessageBegin("get_database", TMessageType.REPLY, seqid) - result.write(oprot) - oprot.writeMessageEnd() - oprot.trans.flush() + def send_get_open_txns_info(self, ): + self._oprot.writeMessageBegin('get_open_txns_info', TMessageType.CALL, self._seqid) + args = get_open_txns_info_args() + args.write(self._oprot) + self._oprot.writeMessageEnd() + self._oprot.trans.flush() - def process_drop_database(self, seqid, iprot, oprot): - args = drop_database_args() - args.read(iprot) - iprot.readMessageEnd() - result = drop_database_result() - try: - self._handler.drop_database(args.name, args.deleteData, args.cascade) - except NoSuchObjectException as o1: - result.o1 = o1 - except InvalidOperationException as o2: - result.o2 = o2 - except MetaException as o3: - result.o3 = o3 - oprot.writeMessageBegin("drop_database", TMessageType.REPLY, seqid) - result.write(oprot) - oprot.writeMessageEnd() - oprot.trans.flush() + def recv_get_open_txns_info(self, ): + (fname, mtype, rseqid) = self._iprot.readMessageBegin() + if mtype == TMessageType.EXCEPTION: + x = TApplicationException() + x.read(self._iprot) + self._iprot.readMessageEnd() + raise x + result = get_open_txns_info_result() + result.read(self._iprot) + self._iprot.readMessageEnd() + if result.success is not None: + return result.success + raise TApplicationException(TApplicationException.MISSING_RESULT, "get_open_txns_info failed: unknown result"); - def process_get_databases(self, seqid, iprot, oprot): - args = get_databases_args() - args.read(iprot) - iprot.readMessageEnd() - result = get_databases_result() - try: - result.success = self._handler.get_databases(args.pattern) - except MetaException as o1: - result.o1 = o1 - oprot.writeMessageBegin("get_databases", TMessageType.REPLY, seqid) - result.write(oprot) - oprot.writeMessageEnd() - oprot.trans.flush() + def open_txns(self, rqst): + """ + Parameters: + - rqst + """ + self.send_open_txns(rqst) + return self.recv_open_txns() - def process_get_all_databases(self, seqid, iprot, oprot): - args = get_all_databases_args() - args.read(iprot) - iprot.readMessageEnd() - result = get_all_databases_result() - try: - result.success = self._handler.get_all_databases() - except MetaException as o1: - result.o1 = o1 - oprot.writeMessageBegin("get_all_databases", TMessageType.REPLY, seqid) - result.write(oprot) - oprot.writeMessageEnd() - oprot.trans.flush() + def send_open_txns(self, rqst): + self._oprot.writeMessageBegin('open_txns', TMessageType.CALL, self._seqid) + args = open_txns_args() + args.rqst = rqst + args.write(self._oprot) + self._oprot.writeMessageEnd() + self._oprot.trans.flush() - def process_alter_database(self, seqid, iprot, oprot): - args = alter_database_args() - args.read(iprot) - iprot.readMessageEnd() - result = alter_database_result() - try: - self._handler.alter_database(args.dbname, args.db) - except MetaException as o1: - result.o1 = o1 - except NoSuchObjectException as o2: - result.o2 = o2 - oprot.writeMessageBegin("alter_database", TMessageType.REPLY, seqid) - result.write(oprot) - oprot.writeMessageEnd() - oprot.trans.flush() + def recv_open_txns(self, ): + (fname, mtype, rseqid) = self._iprot.readMessageBegin() + if mtype == TMessageType.EXCEPTION: + x = TApplicationException() + x.read(self._iprot) + self._iprot.readMessageEnd() + raise x + result = open_txns_result() + result.read(self._iprot) + self._iprot.readMessageEnd() + if result.success is not None: + return result.success + raise TApplicationException(TApplicationException.MISSING_RESULT, "open_txns failed: unknown result"); - def process_get_type(self, seqid, iprot, oprot): - args = get_type_args() - args.read(iprot) - iprot.readMessageEnd() - result = get_type_result() - try: - result.success = self._handler.get_type(args.name) - except MetaException as o1: - result.o1 = o1 - except NoSuchObjectException as o2: - result.o2 = o2 - oprot.writeMessageBegin("get_type", TMessageType.REPLY, seqid) - result.write(oprot) - oprot.writeMessageEnd() - oprot.trans.flush() + def abort_txn(self, rqst): + """ + Parameters: + - rqst + """ + self.send_abort_txn(rqst) + self.recv_abort_txn() - def process_create_type(self, seqid, iprot, oprot): - args = create_type_args() - args.read(iprot) - iprot.readMessageEnd() - result = create_type_result() - try: - result.success = self._handler.create_type(args.type) - except AlreadyExistsException as o1: - result.o1 = o1 - except InvalidObjectException as o2: - result.o2 = o2 - except MetaException as o3: - result.o3 = o3 - oprot.writeMessageBegin("create_type", TMessageType.REPLY, seqid) - result.write(oprot) - oprot.writeMessageEnd() - oprot.trans.flush() + def send_abort_txn(self, rqst): + self._oprot.writeMessageBegin('abort_txn', TMessageType.CALL, self._seqid) + args = abort_txn_args() + args.rqst = rqst + args.write(self._oprot) + self._oprot.writeMessageEnd() + self._oprot.trans.flush() - def process_drop_type(self, seqid, iprot, oprot): - args = drop_type_args() - args.read(iprot) - iprot.readMessageEnd() - result = drop_type_result() - try: - result.success = self._handler.drop_type(args.type) - except MetaException as o1: - result.o1 = o1 - except NoSuchObjectException as o2: - result.o2 = o2 - oprot.writeMessageBegin("drop_type", TMessageType.REPLY, seqid) - result.write(oprot) - oprot.writeMessageEnd() - oprot.trans.flush() + def recv_abort_txn(self, ): + (fname, mtype, rseqid) = self._iprot.readMessageBegin() + if mtype == TMessageType.EXCEPTION: + x = TApplicationException() + x.read(self._iprot) + self._iprot.readMessageEnd() + raise x + result = abort_txn_result() + result.read(self._iprot) + self._iprot.readMessageEnd() + if result.o1 is not None: + raise result.o1 + return - def process_get_type_all(self, seqid, iprot, oprot): - args = get_type_all_args() - args.read(iprot) - iprot.readMessageEnd() - result = get_type_all_result() - try: - result.success = self._handler.get_type_all(args.name) - except MetaException as o2: - result.o2 = o2 - oprot.writeMessageBegin("get_type_all", TMessageType.REPLY, seqid) - result.write(oprot) - oprot.writeMessageEnd() - oprot.trans.flush() + def commit_txn(self, rqst): + """ + Parameters: + - rqst + """ + self.send_commit_txn(rqst) + self.recv_commit_txn() - def process_get_fields(self, seqid, iprot, oprot): - args = get_fields_args() - args.read(iprot) - iprot.readMessageEnd() - result = get_fields_result() - try: - result.success = self._handler.get_fields(args.db_name, args.table_name) - except MetaException as o1: - result.o1 = o1 - except UnknownTableException as o2: - result.o2 = o2 - except UnknownDBException as o3: - result.o3 = o3 - oprot.writeMessageBegin("get_fields", TMessageType.REPLY, seqid) - result.write(oprot) - oprot.writeMessageEnd() - oprot.trans.flush() + def send_commit_txn(self, rqst): + self._oprot.writeMessageBegin('commit_txn', TMessageType.CALL, self._seqid) + args = commit_txn_args() + args.rqst = rqst + args.write(self._oprot) + self._oprot.writeMessageEnd() + self._oprot.trans.flush() - def process_get_schema(self, seqid, iprot, oprot): - args = get_schema_args() - args.read(iprot) - iprot.readMessageEnd() - result = get_schema_result() - try: - result.success = self._handler.get_schema(args.db_name, args.table_name) - except MetaException as o1: - result.o1 = o1 - except UnknownTableException as o2: - result.o2 = o2 - except UnknownDBException as o3: - result.o3 = o3 - oprot.writeMessageBegin("get_schema", TMessageType.REPLY, seqid) - result.write(oprot) - oprot.writeMessageEnd() - oprot.trans.flush() + def recv_commit_txn(self, ): + (fname, mtype, rseqid) = self._iprot.readMessageBegin() + if mtype == TMessageType.EXCEPTION: + x = TApplicationException() + x.read(self._iprot) + self._iprot.readMessageEnd() + raise x + result = commit_txn_result() + result.read(self._iprot) + self._iprot.readMessageEnd() + if result.o1 is not None: + raise result.o1 + if result.o2 is not None: + raise result.o2 + return - def process_create_table(self, seqid, iprot, oprot): - args = create_table_args() - args.read(iprot) - iprot.readMessageEnd() - result = create_table_result() - try: - self._handler.create_table(args.tbl) - except AlreadyExistsException as o1: - result.o1 = o1 - except InvalidObjectException as o2: - result.o2 = o2 - except MetaException as o3: - result.o3 = o3 - except NoSuchObjectException as o4: - result.o4 = o4 - oprot.writeMessageBegin("create_table", TMessageType.REPLY, seqid) - result.write(oprot) - oprot.writeMessageEnd() - oprot.trans.flush() + def lock(self, rqst): + """ + Parameters: + - rqst + """ + self.send_lock(rqst) + return self.recv_lock() - def process_create_table_with_environment_context(self, seqid, iprot, oprot): - args = create_table_with_environment_context_args() - args.read(iprot) - iprot.readMessageEnd() - result = create_table_with_environment_context_result() - try: - self._handler.create_table_with_environment_context(args.tbl, args.environment_context) - except AlreadyExistsException as o1: - result.o1 = o1 - except InvalidObjectException as o2: - result.o2 = o2 - except MetaException as o3: - result.o3 = o3 - except NoSuchObjectException as o4: - result.o4 = o4 - oprot.writeMessageBegin("create_table_with_environment_context", TMessageType.REPLY, seqid) - result.write(oprot) - oprot.writeMessageEnd() - oprot.trans.flush() + def send_lock(self, rqst): + self._oprot.writeMessageBegin('lock', TMessageType.CALL, self._seqid) + args = lock_args() + args.rqst = rqst + args.write(self._oprot) + self._oprot.writeMessageEnd() + self._oprot.trans.flush() - def process_drop_table(self, seqid, iprot, oprot): - args = drop_table_args() - args.read(iprot) - iprot.readMessageEnd() - result = drop_table_result() - try: - self._handler.drop_table(args.dbname, args.name, args.deleteData) - except NoSuchObjectException as o1: - result.o1 = o1 - except MetaException as o3: - result.o3 = o3 - oprot.writeMessageBegin("drop_table", TMessageType.REPLY, seqid) - result.write(oprot) - oprot.writeMessageEnd() - oprot.trans.flush() + def recv_lock(self, ): + (fname, mtype, rseqid) = self._iprot.readMessageBegin() + if mtype == TMessageType.EXCEPTION: + x = TApplicationException() + x.read(self._iprot) + self._iprot.readMessageEnd() + raise x + result = lock_result() + result.read(self._iprot) + self._iprot.readMessageEnd() + if result.success is not None: + return result.success + if result.o1 is not None: + raise result.o1 + if result.o2 is not None: + raise result.o2 + raise TApplicationException(TApplicationException.MISSING_RESULT, "lock failed: unknown result"); - def process_drop_table_with_environment_context(self, seqid, iprot, oprot): - args = drop_table_with_environment_context_args() - args.read(iprot) - iprot.readMessageEnd() - result = drop_table_with_environment_context_result() - try: - self._handler.drop_table_with_environment_context(args.dbname, args.name, args.deleteData, args.environment_context) - except NoSuchObjectException as o1: - result.o1 = o1 - except MetaException as o3: - result.o3 = o3 - oprot.writeMessageBegin("drop_table_with_environment_context", TMessageType.REPLY, seqid) - result.write(oprot) - oprot.writeMessageEnd() - oprot.trans.flush() + def check_lock(self, rqst): + """ + Parameters: + - rqst + """ + self.send_check_lock(rqst) + return self.recv_check_lock() - def process_get_tables(self, seqid, iprot, oprot): - args = get_tables_args() - args.read(iprot) - iprot.readMessageEnd() - result = get_tables_result() - try: - result.success = self._handler.get_tables(args.db_name, args.pattern) - except MetaException as o1: - result.o1 = o1 - oprot.writeMessageBegin("get_tables", TMessageType.REPLY, seqid) - result.write(oprot) - oprot.writeMessageEnd() - oprot.trans.flush() + def send_check_lock(self, rqst): + self._oprot.writeMessageBegin('check_lock', TMessageType.CALL, self._seqid) + args = check_lock_args() + args.rqst = rqst + args.write(self._oprot) + self._oprot.writeMessageEnd() + self._oprot.trans.flush() - def process_get_all_tables(self, seqid, iprot, oprot): - args = get_all_tables_args() - args.read(iprot) - iprot.readMessageEnd() - result = get_all_tables_result() - try: - result.success = self._handler.get_all_tables(args.db_name) - except MetaException as o1: - result.o1 = o1 - oprot.writeMessageBegin("get_all_tables", TMessageType.REPLY, seqid) - result.write(oprot) - oprot.writeMessageEnd() - oprot.trans.flush() + def recv_check_lock(self, ): + (fname, mtype, rseqid) = self._iprot.readMessageBegin() + if mtype == TMessageType.EXCEPTION: + x = TApplicationException() + x.read(self._iprot) + self._iprot.readMessageEnd() + raise x + result = check_lock_result() + result.read(self._iprot) + self._iprot.readMessageEnd() + if result.success is not None: + return result.success + if result.o1 is not None: + raise result.o1 + if result.o2 is not None: + raise result.o2 + if result.o3 is not None: + raise result.o3 + raise TApplicationException(TApplicationException.MISSING_RESULT, "check_lock failed: unknown result"); - def process_get_table(self, seqid, iprot, oprot): - args = get_table_args() - args.read(iprot) - iprot.readMessageEnd() - result = get_table_result() - try: - result.success = self._handler.get_table(args.dbname, args.tbl_name) - except MetaException as o1: - result.o1 = o1 - except NoSuchObjectException as o2: - result.o2 = o2 - oprot.writeMessageBegin("get_table", TMessageType.REPLY, seqid) - result.write(oprot) - oprot.writeMessageEnd() - oprot.trans.flush() + def unlock(self, rqst): + """ + Parameters: + - rqst + """ + self.send_unlock(rqst) + self.recv_unlock() - def process_get_table_objects_by_name(self, seqid, iprot, oprot): - args = get_table_objects_by_name_args() - args.read(iprot) - iprot.readMessageEnd() - result = get_table_objects_by_name_result() - try: - result.success = self._handler.get_table_objects_by_name(args.dbname, args.tbl_names) - except MetaException as o1: - result.o1 = o1 - except InvalidOperationException as o2: - result.o2 = o2 - except UnknownDBException as o3: - result.o3 = o3 - oprot.writeMessageBegin("get_table_objects_by_name", TMessageType.REPLY, seqid) - result.write(oprot) - oprot.writeMessageEnd() - oprot.trans.flush() + def send_unlock(self, rqst): + self._oprot.writeMessageBegin('unlock', TMessageType.CALL, self._seqid) + args = unlock_args() + args.rqst = rqst + args.write(self._oprot) + self._oprot.writeMessageEnd() + self._oprot.trans.flush() - def process_get_table_names_by_filter(self, seqid, iprot, oprot): - args = get_table_names_by_filter_args() - args.read(iprot) - iprot.readMessageEnd() - result = get_table_names_by_filter_result() - try: - result.success = self._handler.get_table_names_by_filter(args.dbname, args.filter, args.max_tables) - except MetaException as o1: - result.o1 = o1 - except InvalidOperationException as o2: - result.o2 = o2 - except UnknownDBException as o3: - result.o3 = o3 - oprot.writeMessageBegin("get_table_names_by_filter", TMessageType.REPLY, seqid) - result.write(oprot) - oprot.writeMessageEnd() - oprot.trans.flush() + def recv_unlock(self, ): + (fname, mtype, rseqid) = self._iprot.readMessageBegin() + if mtype == TMessageType.EXCEPTION: + x = TApplicationException() + x.read(self._iprot) + self._iprot.readMessageEnd() + raise x + result = unlock_result() + result.read(self._iprot) + self._iprot.readMessageEnd() + if result.o1 is not None: + raise result.o1 + if result.o2 is not None: + raise result.o2 + return - def process_alter_table(self, seqid, iprot, oprot): - args = alter_table_args() - args.read(iprot) - iprot.readMessageEnd() - result = alter_table_result() - try: - self._handler.alter_table(args.dbname, args.tbl_name, args.new_tbl) - except InvalidOperationException as o1: - result.o1 = o1 - except MetaException as o2: - result.o2 = o2 - oprot.writeMessageBegin("alter_table", TMessageType.REPLY, seqid) - result.write(oprot) - oprot.writeMessageEnd() - oprot.trans.flush() + def show_locks(self, rqst): + """ + Parameters: + - rqst + """ + self.send_show_locks(rqst) + return self.recv_show_locks() - def process_alter_table_with_environment_context(self, seqid, iprot, oprot): - args = alter_table_with_environment_context_args() - args.read(iprot) - iprot.readMessageEnd() - result = alter_table_with_environment_context_result() - try: - self._handler.alter_table_with_environment_context(args.dbname, args.tbl_name, args.new_tbl, args.environment_context) - except InvalidOperationException as o1: - result.o1 = o1 - except MetaException as o2: - result.o2 = o2 - oprot.writeMessageBegin("alter_table_with_environment_context", TMessageType.REPLY, seqid) - result.write(oprot) - oprot.writeMessageEnd() - oprot.trans.flush() + def send_show_locks(self, rqst): + self._oprot.writeMessageBegin('show_locks', TMessageType.CALL, self._seqid) + args = show_locks_args() + args.rqst = rqst + args.write(self._oprot) + self._oprot.writeMessageEnd() + self._oprot.trans.flush() - def process_add_partition(self, seqid, iprot, oprot): - args = add_partition_args() - args.read(iprot) - iprot.readMessageEnd() - result = add_partition_result() - try: - result.success = self._handler.add_partition(args.new_part) - except InvalidObjectException as o1: - result.o1 = o1 - except AlreadyExistsException as o2: - result.o2 = o2 - except MetaException as o3: - result.o3 = o3 - oprot.writeMessageBegin("add_partition", TMessageType.REPLY, seqid) - result.write(oprot) - oprot.writeMessageEnd() - oprot.trans.flush() + def recv_show_locks(self, ): + (fname, mtype, rseqid) = self._iprot.readMessageBegin() + if mtype == TMessageType.EXCEPTION: + x = TApplicationException() + x.read(self._iprot) + self._iprot.readMessageEnd() + raise x + result = show_locks_result() + result.read(self._iprot) + self._iprot.readMessageEnd() + if result.success is not None: + return result.success + raise TApplicationException(TApplicationException.MISSING_RESULT, "show_locks failed: unknown result"); - def process_add_partition_with_environment_context(self, seqid, iprot, oprot): - args = add_partition_with_environment_context_args() - args.read(iprot) - iprot.readMessageEnd() - result = add_partition_with_environment_context_result() - try: - result.success = self._handler.add_partition_with_environment_context(args.new_part, args.environment_context) - except InvalidObjectException as o1: - result.o1 = o1 - except AlreadyExistsException as o2: - result.o2 = o2 - except MetaException as o3: - result.o3 = o3 - oprot.writeMessageBegin("add_partition_with_environment_context", TMessageType.REPLY, seqid) - result.write(oprot) - oprot.writeMessageEnd() - oprot.trans.flush() + def heartbeat(self, ids): + """ + Parameters: + - ids + """ + self.send_heartbeat(ids) + self.recv_heartbeat() - def process_add_partitions(self, seqid, iprot, oprot): - args = add_partitions_args() - args.read(iprot) - iprot.readMessageEnd() - result = add_partitions_result() - try: - result.success = self._handler.add_partitions(args.new_parts) - except InvalidObjectException as o1: - result.o1 = o1 - except AlreadyExistsException as o2: - result.o2 = o2 - except MetaException as o3: - result.o3 = o3 - oprot.writeMessageBegin("add_partitions", TMessageType.REPLY, seqid) - result.write(oprot) - oprot.writeMessageEnd() - oprot.trans.flush() + def send_heartbeat(self, ids): + self._oprot.writeMessageBegin('heartbeat', TMessageType.CALL, self._seqid) + args = heartbeat_args() + args.ids = ids + args.write(self._oprot) + self._oprot.writeMessageEnd() + self._oprot.trans.flush() - def process_append_partition(self, seqid, iprot, oprot): - args = append_partition_args() - args.read(iprot) - iprot.readMessageEnd() - result = append_partition_result() - try: - result.success = self._handler.append_partition(args.db_name, args.tbl_name, args.part_vals) - except InvalidObjectException as o1: - result.o1 = o1 - except AlreadyExistsException as o2: - result.o2 = o2 - except MetaException as o3: - result.o3 = o3 - oprot.writeMessageBegin("append_partition", TMessageType.REPLY, seqid) - result.write(oprot) - oprot.writeMessageEnd() - oprot.trans.flush() + def recv_heartbeat(self, ): + (fname, mtype, rseqid) = self._iprot.readMessageBegin() + if mtype == TMessageType.EXCEPTION: + x = TApplicationException() + x.read(self._iprot) + self._iprot.readMessageEnd() + raise x + result = heartbeat_result() + result.read(self._iprot) + self._iprot.readMessageEnd() + if result.o1 is not None: + raise result.o1 + if result.o2 is not None: + raise result.o2 + if result.o3 is not None: + raise result.o3 + return - def process_add_partitions_req(self, seqid, iprot, oprot): - args = add_partitions_req_args() - args.read(iprot) - iprot.readMessageEnd() - result = add_partitions_req_result() + def compact(self, rqst): + """ + Parameters: + - rqst + """ + self.send_compact(rqst) + self.recv_compact() + + def send_compact(self, rqst): + self._oprot.writeMessageBegin('compact', TMessageType.CALL, self._seqid) + args = compact_args() + args.rqst = rqst + args.write(self._oprot) + self._oprot.writeMessageEnd() + self._oprot.trans.flush() + + def recv_compact(self, ): + (fname, mtype, rseqid) = self._iprot.readMessageBegin() + if mtype == TMessageType.EXCEPTION: + x = TApplicationException() + x.read(self._iprot) + self._iprot.readMessageEnd() + raise x + result = compact_result() + result.read(self._iprot) + self._iprot.readMessageEnd() + return + + def show_compact(self, rqst): + """ + Parameters: + - rqst + """ + self.send_show_compact(rqst) + return self.recv_show_compact() + + def send_show_compact(self, rqst): + self._oprot.writeMessageBegin('show_compact', TMessageType.CALL, self._seqid) + args = show_compact_args() + args.rqst = rqst + args.write(self._oprot) + self._oprot.writeMessageEnd() + self._oprot.trans.flush() + + def recv_show_compact(self, ): + (fname, mtype, rseqid) = self._iprot.readMessageBegin() + if mtype == TMessageType.EXCEPTION: + x = TApplicationException() + x.read(self._iprot) + self._iprot.readMessageEnd() + raise x + result = show_compact_result() + result.read(self._iprot) + self._iprot.readMessageEnd() + if result.success is not None: + return result.success + raise TApplicationException(TApplicationException.MISSING_RESULT, "show_compact failed: unknown result"); + + +class Processor(fb303.FacebookService.Processor, Iface, TProcessor): + def __init__(self, handler): + fb303.FacebookService.Processor.__init__(self, handler) + self._processMap["create_database"] = Processor.process_create_database + self._processMap["get_database"] = Processor.process_get_database + self._processMap["drop_database"] = Processor.process_drop_database + self._processMap["get_databases"] = Processor.process_get_databases + self._processMap["get_all_databases"] = Processor.process_get_all_databases + self._processMap["alter_database"] = Processor.process_alter_database + self._processMap["get_type"] = Processor.process_get_type + self._processMap["create_type"] = Processor.process_create_type + self._processMap["drop_type"] = Processor.process_drop_type + self._processMap["get_type_all"] = Processor.process_get_type_all + self._processMap["get_fields"] = Processor.process_get_fields + self._processMap["get_schema"] = Processor.process_get_schema + self._processMap["create_table"] = Processor.process_create_table + self._processMap["create_table_with_environment_context"] = Processor.process_create_table_with_environment_context + self._processMap["drop_table"] = Processor.process_drop_table + self._processMap["drop_table_with_environment_context"] = Processor.process_drop_table_with_environment_context + self._processMap["get_tables"] = Processor.process_get_tables + self._processMap["get_all_tables"] = Processor.process_get_all_tables + self._processMap["get_table"] = Processor.process_get_table + self._processMap["get_table_objects_by_name"] = Processor.process_get_table_objects_by_name + self._processMap["get_table_names_by_filter"] = Processor.process_get_table_names_by_filter + self._processMap["alter_table"] = Processor.process_alter_table + self._processMap["alter_table_with_environment_context"] = Processor.process_alter_table_with_environment_context + self._processMap["add_partition"] = Processor.process_add_partition + self._processMap["add_partition_with_environment_context"] = Processor.process_add_partition_with_environment_context + self._processMap["add_partitions"] = Processor.process_add_partitions + self._processMap["append_partition"] = Processor.process_append_partition + self._processMap["add_partitions_req"] = Processor.process_add_partitions_req + self._processMap["append_partition_with_environment_context"] = Processor.process_append_partition_with_environment_context + self._processMap["append_partition_by_name"] = Processor.process_append_partition_by_name + self._processMap["append_partition_by_name_with_environment_context"] = Processor.process_append_partition_by_name_with_environment_context + self._processMap["drop_partition"] = Processor.process_drop_partition + self._processMap["drop_partition_with_environment_context"] = Processor.process_drop_partition_with_environment_context + self._processMap["drop_partition_by_name"] = Processor.process_drop_partition_by_name + self._processMap["drop_partition_by_name_with_environment_context"] = Processor.process_drop_partition_by_name_with_environment_context + self._processMap["drop_partitions_req"] = Processor.process_drop_partitions_req + self._processMap["get_partition"] = Processor.process_get_partition + self._processMap["exchange_partition"] = Processor.process_exchange_partition + self._processMap["get_partition_with_auth"] = Processor.process_get_partition_with_auth + self._processMap["get_partition_by_name"] = Processor.process_get_partition_by_name + self._processMap["get_partitions"] = Processor.process_get_partitions + self._processMap["get_partitions_with_auth"] = Processor.process_get_partitions_with_auth + self._processMap["get_partition_names"] = Processor.process_get_partition_names + self._processMap["get_partitions_ps"] = Processor.process_get_partitions_ps + self._processMap["get_partitions_ps_with_auth"] = Processor.process_get_partitions_ps_with_auth + self._processMap["get_partition_names_ps"] = Processor.process_get_partition_names_ps + self._processMap["get_partitions_by_filter"] = Processor.process_get_partitions_by_filter + self._processMap["get_partitions_by_expr"] = Processor.process_get_partitions_by_expr + self._processMap["get_partitions_by_names"] = Processor.process_get_partitions_by_names + self._processMap["alter_partition"] = Processor.process_alter_partition + self._processMap["alter_partitions"] = Processor.process_alter_partitions + self._processMap["alter_partition_with_environment_context"] = Processor.process_alter_partition_with_environment_context + self._processMap["rename_partition"] = Processor.process_rename_partition + self._processMap["partition_name_has_valid_characters"] = Processor.process_partition_name_has_valid_characters + self._processMap["get_config_value"] = Processor.process_get_config_value + self._processMap["partition_name_to_vals"] = Processor.process_partition_name_to_vals + self._processMap["partition_name_to_spec"] = Processor.process_partition_name_to_spec + self._processMap["markPartitionForEvent"] = Processor.process_markPartitionForEvent + self._processMap["isPartitionMarkedForEvent"] = Processor.process_isPartitionMarkedForEvent + self._processMap["add_index"] = Processor.process_add_index + self._processMap["alter_index"] = Processor.process_alter_index + self._processMap["drop_index_by_name"] = Processor.process_drop_index_by_name + self._processMap["get_index_by_name"] = Processor.process_get_index_by_name + self._processMap["get_indexes"] = Processor.process_get_indexes + self._processMap["get_index_names"] = Processor.process_get_index_names + self._processMap["update_table_column_statistics"] = Processor.process_update_table_column_statistics + self._processMap["update_partition_column_statistics"] = Processor.process_update_partition_column_statistics + self._processMap["get_table_column_statistics"] = Processor.process_get_table_column_statistics + self._processMap["get_partition_column_statistics"] = Processor.process_get_partition_column_statistics + self._processMap["get_table_statistics_req"] = Processor.process_get_table_statistics_req + self._processMap["get_partitions_statistics_req"] = Processor.process_get_partitions_statistics_req + self._processMap["delete_partition_column_statistics"] = Processor.process_delete_partition_column_statistics + self._processMap["delete_table_column_statistics"] = Processor.process_delete_table_column_statistics + self._processMap["create_function"] = Processor.process_create_function + self._processMap["drop_function"] = Processor.process_drop_function + self._processMap["alter_function"] = Processor.process_alter_function + self._processMap["get_functions"] = Processor.process_get_functions + self._processMap["get_function"] = Processor.process_get_function + self._processMap["create_role"] = Processor.process_create_role + self._processMap["drop_role"] = Processor.process_drop_role + self._processMap["get_role_names"] = Processor.process_get_role_names + self._processMap["grant_role"] = Processor.process_grant_role + self._processMap["revoke_role"] = Processor.process_revoke_role + self._processMap["list_roles"] = Processor.process_list_roles + self._processMap["get_privilege_set"] = Processor.process_get_privilege_set + self._processMap["list_privileges"] = Processor.process_list_privileges + self._processMap["grant_privileges"] = Processor.process_grant_privileges + self._processMap["revoke_privileges"] = Processor.process_revoke_privileges + self._processMap["set_ugi"] = Processor.process_set_ugi + self._processMap["get_delegation_token"] = Processor.process_get_delegation_token + self._processMap["renew_delegation_token"] = Processor.process_renew_delegation_token + self._processMap["cancel_delegation_token"] = Processor.process_cancel_delegation_token + self._processMap["get_open_txns"] = Processor.process_get_open_txns + self._processMap["get_open_txns_info"] = Processor.process_get_open_txns_info + self._processMap["open_txns"] = Processor.process_open_txns + self._processMap["abort_txn"] = Processor.process_abort_txn + self._processMap["commit_txn"] = Processor.process_commit_txn + self._processMap["lock"] = Processor.process_lock + self._processMap["check_lock"] = Processor.process_check_lock + self._processMap["unlock"] = Processor.process_unlock + self._processMap["show_locks"] = Processor.process_show_locks + self._processMap["heartbeat"] = Processor.process_heartbeat + self._processMap["compact"] = Processor.process_compact + self._processMap["show_compact"] = Processor.process_show_compact + + def process(self, iprot, oprot): + (name, type, seqid) = iprot.readMessageBegin() + if name not in self._processMap: + iprot.skip(TType.STRUCT) + iprot.readMessageEnd() + x = TApplicationException(TApplicationException.UNKNOWN_METHOD, 'Unknown function %s' % (name)) + oprot.writeMessageBegin(name, TMessageType.EXCEPTION, seqid) + x.write(oprot) + oprot.writeMessageEnd() + oprot.trans.flush() + return + else: + self._processMap[name](self, seqid, iprot, oprot) + return True + + def process_create_database(self, seqid, iprot, oprot): + args = create_database_args() + args.read(iprot) + iprot.readMessageEnd() + result = create_database_result() try: - result.success = self._handler.add_partitions_req(args.request) - except InvalidObjectException as o1: + self._handler.create_database(args.database) + except AlreadyExistsException as o1: result.o1 = o1 - except AlreadyExistsException as o2: + except InvalidObjectException as o2: result.o2 = o2 except MetaException as o3: result.o3 = o3 - oprot.writeMessageBegin("add_partitions_req", TMessageType.REPLY, seqid) + oprot.writeMessageBegin("create_database", TMessageType.REPLY, seqid) result.write(oprot) oprot.writeMessageEnd() oprot.trans.flush() - def process_append_partition_with_environment_context(self, seqid, iprot, oprot): - args = append_partition_with_environment_context_args() + def process_get_database(self, seqid, iprot, oprot): + args = get_database_args() args.read(iprot) iprot.readMessageEnd() - result = append_partition_with_environment_context_result() + result = get_database_result() try: - result.success = self._handler.append_partition_with_environment_context(args.db_name, args.tbl_name, args.part_vals, args.environment_context) - except InvalidObjectException as o1: + result.success = self._handler.get_database(args.name) + except NoSuchObjectException as o1: result.o1 = o1 - except AlreadyExistsException as o2: + except MetaException as o2: result.o2 = o2 - except MetaException as o3: - result.o3 = o3 - oprot.writeMessageBegin("append_partition_with_environment_context", TMessageType.REPLY, seqid) + oprot.writeMessageBegin("get_database", TMessageType.REPLY, seqid) result.write(oprot) oprot.writeMessageEnd() oprot.trans.flush() - def process_append_partition_by_name(self, seqid, iprot, oprot): - args = append_partition_by_name_args() + def process_drop_database(self, seqid, iprot, oprot): + args = drop_database_args() args.read(iprot) iprot.readMessageEnd() - result = append_partition_by_name_result() + result = drop_database_result() try: - result.success = self._handler.append_partition_by_name(args.db_name, args.tbl_name, args.part_name) - except InvalidObjectException as o1: + self._handler.drop_database(args.name, args.deleteData, args.cascade) + except NoSuchObjectException as o1: result.o1 = o1 - except AlreadyExistsException as o2: + except InvalidOperationException as o2: result.o2 = o2 except MetaException as o3: result.o3 = o3 - oprot.writeMessageBegin("append_partition_by_name", TMessageType.REPLY, seqid) + oprot.writeMessageBegin("drop_database", TMessageType.REPLY, seqid) result.write(oprot) oprot.writeMessageEnd() oprot.trans.flush() - def process_append_partition_by_name_with_environment_context(self, seqid, iprot, oprot): - args = append_partition_by_name_with_environment_context_args() + def process_get_databases(self, seqid, iprot, oprot): + args = get_databases_args() args.read(iprot) iprot.readMessageEnd() - result = append_partition_by_name_with_environment_context_result() + result = get_databases_result() try: - result.success = self._handler.append_partition_by_name_with_environment_context(args.db_name, args.tbl_name, args.part_name, args.environment_context) - except InvalidObjectException as o1: + result.success = self._handler.get_databases(args.pattern) + except MetaException as o1: result.o1 = o1 - except AlreadyExistsException as o2: - result.o2 = o2 - except MetaException as o3: - result.o3 = o3 - oprot.writeMessageBegin("append_partition_by_name_with_environment_context", TMessageType.REPLY, seqid) + oprot.writeMessageBegin("get_databases", TMessageType.REPLY, seqid) result.write(oprot) oprot.writeMessageEnd() oprot.trans.flush() - def process_drop_partition(self, seqid, iprot, oprot): - args = drop_partition_args() + def process_get_all_databases(self, seqid, iprot, oprot): + args = get_all_databases_args() args.read(iprot) iprot.readMessageEnd() - result = drop_partition_result() + result = get_all_databases_result() try: - result.success = self._handler.drop_partition(args.db_name, args.tbl_name, args.part_vals, args.deleteData) - except NoSuchObjectException as o1: + result.success = self._handler.get_all_databases() + except MetaException as o1: result.o1 = o1 - except MetaException as o2: - result.o2 = o2 - oprot.writeMessageBegin("drop_partition", TMessageType.REPLY, seqid) + oprot.writeMessageBegin("get_all_databases", TMessageType.REPLY, seqid) result.write(oprot) oprot.writeMessageEnd() oprot.trans.flush() - def process_drop_partition_with_environment_context(self, seqid, iprot, oprot): - args = drop_partition_with_environment_context_args() + def process_alter_database(self, seqid, iprot, oprot): + args = alter_database_args() args.read(iprot) iprot.readMessageEnd() - result = drop_partition_with_environment_context_result() + result = alter_database_result() try: - result.success = self._handler.drop_partition_with_environment_context(args.db_name, args.tbl_name, args.part_vals, args.deleteData, args.environment_context) - except NoSuchObjectException as o1: + self._handler.alter_database(args.dbname, args.db) + except MetaException as o1: result.o1 = o1 - except MetaException as o2: + except NoSuchObjectException as o2: result.o2 = o2 - oprot.writeMessageBegin("drop_partition_with_environment_context", TMessageType.REPLY, seqid) + oprot.writeMessageBegin("alter_database", TMessageType.REPLY, seqid) result.write(oprot) oprot.writeMessageEnd() oprot.trans.flush() - def process_drop_partition_by_name(self, seqid, iprot, oprot): - args = drop_partition_by_name_args() + def process_get_type(self, seqid, iprot, oprot): + args = get_type_args() args.read(iprot) iprot.readMessageEnd() - result = drop_partition_by_name_result() + result = get_type_result() try: - result.success = self._handler.drop_partition_by_name(args.db_name, args.tbl_name, args.part_name, args.deleteData) - except NoSuchObjectException as o1: + result.success = self._handler.get_type(args.name) + except MetaException as o1: result.o1 = o1 - except MetaException as o2: + except NoSuchObjectException as o2: result.o2 = o2 - oprot.writeMessageBegin("drop_partition_by_name", TMessageType.REPLY, seqid) + oprot.writeMessageBegin("get_type", TMessageType.REPLY, seqid) result.write(oprot) oprot.writeMessageEnd() oprot.trans.flush() - def process_drop_partition_by_name_with_environment_context(self, seqid, iprot, oprot): - args = drop_partition_by_name_with_environment_context_args() + def process_create_type(self, seqid, iprot, oprot): + args = create_type_args() args.read(iprot) iprot.readMessageEnd() - result = drop_partition_by_name_with_environment_context_result() + result = create_type_result() try: - result.success = self._handler.drop_partition_by_name_with_environment_context(args.db_name, args.tbl_name, args.part_name, args.deleteData, args.environment_context) - except NoSuchObjectException as o1: + result.success = self._handler.create_type(args.type) + except AlreadyExistsException as o1: result.o1 = o1 - except MetaException as o2: + except InvalidObjectException as o2: result.o2 = o2 - oprot.writeMessageBegin("drop_partition_by_name_with_environment_context", TMessageType.REPLY, seqid) + except MetaException as o3: + result.o3 = o3 + oprot.writeMessageBegin("create_type", TMessageType.REPLY, seqid) result.write(oprot) oprot.writeMessageEnd() oprot.trans.flush() - def process_drop_partitions_req(self, seqid, iprot, oprot): - args = drop_partitions_req_args() + def process_drop_type(self, seqid, iprot, oprot): + args = drop_type_args() args.read(iprot) iprot.readMessageEnd() - result = drop_partitions_req_result() + result = drop_type_result() try: - result.success = self._handler.drop_partitions_req(args.req) - except NoSuchObjectException as o1: + result.success = self._handler.drop_type(args.type) + except MetaException as o1: result.o1 = o1 - except MetaException as o2: + except NoSuchObjectException as o2: result.o2 = o2 - oprot.writeMessageBegin("drop_partitions_req", TMessageType.REPLY, seqid) + oprot.writeMessageBegin("drop_type", TMessageType.REPLY, seqid) result.write(oprot) oprot.writeMessageEnd() oprot.trans.flush() - def process_get_partition(self, seqid, iprot, oprot): - args = get_partition_args() + def process_get_type_all(self, seqid, iprot, oprot): + args = get_type_all_args() args.read(iprot) iprot.readMessageEnd() - result = get_partition_result() + result = get_type_all_result() try: - result.success = self._handler.get_partition(args.db_name, args.tbl_name, args.part_vals) - except MetaException as o1: - result.o1 = o1 - except NoSuchObjectException as o2: + result.success = self._handler.get_type_all(args.name) + except MetaException as o2: result.o2 = o2 - oprot.writeMessageBegin("get_partition", TMessageType.REPLY, seqid) + oprot.writeMessageBegin("get_type_all", TMessageType.REPLY, seqid) result.write(oprot) oprot.writeMessageEnd() oprot.trans.flush() - def process_exchange_partition(self, seqid, iprot, oprot): - args = exchange_partition_args() + def process_get_fields(self, seqid, iprot, oprot): + args = get_fields_args() args.read(iprot) iprot.readMessageEnd() - result = exchange_partition_result() + result = get_fields_result() try: - result.success = self._handler.exchange_partition(args.partitionSpecs, args.source_db, args.source_table_name, args.dest_db, args.dest_table_name) + result.success = self._handler.get_fields(args.db_name, args.table_name) except MetaException as o1: result.o1 = o1 - except NoSuchObjectException as o2: + except UnknownTableException as o2: result.o2 = o2 - except InvalidObjectException as o3: + except UnknownDBException as o3: result.o3 = o3 - except InvalidInputException as o4: - result.o4 = o4 - oprot.writeMessageBegin("exchange_partition", TMessageType.REPLY, seqid) + oprot.writeMessageBegin("get_fields", TMessageType.REPLY, seqid) result.write(oprot) oprot.writeMessageEnd() oprot.trans.flush() - def process_get_partition_with_auth(self, seqid, iprot, oprot): - args = get_partition_with_auth_args() + def process_get_schema(self, seqid, iprot, oprot): + args = get_schema_args() args.read(iprot) iprot.readMessageEnd() - result = get_partition_with_auth_result() + result = get_schema_result() try: - result.success = self._handler.get_partition_with_auth(args.db_name, args.tbl_name, args.part_vals, args.user_name, args.group_names) + result.success = self._handler.get_schema(args.db_name, args.table_name) except MetaException as o1: result.o1 = o1 - except NoSuchObjectException as o2: + except UnknownTableException as o2: result.o2 = o2 - oprot.writeMessageBegin("get_partition_with_auth", TMessageType.REPLY, seqid) + except UnknownDBException as o3: + result.o3 = o3 + oprot.writeMessageBegin("get_schema", TMessageType.REPLY, seqid) result.write(oprot) oprot.writeMessageEnd() oprot.trans.flush() - def process_get_partition_by_name(self, seqid, iprot, oprot): - args = get_partition_by_name_args() + def process_create_table(self, seqid, iprot, oprot): + args = create_table_args() args.read(iprot) iprot.readMessageEnd() - result = get_partition_by_name_result() + result = create_table_result() try: - result.success = self._handler.get_partition_by_name(args.db_name, args.tbl_name, args.part_name) - except MetaException as o1: + self._handler.create_table(args.tbl) + except AlreadyExistsException as o1: result.o1 = o1 - except NoSuchObjectException as o2: + except InvalidObjectException as o2: result.o2 = o2 - oprot.writeMessageBegin("get_partition_by_name", TMessageType.REPLY, seqid) + except MetaException as o3: + result.o3 = o3 + except NoSuchObjectException as o4: + result.o4 = o4 + oprot.writeMessageBegin("create_table", TMessageType.REPLY, seqid) result.write(oprot) oprot.writeMessageEnd() oprot.trans.flush() - def process_get_partitions(self, seqid, iprot, oprot): - args = get_partitions_args() + def process_create_table_with_environment_context(self, seqid, iprot, oprot): + args = create_table_with_environment_context_args() args.read(iprot) iprot.readMessageEnd() - result = get_partitions_result() + result = create_table_with_environment_context_result() try: - result.success = self._handler.get_partitions(args.db_name, args.tbl_name, args.max_parts) - except NoSuchObjectException as o1: + self._handler.create_table_with_environment_context(args.tbl, args.environment_context) + except AlreadyExistsException as o1: result.o1 = o1 - except MetaException as o2: + except InvalidObjectException as o2: result.o2 = o2 - oprot.writeMessageBegin("get_partitions", TMessageType.REPLY, seqid) + except MetaException as o3: + result.o3 = o3 + except NoSuchObjectException as o4: + result.o4 = o4 + oprot.writeMessageBegin("create_table_with_environment_context", TMessageType.REPLY, seqid) result.write(oprot) oprot.writeMessageEnd() oprot.trans.flush() - def process_get_partitions_with_auth(self, seqid, iprot, oprot): - args = get_partitions_with_auth_args() + def process_drop_table(self, seqid, iprot, oprot): + args = drop_table_args() args.read(iprot) iprot.readMessageEnd() - result = get_partitions_with_auth_result() + result = drop_table_result() try: - result.success = self._handler.get_partitions_with_auth(args.db_name, args.tbl_name, args.max_parts, args.user_name, args.group_names) + self._handler.drop_table(args.dbname, args.name, args.deleteData) except NoSuchObjectException as o1: result.o1 = o1 - except MetaException as o2: - result.o2 = o2 - oprot.writeMessageBegin("get_partitions_with_auth", TMessageType.REPLY, seqid) + except MetaException as o3: + result.o3 = o3 + oprot.writeMessageBegin("drop_table", TMessageType.REPLY, seqid) result.write(oprot) oprot.writeMessageEnd() oprot.trans.flush() - def process_get_partition_names(self, seqid, iprot, oprot): - args = get_partition_names_args() + def process_drop_table_with_environment_context(self, seqid, iprot, oprot): + args = drop_table_with_environment_context_args() args.read(iprot) iprot.readMessageEnd() - result = get_partition_names_result() + result = drop_table_with_environment_context_result() try: - result.success = self._handler.get_partition_names(args.db_name, args.tbl_name, args.max_parts) - except MetaException as o2: - result.o2 = o2 - oprot.writeMessageBegin("get_partition_names", TMessageType.REPLY, seqid) + self._handler.drop_table_with_environment_context(args.dbname, args.name, args.deleteData, args.environment_context) + except NoSuchObjectException as o1: + result.o1 = o1 + except MetaException as o3: + result.o3 = o3 + oprot.writeMessageBegin("drop_table_with_environment_context", TMessageType.REPLY, seqid) result.write(oprot) oprot.writeMessageEnd() oprot.trans.flush() - def process_get_partitions_ps(self, seqid, iprot, oprot): - args = get_partitions_ps_args() + def process_get_tables(self, seqid, iprot, oprot): + args = get_tables_args() args.read(iprot) iprot.readMessageEnd() - result = get_partitions_ps_result() + result = get_tables_result() try: - result.success = self._handler.get_partitions_ps(args.db_name, args.tbl_name, args.part_vals, args.max_parts) + result.success = self._handler.get_tables(args.db_name, args.pattern) except MetaException as o1: result.o1 = o1 - except NoSuchObjectException as o2: - result.o2 = o2 - oprot.writeMessageBegin("get_partitions_ps", TMessageType.REPLY, seqid) + oprot.writeMessageBegin("get_tables", TMessageType.REPLY, seqid) result.write(oprot) oprot.writeMessageEnd() oprot.trans.flush() - def process_get_partitions_ps_with_auth(self, seqid, iprot, oprot): - args = get_partitions_ps_with_auth_args() + def process_get_all_tables(self, seqid, iprot, oprot): + args = get_all_tables_args() args.read(iprot) iprot.readMessageEnd() - result = get_partitions_ps_with_auth_result() + result = get_all_tables_result() try: - result.success = self._handler.get_partitions_ps_with_auth(args.db_name, args.tbl_name, args.part_vals, args.max_parts, args.user_name, args.group_names) - except NoSuchObjectException as o1: + result.success = self._handler.get_all_tables(args.db_name) + except MetaException as o1: result.o1 = o1 - except MetaException as o2: - result.o2 = o2 - oprot.writeMessageBegin("get_partitions_ps_with_auth", TMessageType.REPLY, seqid) + oprot.writeMessageBegin("get_all_tables", TMessageType.REPLY, seqid) result.write(oprot) oprot.writeMessageEnd() oprot.trans.flush() - def process_get_partition_names_ps(self, seqid, iprot, oprot): - args = get_partition_names_ps_args() + def process_get_table(self, seqid, iprot, oprot): + args = get_table_args() args.read(iprot) iprot.readMessageEnd() - result = get_partition_names_ps_result() + result = get_table_result() try: - result.success = self._handler.get_partition_names_ps(args.db_name, args.tbl_name, args.part_vals, args.max_parts) + result.success = self._handler.get_table(args.dbname, args.tbl_name) except MetaException as o1: result.o1 = o1 except NoSuchObjectException as o2: result.o2 = o2 - oprot.writeMessageBegin("get_partition_names_ps", TMessageType.REPLY, seqid) + oprot.writeMessageBegin("get_table", TMessageType.REPLY, seqid) result.write(oprot) oprot.writeMessageEnd() oprot.trans.flush() - def process_get_partitions_by_filter(self, seqid, iprot, oprot): - args = get_partitions_by_filter_args() + def process_get_table_objects_by_name(self, seqid, iprot, oprot): + args = get_table_objects_by_name_args() args.read(iprot) iprot.readMessageEnd() - result = get_partitions_by_filter_result() + result = get_table_objects_by_name_result() try: - result.success = self._handler.get_partitions_by_filter(args.db_name, args.tbl_name, args.filter, args.max_parts) + result.success = self._handler.get_table_objects_by_name(args.dbname, args.tbl_names) except MetaException as o1: result.o1 = o1 - except NoSuchObjectException as o2: + except InvalidOperationException as o2: result.o2 = o2 - oprot.writeMessageBegin("get_partitions_by_filter", TMessageType.REPLY, seqid) + except UnknownDBException as o3: + result.o3 = o3 + oprot.writeMessageBegin("get_table_objects_by_name", TMessageType.REPLY, seqid) result.write(oprot) oprot.writeMessageEnd() oprot.trans.flush() - def process_get_partitions_by_expr(self, seqid, iprot, oprot): - args = get_partitions_by_expr_args() + def process_get_table_names_by_filter(self, seqid, iprot, oprot): + args = get_table_names_by_filter_args() args.read(iprot) iprot.readMessageEnd() - result = get_partitions_by_expr_result() + result = get_table_names_by_filter_result() try: - result.success = self._handler.get_partitions_by_expr(args.req) + result.success = self._handler.get_table_names_by_filter(args.dbname, args.filter, args.max_tables) except MetaException as o1: result.o1 = o1 - except NoSuchObjectException as o2: + except InvalidOperationException as o2: result.o2 = o2 - oprot.writeMessageBegin("get_partitions_by_expr", TMessageType.REPLY, seqid) + except UnknownDBException as o3: + result.o3 = o3 + oprot.writeMessageBegin("get_table_names_by_filter", TMessageType.REPLY, seqid) result.write(oprot) oprot.writeMessageEnd() oprot.trans.flush() - def process_get_partitions_by_names(self, seqid, iprot, oprot): - args = get_partitions_by_names_args() + def process_alter_table(self, seqid, iprot, oprot): + args = alter_table_args() args.read(iprot) iprot.readMessageEnd() - result = get_partitions_by_names_result() + result = alter_table_result() try: - result.success = self._handler.get_partitions_by_names(args.db_name, args.tbl_name, args.names) - except MetaException as o1: + self._handler.alter_table(args.dbname, args.tbl_name, args.new_tbl) + except InvalidOperationException as o1: result.o1 = o1 - except NoSuchObjectException as o2: + except MetaException as o2: result.o2 = o2 - oprot.writeMessageBegin("get_partitions_by_names", TMessageType.REPLY, seqid) + oprot.writeMessageBegin("alter_table", TMessageType.REPLY, seqid) result.write(oprot) oprot.writeMessageEnd() oprot.trans.flush() - def process_alter_partition(self, seqid, iprot, oprot): - args = alter_partition_args() + def process_alter_table_with_environment_context(self, seqid, iprot, oprot): + args = alter_table_with_environment_context_args() args.read(iprot) iprot.readMessageEnd() - result = alter_partition_result() + result = alter_table_with_environment_context_result() try: - self._handler.alter_partition(args.db_name, args.tbl_name, args.new_part) + self._handler.alter_table_with_environment_context(args.dbname, args.tbl_name, args.new_tbl, args.environment_context) except InvalidOperationException as o1: result.o1 = o1 except MetaException as o2: result.o2 = o2 - oprot.writeMessageBegin("alter_partition", TMessageType.REPLY, seqid) + oprot.writeMessageBegin("alter_table_with_environment_context", TMessageType.REPLY, seqid) result.write(oprot) oprot.writeMessageEnd() oprot.trans.flush() - def process_alter_partitions(self, seqid, iprot, oprot): - args = alter_partitions_args() + def process_add_partition(self, seqid, iprot, oprot): + args = add_partition_args() args.read(iprot) iprot.readMessageEnd() - result = alter_partitions_result() + result = add_partition_result() try: - self._handler.alter_partitions(args.db_name, args.tbl_name, args.new_parts) - except InvalidOperationException as o1: + result.success = self._handler.add_partition(args.new_part) + except InvalidObjectException as o1: result.o1 = o1 - except MetaException as o2: + except AlreadyExistsException as o2: result.o2 = o2 - oprot.writeMessageBegin("alter_partitions", TMessageType.REPLY, seqid) + except MetaException as o3: + result.o3 = o3 + oprot.writeMessageBegin("add_partition", TMessageType.REPLY, seqid) result.write(oprot) oprot.writeMessageEnd() oprot.trans.flush() - def process_alter_partition_with_environment_context(self, seqid, iprot, oprot): - args = alter_partition_with_environment_context_args() + def process_add_partition_with_environment_context(self, seqid, iprot, oprot): + args = add_partition_with_environment_context_args() args.read(iprot) iprot.readMessageEnd() - result = alter_partition_with_environment_context_result() + result = add_partition_with_environment_context_result() try: - self._handler.alter_partition_with_environment_context(args.db_name, args.tbl_name, args.new_part, args.environment_context) - except InvalidOperationException as o1: + result.success = self._handler.add_partition_with_environment_context(args.new_part, args.environment_context) + except InvalidObjectException as o1: result.o1 = o1 - except MetaException as o2: + except AlreadyExistsException as o2: result.o2 = o2 - oprot.writeMessageBegin("alter_partition_with_environment_context", TMessageType.REPLY, seqid) + except MetaException as o3: + result.o3 = o3 + oprot.writeMessageBegin("add_partition_with_environment_context", TMessageType.REPLY, seqid) result.write(oprot) oprot.writeMessageEnd() oprot.trans.flush() - def process_rename_partition(self, seqid, iprot, oprot): - args = rename_partition_args() + def process_add_partitions(self, seqid, iprot, oprot): + args = add_partitions_args() args.read(iprot) iprot.readMessageEnd() - result = rename_partition_result() + result = add_partitions_result() try: - self._handler.rename_partition(args.db_name, args.tbl_name, args.part_vals, args.new_part) - except InvalidOperationException as o1: + result.success = self._handler.add_partitions(args.new_parts) + except InvalidObjectException as o1: result.o1 = o1 - except MetaException as o2: + except AlreadyExistsException as o2: result.o2 = o2 - oprot.writeMessageBegin("rename_partition", TMessageType.REPLY, seqid) - result.write(oprot) - oprot.writeMessageEnd() - oprot.trans.flush() - - def process_partition_name_has_valid_characters(self, seqid, iprot, oprot): - args = partition_name_has_valid_characters_args() - args.read(iprot) - iprot.readMessageEnd() - result = partition_name_has_valid_characters_result() - try: - result.success = self._handler.partition_name_has_valid_characters(args.part_vals, args.throw_exception) - except MetaException as o1: - result.o1 = o1 - oprot.writeMessageBegin("partition_name_has_valid_characters", TMessageType.REPLY, seqid) + except MetaException as o3: + result.o3 = o3 + oprot.writeMessageBegin("add_partitions", TMessageType.REPLY, seqid) result.write(oprot) oprot.writeMessageEnd() oprot.trans.flush() - def process_get_config_value(self, seqid, iprot, oprot): - args = get_config_value_args() + def process_append_partition(self, seqid, iprot, oprot): + args = append_partition_args() args.read(iprot) iprot.readMessageEnd() - result = get_config_value_result() + result = append_partition_result() try: - result.success = self._handler.get_config_value(args.name, args.defaultValue) - except ConfigValSecurityException as o1: + result.success = self._handler.append_partition(args.db_name, args.tbl_name, args.part_vals) + except InvalidObjectException as o1: result.o1 = o1 - oprot.writeMessageBegin("get_config_value", TMessageType.REPLY, seqid) + except AlreadyExistsException as o2: + result.o2 = o2 + except MetaException as o3: + result.o3 = o3 + oprot.writeMessageBegin("append_partition", TMessageType.REPLY, seqid) result.write(oprot) oprot.writeMessageEnd() oprot.trans.flush() - def process_partition_name_to_vals(self, seqid, iprot, oprot): - args = partition_name_to_vals_args() + def process_add_partitions_req(self, seqid, iprot, oprot): + args = add_partitions_req_args() args.read(iprot) iprot.readMessageEnd() - result = partition_name_to_vals_result() + result = add_partitions_req_result() try: - result.success = self._handler.partition_name_to_vals(args.part_name) - except MetaException as o1: + result.success = self._handler.add_partitions_req(args.request) + except InvalidObjectException as o1: result.o1 = o1 - oprot.writeMessageBegin("partition_name_to_vals", TMessageType.REPLY, seqid) + except AlreadyExistsException as o2: + result.o2 = o2 + except MetaException as o3: + result.o3 = o3 + oprot.writeMessageBegin("add_partitions_req", TMessageType.REPLY, seqid) result.write(oprot) oprot.writeMessageEnd() oprot.trans.flush() - def process_partition_name_to_spec(self, seqid, iprot, oprot): - args = partition_name_to_spec_args() + def process_append_partition_with_environment_context(self, seqid, iprot, oprot): + args = append_partition_with_environment_context_args() args.read(iprot) iprot.readMessageEnd() - result = partition_name_to_spec_result() + result = append_partition_with_environment_context_result() try: - result.success = self._handler.partition_name_to_spec(args.part_name) - except MetaException as o1: + result.success = self._handler.append_partition_with_environment_context(args.db_name, args.tbl_name, args.part_vals, args.environment_context) + except InvalidObjectException as o1: result.o1 = o1 - oprot.writeMessageBegin("partition_name_to_spec", TMessageType.REPLY, seqid) + except AlreadyExistsException as o2: + result.o2 = o2 + except MetaException as o3: + result.o3 = o3 + oprot.writeMessageBegin("append_partition_with_environment_context", TMessageType.REPLY, seqid) result.write(oprot) oprot.writeMessageEnd() oprot.trans.flush() - def process_markPartitionForEvent(self, seqid, iprot, oprot): - args = markPartitionForEvent_args() + def process_append_partition_by_name(self, seqid, iprot, oprot): + args = append_partition_by_name_args() args.read(iprot) iprot.readMessageEnd() - result = markPartitionForEvent_result() + result = append_partition_by_name_result() try: - self._handler.markPartitionForEvent(args.db_name, args.tbl_name, args.part_vals, args.eventType) - except MetaException as o1: + result.success = self._handler.append_partition_by_name(args.db_name, args.tbl_name, args.part_name) + except InvalidObjectException as o1: result.o1 = o1 - except NoSuchObjectException as o2: + except AlreadyExistsException as o2: result.o2 = o2 - except UnknownDBException as o3: + except MetaException as o3: result.o3 = o3 - except UnknownTableException as o4: - result.o4 = o4 - except UnknownPartitionException as o5: - result.o5 = o5 - except InvalidPartitionException as o6: - result.o6 = o6 - oprot.writeMessageBegin("markPartitionForEvent", TMessageType.REPLY, seqid) + oprot.writeMessageBegin("append_partition_by_name", TMessageType.REPLY, seqid) result.write(oprot) oprot.writeMessageEnd() oprot.trans.flush() - def process_isPartitionMarkedForEvent(self, seqid, iprot, oprot): - args = isPartitionMarkedForEvent_args() + def process_append_partition_by_name_with_environment_context(self, seqid, iprot, oprot): + args = append_partition_by_name_with_environment_context_args() args.read(iprot) iprot.readMessageEnd() - result = isPartitionMarkedForEvent_result() + result = append_partition_by_name_with_environment_context_result() try: - result.success = self._handler.isPartitionMarkedForEvent(args.db_name, args.tbl_name, args.part_vals, args.eventType) - except MetaException as o1: + result.success = self._handler.append_partition_by_name_with_environment_context(args.db_name, args.tbl_name, args.part_name, args.environment_context) + except InvalidObjectException as o1: result.o1 = o1 - except NoSuchObjectException as o2: + except AlreadyExistsException as o2: result.o2 = o2 - except UnknownDBException as o3: + except MetaException as o3: result.o3 = o3 - except UnknownTableException as o4: - result.o4 = o4 - except UnknownPartitionException as o5: - result.o5 = o5 - except InvalidPartitionException as o6: - result.o6 = o6 - oprot.writeMessageBegin("isPartitionMarkedForEvent", TMessageType.REPLY, seqid) + oprot.writeMessageBegin("append_partition_by_name_with_environment_context", TMessageType.REPLY, seqid) result.write(oprot) oprot.writeMessageEnd() oprot.trans.flush() - def process_add_index(self, seqid, iprot, oprot): - args = add_index_args() + def process_drop_partition(self, seqid, iprot, oprot): + args = drop_partition_args() args.read(iprot) iprot.readMessageEnd() - result = add_index_result() + result = drop_partition_result() try: - result.success = self._handler.add_index(args.new_index, args.index_table) - except InvalidObjectException as o1: + result.success = self._handler.drop_partition(args.db_name, args.tbl_name, args.part_vals, args.deleteData) + except NoSuchObjectException as o1: result.o1 = o1 - except AlreadyExistsException as o2: + except MetaException as o2: result.o2 = o2 - except MetaException as o3: - result.o3 = o3 - oprot.writeMessageBegin("add_index", TMessageType.REPLY, seqid) + oprot.writeMessageBegin("drop_partition", TMessageType.REPLY, seqid) result.write(oprot) oprot.writeMessageEnd() oprot.trans.flush() - def process_alter_index(self, seqid, iprot, oprot): - args = alter_index_args() + def process_drop_partition_with_environment_context(self, seqid, iprot, oprot): + args = drop_partition_with_environment_context_args() args.read(iprot) iprot.readMessageEnd() - result = alter_index_result() + result = drop_partition_with_environment_context_result() try: - self._handler.alter_index(args.dbname, args.base_tbl_name, args.idx_name, args.new_idx) - except InvalidOperationException as o1: + result.success = self._handler.drop_partition_with_environment_context(args.db_name, args.tbl_name, args.part_vals, args.deleteData, args.environment_context) + except NoSuchObjectException as o1: result.o1 = o1 except MetaException as o2: result.o2 = o2 - oprot.writeMessageBegin("alter_index", TMessageType.REPLY, seqid) + oprot.writeMessageBegin("drop_partition_with_environment_context", TMessageType.REPLY, seqid) result.write(oprot) oprot.writeMessageEnd() oprot.trans.flush() - def process_drop_index_by_name(self, seqid, iprot, oprot): - args = drop_index_by_name_args() + def process_drop_partition_by_name(self, seqid, iprot, oprot): + args = drop_partition_by_name_args() args.read(iprot) iprot.readMessageEnd() - result = drop_index_by_name_result() + result = drop_partition_by_name_result() try: - result.success = self._handler.drop_index_by_name(args.db_name, args.tbl_name, args.index_name, args.deleteData) + result.success = self._handler.drop_partition_by_name(args.db_name, args.tbl_name, args.part_name, args.deleteData) except NoSuchObjectException as o1: result.o1 = o1 except MetaException as o2: result.o2 = o2 - oprot.writeMessageBegin("drop_index_by_name", TMessageType.REPLY, seqid) + oprot.writeMessageBegin("drop_partition_by_name", TMessageType.REPLY, seqid) result.write(oprot) oprot.writeMessageEnd() oprot.trans.flush() - def process_get_index_by_name(self, seqid, iprot, oprot): - args = get_index_by_name_args() + def process_drop_partition_by_name_with_environment_context(self, seqid, iprot, oprot): + args = drop_partition_by_name_with_environment_context_args() args.read(iprot) iprot.readMessageEnd() - result = get_index_by_name_result() + result = drop_partition_by_name_with_environment_context_result() try: - result.success = self._handler.get_index_by_name(args.db_name, args.tbl_name, args.index_name) - except MetaException as o1: + result.success = self._handler.drop_partition_by_name_with_environment_context(args.db_name, args.tbl_name, args.part_name, args.deleteData, args.environment_context) + except NoSuchObjectException as o1: result.o1 = o1 - except NoSuchObjectException as o2: + except MetaException as o2: result.o2 = o2 - oprot.writeMessageBegin("get_index_by_name", TMessageType.REPLY, seqid) + oprot.writeMessageBegin("drop_partition_by_name_with_environment_context", TMessageType.REPLY, seqid) result.write(oprot) oprot.writeMessageEnd() oprot.trans.flush() - def process_get_indexes(self, seqid, iprot, oprot): - args = get_indexes_args() + def process_drop_partitions_req(self, seqid, iprot, oprot): + args = drop_partitions_req_args() args.read(iprot) iprot.readMessageEnd() - result = get_indexes_result() + result = drop_partitions_req_result() try: - result.success = self._handler.get_indexes(args.db_name, args.tbl_name, args.max_indexes) + result.success = self._handler.drop_partitions_req(args.req) except NoSuchObjectException as o1: result.o1 = o1 except MetaException as o2: result.o2 = o2 - oprot.writeMessageBegin("get_indexes", TMessageType.REPLY, seqid) + oprot.writeMessageBegin("drop_partitions_req", TMessageType.REPLY, seqid) result.write(oprot) oprot.writeMessageEnd() oprot.trans.flush() - def process_get_index_names(self, seqid, iprot, oprot): - args = get_index_names_args() + def process_get_partition(self, seqid, iprot, oprot): + args = get_partition_args() args.read(iprot) iprot.readMessageEnd() - result = get_index_names_result() + result = get_partition_result() try: - result.success = self._handler.get_index_names(args.db_name, args.tbl_name, args.max_indexes) - except MetaException as o2: + result.success = self._handler.get_partition(args.db_name, args.tbl_name, args.part_vals) + except MetaException as o1: + result.o1 = o1 + except NoSuchObjectException as o2: result.o2 = o2 - oprot.writeMessageBegin("get_index_names", TMessageType.REPLY, seqid) + oprot.writeMessageBegin("get_partition", TMessageType.REPLY, seqid) result.write(oprot) oprot.writeMessageEnd() oprot.trans.flush() - def process_update_table_column_statistics(self, seqid, iprot, oprot): - args = update_table_column_statistics_args() + def process_exchange_partition(self, seqid, iprot, oprot): + args = exchange_partition_args() args.read(iprot) iprot.readMessageEnd() - result = update_table_column_statistics_result() + result = exchange_partition_result() try: - result.success = self._handler.update_table_column_statistics(args.stats_obj) - except NoSuchObjectException as o1: + result.success = self._handler.exchange_partition(args.partitionSpecs, args.source_db, args.source_table_name, args.dest_db, args.dest_table_name) + except MetaException as o1: result.o1 = o1 - except InvalidObjectException as o2: + except NoSuchObjectException as o2: result.o2 = o2 - except MetaException as o3: + except InvalidObjectException as o3: result.o3 = o3 except InvalidInputException as o4: result.o4 = o4 - oprot.writeMessageBegin("update_table_column_statistics", TMessageType.REPLY, seqid) + oprot.writeMessageBegin("exchange_partition", TMessageType.REPLY, seqid) result.write(oprot) oprot.writeMessageEnd() oprot.trans.flush() - def process_update_partition_column_statistics(self, seqid, iprot, oprot): - args = update_partition_column_statistics_args() + def process_get_partition_with_auth(self, seqid, iprot, oprot): + args = get_partition_with_auth_args() args.read(iprot) iprot.readMessageEnd() - result = update_partition_column_statistics_result() + result = get_partition_with_auth_result() try: - result.success = self._handler.update_partition_column_statistics(args.stats_obj) - except NoSuchObjectException as o1: + result.success = self._handler.get_partition_with_auth(args.db_name, args.tbl_name, args.part_vals, args.user_name, args.group_names) + except MetaException as o1: result.o1 = o1 - except InvalidObjectException as o2: + except NoSuchObjectException as o2: result.o2 = o2 - except MetaException as o3: - result.o3 = o3 - except InvalidInputException as o4: - result.o4 = o4 - oprot.writeMessageBegin("update_partition_column_statistics", TMessageType.REPLY, seqid) + oprot.writeMessageBegin("get_partition_with_auth", TMessageType.REPLY, seqid) result.write(oprot) oprot.writeMessageEnd() oprot.trans.flush() - def process_get_table_column_statistics(self, seqid, iprot, oprot): - args = get_table_column_statistics_args() + def process_get_partition_by_name(self, seqid, iprot, oprot): + args = get_partition_by_name_args() args.read(iprot) iprot.readMessageEnd() - result = get_table_column_statistics_result() + result = get_partition_by_name_result() try: - result.success = self._handler.get_table_column_statistics(args.db_name, args.tbl_name, args.col_name) - except NoSuchObjectException as o1: + result.success = self._handler.get_partition_by_name(args.db_name, args.tbl_name, args.part_name) + except MetaException as o1: result.o1 = o1 - except MetaException as o2: + except NoSuchObjectException as o2: result.o2 = o2 - except InvalidInputException as o3: - result.o3 = o3 - except InvalidObjectException as o4: - result.o4 = o4 - oprot.writeMessageBegin("get_table_column_statistics", TMessageType.REPLY, seqid) + oprot.writeMessageBegin("get_partition_by_name", TMessageType.REPLY, seqid) result.write(oprot) oprot.writeMessageEnd() oprot.trans.flush() - def process_get_partition_column_statistics(self, seqid, iprot, oprot): - args = get_partition_column_statistics_args() + def process_get_partitions(self, seqid, iprot, oprot): + args = get_partitions_args() args.read(iprot) iprot.readMessageEnd() - result = get_partition_column_statistics_result() + result = get_partitions_result() try: - result.success = self._handler.get_partition_column_statistics(args.db_name, args.tbl_name, args.part_name, args.col_name) + result.success = self._handler.get_partitions(args.db_name, args.tbl_name, args.max_parts) except NoSuchObjectException as o1: result.o1 = o1 except MetaException as o2: result.o2 = o2 - except InvalidInputException as o3: - result.o3 = o3 - except InvalidObjectException as o4: - result.o4 = o4 - oprot.writeMessageBegin("get_partition_column_statistics", TMessageType.REPLY, seqid) + oprot.writeMessageBegin("get_partitions", TMessageType.REPLY, seqid) result.write(oprot) oprot.writeMessageEnd() oprot.trans.flush() - def process_get_table_statistics_req(self, seqid, iprot, oprot): - args = get_table_statistics_req_args() + def process_get_partitions_with_auth(self, seqid, iprot, oprot): + args = get_partitions_with_auth_args() args.read(iprot) iprot.readMessageEnd() - result = get_table_statistics_req_result() + result = get_partitions_with_auth_result() try: - result.success = self._handler.get_table_statistics_req(args.request) + result.success = self._handler.get_partitions_with_auth(args.db_name, args.tbl_name, args.max_parts, args.user_name, args.group_names) except NoSuchObjectException as o1: result.o1 = o1 except MetaException as o2: result.o2 = o2 - oprot.writeMessageBegin("get_table_statistics_req", TMessageType.REPLY, seqid) + oprot.writeMessageBegin("get_partitions_with_auth", TMessageType.REPLY, seqid) result.write(oprot) oprot.writeMessageEnd() oprot.trans.flush() - def process_get_partitions_statistics_req(self, seqid, iprot, oprot): - args = get_partitions_statistics_req_args() + def process_get_partition_names(self, seqid, iprot, oprot): + args = get_partition_names_args() args.read(iprot) iprot.readMessageEnd() - result = get_partitions_statistics_req_result() + result = get_partition_names_result() try: - result.success = self._handler.get_partitions_statistics_req(args.request) + result.success = self._handler.get_partition_names(args.db_name, args.tbl_name, args.max_parts) + except MetaException as o2: + result.o2 = o2 + oprot.writeMessageBegin("get_partition_names", TMessageType.REPLY, seqid) + result.write(oprot) + oprot.writeMessageEnd() + oprot.trans.flush() + + def process_get_partitions_ps(self, seqid, iprot, oprot): + args = get_partitions_ps_args() + args.read(iprot) + iprot.readMessageEnd() + result = get_partitions_ps_result() + try: + result.success = self._handler.get_partitions_ps(args.db_name, args.tbl_name, args.part_vals, args.max_parts) + except MetaException as o1: + result.o1 = o1 + except NoSuchObjectException as o2: + result.o2 = o2 + oprot.writeMessageBegin("get_partitions_ps", TMessageType.REPLY, seqid) + result.write(oprot) + oprot.writeMessageEnd() + oprot.trans.flush() + + def process_get_partitions_ps_with_auth(self, seqid, iprot, oprot): + args = get_partitions_ps_with_auth_args() + args.read(iprot) + iprot.readMessageEnd() + result = get_partitions_ps_with_auth_result() + try: + result.success = self._handler.get_partitions_ps_with_auth(args.db_name, args.tbl_name, args.part_vals, args.max_parts, args.user_name, args.group_names) except NoSuchObjectException as o1: result.o1 = o1 except MetaException as o2: result.o2 = o2 - oprot.writeMessageBegin("get_partitions_statistics_req", TMessageType.REPLY, seqid) + oprot.writeMessageBegin("get_partitions_ps_with_auth", TMessageType.REPLY, seqid) result.write(oprot) oprot.writeMessageEnd() oprot.trans.flush() - def process_delete_partition_column_statistics(self, seqid, iprot, oprot): - args = delete_partition_column_statistics_args() + def process_get_partition_names_ps(self, seqid, iprot, oprot): + args = get_partition_names_ps_args() args.read(iprot) iprot.readMessageEnd() - result = delete_partition_column_statistics_result() + result = get_partition_names_ps_result() try: - result.success = self._handler.delete_partition_column_statistics(args.db_name, args.tbl_name, args.part_name, args.col_name) - except NoSuchObjectException as o1: + result.success = self._handler.get_partition_names_ps(args.db_name, args.tbl_name, args.part_vals, args.max_parts) + except MetaException as o1: + result.o1 = o1 + except NoSuchObjectException as o2: + result.o2 = o2 + oprot.writeMessageBegin("get_partition_names_ps", TMessageType.REPLY, seqid) + result.write(oprot) + oprot.writeMessageEnd() + oprot.trans.flush() + + def process_get_partitions_by_filter(self, seqid, iprot, oprot): + args = get_partitions_by_filter_args() + args.read(iprot) + iprot.readMessageEnd() + result = get_partitions_by_filter_result() + try: + result.success = self._handler.get_partitions_by_filter(args.db_name, args.tbl_name, args.filter, args.max_parts) + except MetaException as o1: + result.o1 = o1 + except NoSuchObjectException as o2: + result.o2 = o2 + oprot.writeMessageBegin("get_partitions_by_filter", TMessageType.REPLY, seqid) + result.write(oprot) + oprot.writeMessageEnd() + oprot.trans.flush() + + def process_get_partitions_by_expr(self, seqid, iprot, oprot): + args = get_partitions_by_expr_args() + args.read(iprot) + iprot.readMessageEnd() + result = get_partitions_by_expr_result() + try: + result.success = self._handler.get_partitions_by_expr(args.req) + except MetaException as o1: + result.o1 = o1 + except NoSuchObjectException as o2: + result.o2 = o2 + oprot.writeMessageBegin("get_partitions_by_expr", TMessageType.REPLY, seqid) + result.write(oprot) + oprot.writeMessageEnd() + oprot.trans.flush() + + def process_get_partitions_by_names(self, seqid, iprot, oprot): + args = get_partitions_by_names_args() + args.read(iprot) + iprot.readMessageEnd() + result = get_partitions_by_names_result() + try: + result.success = self._handler.get_partitions_by_names(args.db_name, args.tbl_name, args.names) + except MetaException as o1: + result.o1 = o1 + except NoSuchObjectException as o2: + result.o2 = o2 + oprot.writeMessageBegin("get_partitions_by_names", TMessageType.REPLY, seqid) + result.write(oprot) + oprot.writeMessageEnd() + oprot.trans.flush() + + def process_alter_partition(self, seqid, iprot, oprot): + args = alter_partition_args() + args.read(iprot) + iprot.readMessageEnd() + result = alter_partition_result() + try: + self._handler.alter_partition(args.db_name, args.tbl_name, args.new_part) + except InvalidOperationException as o1: result.o1 = o1 except MetaException as o2: result.o2 = o2 - except InvalidObjectException as o3: - result.o3 = o3 - except InvalidInputException as o4: - result.o4 = o4 - oprot.writeMessageBegin("delete_partition_column_statistics", TMessageType.REPLY, seqid) + oprot.writeMessageBegin("alter_partition", TMessageType.REPLY, seqid) + result.write(oprot) + oprot.writeMessageEnd() + oprot.trans.flush() + + def process_alter_partitions(self, seqid, iprot, oprot): + args = alter_partitions_args() + args.read(iprot) + iprot.readMessageEnd() + result = alter_partitions_result() + try: + self._handler.alter_partitions(args.db_name, args.tbl_name, args.new_parts) + except InvalidOperationException as o1: + result.o1 = o1 + except MetaException as o2: + result.o2 = o2 + oprot.writeMessageBegin("alter_partitions", TMessageType.REPLY, seqid) + result.write(oprot) + oprot.writeMessageEnd() + oprot.trans.flush() + + def process_alter_partition_with_environment_context(self, seqid, iprot, oprot): + args = alter_partition_with_environment_context_args() + args.read(iprot) + iprot.readMessageEnd() + result = alter_partition_with_environment_context_result() + try: + self._handler.alter_partition_with_environment_context(args.db_name, args.tbl_name, args.new_part, args.environment_context) + except InvalidOperationException as o1: + result.o1 = o1 + except MetaException as o2: + result.o2 = o2 + oprot.writeMessageBegin("alter_partition_with_environment_context", TMessageType.REPLY, seqid) + result.write(oprot) + oprot.writeMessageEnd() + oprot.trans.flush() + + def process_rename_partition(self, seqid, iprot, oprot): + args = rename_partition_args() + args.read(iprot) + iprot.readMessageEnd() + result = rename_partition_result() + try: + self._handler.rename_partition(args.db_name, args.tbl_name, args.part_vals, args.new_part) + except InvalidOperationException as o1: + result.o1 = o1 + except MetaException as o2: + result.o2 = o2 + oprot.writeMessageBegin("rename_partition", TMessageType.REPLY, seqid) + result.write(oprot) + oprot.writeMessageEnd() + oprot.trans.flush() + + def process_partition_name_has_valid_characters(self, seqid, iprot, oprot): + args = partition_name_has_valid_characters_args() + args.read(iprot) + iprot.readMessageEnd() + result = partition_name_has_valid_characters_result() + try: + result.success = self._handler.partition_name_has_valid_characters(args.part_vals, args.throw_exception) + except MetaException as o1: + result.o1 = o1 + oprot.writeMessageBegin("partition_name_has_valid_characters", TMessageType.REPLY, seqid) + result.write(oprot) + oprot.writeMessageEnd() + oprot.trans.flush() + + def process_get_config_value(self, seqid, iprot, oprot): + args = get_config_value_args() + args.read(iprot) + iprot.readMessageEnd() + result = get_config_value_result() + try: + result.success = self._handler.get_config_value(args.name, args.defaultValue) + except ConfigValSecurityException as o1: + result.o1 = o1 + oprot.writeMessageBegin("get_config_value", TMessageType.REPLY, seqid) + result.write(oprot) + oprot.writeMessageEnd() + oprot.trans.flush() + + def process_partition_name_to_vals(self, seqid, iprot, oprot): + args = partition_name_to_vals_args() + args.read(iprot) + iprot.readMessageEnd() + result = partition_name_to_vals_result() + try: + result.success = self._handler.partition_name_to_vals(args.part_name) + except MetaException as o1: + result.o1 = o1 + oprot.writeMessageBegin("partition_name_to_vals", TMessageType.REPLY, seqid) + result.write(oprot) + oprot.writeMessageEnd() + oprot.trans.flush() + + def process_partition_name_to_spec(self, seqid, iprot, oprot): + args = partition_name_to_spec_args() + args.read(iprot) + iprot.readMessageEnd() + result = partition_name_to_spec_result() + try: + result.success = self._handler.partition_name_to_spec(args.part_name) + except MetaException as o1: + result.o1 = o1 + oprot.writeMessageBegin("partition_name_to_spec", TMessageType.REPLY, seqid) result.write(oprot) oprot.writeMessageEnd() oprot.trans.flush() - def process_delete_table_column_statistics(self, seqid, iprot, oprot): - args = delete_table_column_statistics_args() - args.read(iprot) - iprot.readMessageEnd() - result = delete_table_column_statistics_result() - try: - result.success = self._handler.delete_table_column_statistics(args.db_name, args.tbl_name, args.col_name) - except NoSuchObjectException as o1: - result.o1 = o1 - except MetaException as o2: - result.o2 = o2 - except InvalidObjectException as o3: - result.o3 = o3 - except InvalidInputException as o4: - result.o4 = o4 - oprot.writeMessageBegin("delete_table_column_statistics", TMessageType.REPLY, seqid) - result.write(oprot) - oprot.writeMessageEnd() - oprot.trans.flush() + def process_markPartitionForEvent(self, seqid, iprot, oprot): + args = markPartitionForEvent_args() + args.read(iprot) + iprot.readMessageEnd() + result = markPartitionForEvent_result() + try: + self._handler.markPartitionForEvent(args.db_name, args.tbl_name, args.part_vals, args.eventType) + except MetaException as o1: + result.o1 = o1 + except NoSuchObjectException as o2: + result.o2 = o2 + except UnknownDBException as o3: + result.o3 = o3 + except UnknownTableException as o4: + result.o4 = o4 + except UnknownPartitionException as o5: + result.o5 = o5 + except InvalidPartitionException as o6: + result.o6 = o6 + oprot.writeMessageBegin("markPartitionForEvent", TMessageType.REPLY, seqid) + result.write(oprot) + oprot.writeMessageEnd() + oprot.trans.flush() + + def process_isPartitionMarkedForEvent(self, seqid, iprot, oprot): + args = isPartitionMarkedForEvent_args() + args.read(iprot) + iprot.readMessageEnd() + result = isPartitionMarkedForEvent_result() + try: + result.success = self._handler.isPartitionMarkedForEvent(args.db_name, args.tbl_name, args.part_vals, args.eventType) + except MetaException as o1: + result.o1 = o1 + except NoSuchObjectException as o2: + result.o2 = o2 + except UnknownDBException as o3: + result.o3 = o3 + except UnknownTableException as o4: + result.o4 = o4 + except UnknownPartitionException as o5: + result.o5 = o5 + except InvalidPartitionException as o6: + result.o6 = o6 + oprot.writeMessageBegin("isPartitionMarkedForEvent", TMessageType.REPLY, seqid) + result.write(oprot) + oprot.writeMessageEnd() + oprot.trans.flush() + + def process_add_index(self, seqid, iprot, oprot): + args = add_index_args() + args.read(iprot) + iprot.readMessageEnd() + result = add_index_result() + try: + result.success = self._handler.add_index(args.new_index, args.index_table) + except InvalidObjectException as o1: + result.o1 = o1 + except AlreadyExistsException as o2: + result.o2 = o2 + except MetaException as o3: + result.o3 = o3 + oprot.writeMessageBegin("add_index", TMessageType.REPLY, seqid) + result.write(oprot) + oprot.writeMessageEnd() + oprot.trans.flush() + + def process_alter_index(self, seqid, iprot, oprot): + args = alter_index_args() + args.read(iprot) + iprot.readMessageEnd() + result = alter_index_result() + try: + self._handler.alter_index(args.dbname, args.base_tbl_name, args.idx_name, args.new_idx) + except InvalidOperationException as o1: + result.o1 = o1 + except MetaException as o2: + result.o2 = o2 + oprot.writeMessageBegin("alter_index", TMessageType.REPLY, seqid) + result.write(oprot) + oprot.writeMessageEnd() + oprot.trans.flush() + + def process_drop_index_by_name(self, seqid, iprot, oprot): + args = drop_index_by_name_args() + args.read(iprot) + iprot.readMessageEnd() + result = drop_index_by_name_result() + try: + result.success = self._handler.drop_index_by_name(args.db_name, args.tbl_name, args.index_name, args.deleteData) + except NoSuchObjectException as o1: + result.o1 = o1 + except MetaException as o2: + result.o2 = o2 + oprot.writeMessageBegin("drop_index_by_name", TMessageType.REPLY, seqid) + result.write(oprot) + oprot.writeMessageEnd() + oprot.trans.flush() + + def process_get_index_by_name(self, seqid, iprot, oprot): + args = get_index_by_name_args() + args.read(iprot) + iprot.readMessageEnd() + result = get_index_by_name_result() + try: + result.success = self._handler.get_index_by_name(args.db_name, args.tbl_name, args.index_name) + except MetaException as o1: + result.o1 = o1 + except NoSuchObjectException as o2: + result.o2 = o2 + oprot.writeMessageBegin("get_index_by_name", TMessageType.REPLY, seqid) + result.write(oprot) + oprot.writeMessageEnd() + oprot.trans.flush() + + def process_get_indexes(self, seqid, iprot, oprot): + args = get_indexes_args() + args.read(iprot) + iprot.readMessageEnd() + result = get_indexes_result() + try: + result.success = self._handler.get_indexes(args.db_name, args.tbl_name, args.max_indexes) + except NoSuchObjectException as o1: + result.o1 = o1 + except MetaException as o2: + result.o2 = o2 + oprot.writeMessageBegin("get_indexes", TMessageType.REPLY, seqid) + result.write(oprot) + oprot.writeMessageEnd() + oprot.trans.flush() + + def process_get_index_names(self, seqid, iprot, oprot): + args = get_index_names_args() + args.read(iprot) + iprot.readMessageEnd() + result = get_index_names_result() + try: + result.success = self._handler.get_index_names(args.db_name, args.tbl_name, args.max_indexes) + except MetaException as o2: + result.o2 = o2 + oprot.writeMessageBegin("get_index_names", TMessageType.REPLY, seqid) + result.write(oprot) + oprot.writeMessageEnd() + oprot.trans.flush() + + def process_update_table_column_statistics(self, seqid, iprot, oprot): + args = update_table_column_statistics_args() + args.read(iprot) + iprot.readMessageEnd() + result = update_table_column_statistics_result() + try: + result.success = self._handler.update_table_column_statistics(args.stats_obj) + except NoSuchObjectException as o1: + result.o1 = o1 + except InvalidObjectException as o2: + result.o2 = o2 + except MetaException as o3: + result.o3 = o3 + except InvalidInputException as o4: + result.o4 = o4 + oprot.writeMessageBegin("update_table_column_statistics", TMessageType.REPLY, seqid) + result.write(oprot) + oprot.writeMessageEnd() + oprot.trans.flush() + + def process_update_partition_column_statistics(self, seqid, iprot, oprot): + args = update_partition_column_statistics_args() + args.read(iprot) + iprot.readMessageEnd() + result = update_partition_column_statistics_result() + try: + result.success = self._handler.update_partition_column_statistics(args.stats_obj) + except NoSuchObjectException as o1: + result.o1 = o1 + except InvalidObjectException as o2: + result.o2 = o2 + except MetaException as o3: + result.o3 = o3 + except InvalidInputException as o4: + result.o4 = o4 + oprot.writeMessageBegin("update_partition_column_statistics", TMessageType.REPLY, seqid) + result.write(oprot) + oprot.writeMessageEnd() + oprot.trans.flush() + + def process_get_table_column_statistics(self, seqid, iprot, oprot): + args = get_table_column_statistics_args() + args.read(iprot) + iprot.readMessageEnd() + result = get_table_column_statistics_result() + try: + result.success = self._handler.get_table_column_statistics(args.db_name, args.tbl_name, args.col_name) + except NoSuchObjectException as o1: + result.o1 = o1 + except MetaException as o2: + result.o2 = o2 + except InvalidInputException as o3: + result.o3 = o3 + except InvalidObjectException as o4: + result.o4 = o4 + oprot.writeMessageBegin("get_table_column_statistics", TMessageType.REPLY, seqid) + result.write(oprot) + oprot.writeMessageEnd() + oprot.trans.flush() + + def process_get_partition_column_statistics(self, seqid, iprot, oprot): + args = get_partition_column_statistics_args() + args.read(iprot) + iprot.readMessageEnd() + result = get_partition_column_statistics_result() + try: + result.success = self._handler.get_partition_column_statistics(args.db_name, args.tbl_name, args.part_name, args.col_name) + except NoSuchObjectException as o1: + result.o1 = o1 + except MetaException as o2: + result.o2 = o2 + except InvalidInputException as o3: + result.o3 = o3 + except InvalidObjectException as o4: + result.o4 = o4 + oprot.writeMessageBegin("get_partition_column_statistics", TMessageType.REPLY, seqid) + result.write(oprot) + oprot.writeMessageEnd() + oprot.trans.flush() + + def process_get_table_statistics_req(self, seqid, iprot, oprot): + args = get_table_statistics_req_args() + args.read(iprot) + iprot.readMessageEnd() + result = get_table_statistics_req_result() + try: + result.success = self._handler.get_table_statistics_req(args.request) + except NoSuchObjectException as o1: + result.o1 = o1 + except MetaException as o2: + result.o2 = o2 + oprot.writeMessageBegin("get_table_statistics_req", TMessageType.REPLY, seqid) + result.write(oprot) + oprot.writeMessageEnd() + oprot.trans.flush() + + def process_get_partitions_statistics_req(self, seqid, iprot, oprot): + args = get_partitions_statistics_req_args() + args.read(iprot) + iprot.readMessageEnd() + result = get_partitions_statistics_req_result() + try: + result.success = self._handler.get_partitions_statistics_req(args.request) + except NoSuchObjectException as o1: + result.o1 = o1 + except MetaException as o2: + result.o2 = o2 + oprot.writeMessageBegin("get_partitions_statistics_req", TMessageType.REPLY, seqid) + result.write(oprot) + oprot.writeMessageEnd() + oprot.trans.flush() + + def process_delete_partition_column_statistics(self, seqid, iprot, oprot): + args = delete_partition_column_statistics_args() + args.read(iprot) + iprot.readMessageEnd() + result = delete_partition_column_statistics_result() + try: + result.success = self._handler.delete_partition_column_statistics(args.db_name, args.tbl_name, args.part_name, args.col_name) + except NoSuchObjectException as o1: + result.o1 = o1 + except MetaException as o2: + result.o2 = o2 + except InvalidObjectException as o3: + result.o3 = o3 + except InvalidInputException as o4: + result.o4 = o4 + oprot.writeMessageBegin("delete_partition_column_statistics", TMessageType.REPLY, seqid) + result.write(oprot) + oprot.writeMessageEnd() + oprot.trans.flush() + + def process_delete_table_column_statistics(self, seqid, iprot, oprot): + args = delete_table_column_statistics_args() + args.read(iprot) + iprot.readMessageEnd() + result = delete_table_column_statistics_result() + try: + result.success = self._handler.delete_table_column_statistics(args.db_name, args.tbl_name, args.col_name) + except NoSuchObjectException as o1: + result.o1 = o1 + except MetaException as o2: + result.o2 = o2 + except InvalidObjectException as o3: + result.o3 = o3 + except InvalidInputException as o4: + result.o4 = o4 + oprot.writeMessageBegin("delete_table_column_statistics", TMessageType.REPLY, seqid) + result.write(oprot) + oprot.writeMessageEnd() + oprot.trans.flush() + + def process_create_function(self, seqid, iprot, oprot): + args = create_function_args() + args.read(iprot) + iprot.readMessageEnd() + result = create_function_result() + try: + self._handler.create_function(args.func) + except AlreadyExistsException as o1: + result.o1 = o1 + except InvalidObjectException as o2: + result.o2 = o2 + except MetaException as o3: + result.o3 = o3 + except NoSuchObjectException as o4: + result.o4 = o4 + oprot.writeMessageBegin("create_function", TMessageType.REPLY, seqid) + result.write(oprot) + oprot.writeMessageEnd() + oprot.trans.flush() + + def process_drop_function(self, seqid, iprot, oprot): + args = drop_function_args() + args.read(iprot) + iprot.readMessageEnd() + result = drop_function_result() + try: + self._handler.drop_function(args.dbName, args.funcName) + except NoSuchObjectException as o1: + result.o1 = o1 + except MetaException as o3: + result.o3 = o3 + oprot.writeMessageBegin("drop_function", TMessageType.REPLY, seqid) + result.write(oprot) + oprot.writeMessageEnd() + oprot.trans.flush() + + def process_alter_function(self, seqid, iprot, oprot): + args = alter_function_args() + args.read(iprot) + iprot.readMessageEnd() + result = alter_function_result() + try: + self._handler.alter_function(args.dbName, args.funcName, args.newFunc) + except InvalidOperationException as o1: + result.o1 = o1 + except MetaException as o2: + result.o2 = o2 + oprot.writeMessageBegin("alter_function", TMessageType.REPLY, seqid) + result.write(oprot) + oprot.writeMessageEnd() + oprot.trans.flush() + + def process_get_functions(self, seqid, iprot, oprot): + args = get_functions_args() + args.read(iprot) + iprot.readMessageEnd() + result = get_functions_result() + try: + result.success = self._handler.get_functions(args.dbName, args.pattern) + except MetaException as o1: + result.o1 = o1 + oprot.writeMessageBegin("get_functions", TMessageType.REPLY, seqid) + result.write(oprot) + oprot.writeMessageEnd() + oprot.trans.flush() + + def process_get_function(self, seqid, iprot, oprot): + args = get_function_args() + args.read(iprot) + iprot.readMessageEnd() + result = get_function_result() + try: + result.success = self._handler.get_function(args.dbName, args.funcName) + except MetaException as o1: + result.o1 = o1 + except NoSuchObjectException as o2: + result.o2 = o2 + oprot.writeMessageBegin("get_function", TMessageType.REPLY, seqid) + result.write(oprot) + oprot.writeMessageEnd() + oprot.trans.flush() + + def process_create_role(self, seqid, iprot, oprot): + args = create_role_args() + args.read(iprot) + iprot.readMessageEnd() + result = create_role_result() + try: + result.success = self._handler.create_role(args.role) + except MetaException as o1: + result.o1 = o1 + oprot.writeMessageBegin("create_role", TMessageType.REPLY, seqid) + result.write(oprot) + oprot.writeMessageEnd() + oprot.trans.flush() + + def process_drop_role(self, seqid, iprot, oprot): + args = drop_role_args() + args.read(iprot) + iprot.readMessageEnd() + result = drop_role_result() + try: + result.success = self._handler.drop_role(args.role_name) + except MetaException as o1: + result.o1 = o1 + oprot.writeMessageBegin("drop_role", TMessageType.REPLY, seqid) + result.write(oprot) + oprot.writeMessageEnd() + oprot.trans.flush() + + def process_get_role_names(self, seqid, iprot, oprot): + args = get_role_names_args() + args.read(iprot) + iprot.readMessageEnd() + result = get_role_names_result() + try: + result.success = self._handler.get_role_names() + except MetaException as o1: + result.o1 = o1 + oprot.writeMessageBegin("get_role_names", TMessageType.REPLY, seqid) + result.write(oprot) + oprot.writeMessageEnd() + oprot.trans.flush() + + def process_grant_role(self, seqid, iprot, oprot): + args = grant_role_args() + args.read(iprot) + iprot.readMessageEnd() + result = grant_role_result() + try: + result.success = self._handler.grant_role(args.role_name, args.principal_name, args.principal_type, args.grantor, args.grantorType, args.grant_option) + except MetaException as o1: + result.o1 = o1 + oprot.writeMessageBegin("grant_role", TMessageType.REPLY, seqid) + result.write(oprot) + oprot.writeMessageEnd() + oprot.trans.flush() + + def process_revoke_role(self, seqid, iprot, oprot): + args = revoke_role_args() + args.read(iprot) + iprot.readMessageEnd() + result = revoke_role_result() + try: + result.success = self._handler.revoke_role(args.role_name, args.principal_name, args.principal_type) + except MetaException as o1: + result.o1 = o1 + oprot.writeMessageBegin("revoke_role", TMessageType.REPLY, seqid) + result.write(oprot) + oprot.writeMessageEnd() + oprot.trans.flush() + + def process_list_roles(self, seqid, iprot, oprot): + args = list_roles_args() + args.read(iprot) + iprot.readMessageEnd() + result = list_roles_result() + try: + result.success = self._handler.list_roles(args.principal_name, args.principal_type) + except MetaException as o1: + result.o1 = o1 + oprot.writeMessageBegin("list_roles", TMessageType.REPLY, seqid) + result.write(oprot) + oprot.writeMessageEnd() + oprot.trans.flush() + + def process_get_privilege_set(self, seqid, iprot, oprot): + args = get_privilege_set_args() + args.read(iprot) + iprot.readMessageEnd() + result = get_privilege_set_result() + try: + result.success = self._handler.get_privilege_set(args.hiveObject, args.user_name, args.group_names) + except MetaException as o1: + result.o1 = o1 + oprot.writeMessageBegin("get_privilege_set", TMessageType.REPLY, seqid) + result.write(oprot) + oprot.writeMessageEnd() + oprot.trans.flush() + + def process_list_privileges(self, seqid, iprot, oprot): + args = list_privileges_args() + args.read(iprot) + iprot.readMessageEnd() + result = list_privileges_result() + try: + result.success = self._handler.list_privileges(args.principal_name, args.principal_type, args.hiveObject) + except MetaException as o1: + result.o1 = o1 + oprot.writeMessageBegin("list_privileges", TMessageType.REPLY, seqid) + result.write(oprot) + oprot.writeMessageEnd() + oprot.trans.flush() + + def process_grant_privileges(self, seqid, iprot, oprot): + args = grant_privileges_args() + args.read(iprot) + iprot.readMessageEnd() + result = grant_privileges_result() + try: + result.success = self._handler.grant_privileges(args.privileges) + except MetaException as o1: + result.o1 = o1 + oprot.writeMessageBegin("grant_privileges", TMessageType.REPLY, seqid) + result.write(oprot) + oprot.writeMessageEnd() + oprot.trans.flush() + + def process_revoke_privileges(self, seqid, iprot, oprot): + args = revoke_privileges_args() + args.read(iprot) + iprot.readMessageEnd() + result = revoke_privileges_result() + try: + result.success = self._handler.revoke_privileges(args.privileges) + except MetaException as o1: + result.o1 = o1 + oprot.writeMessageBegin("revoke_privileges", TMessageType.REPLY, seqid) + result.write(oprot) + oprot.writeMessageEnd() + oprot.trans.flush() + + def process_set_ugi(self, seqid, iprot, oprot): + args = set_ugi_args() + args.read(iprot) + iprot.readMessageEnd() + result = set_ugi_result() + try: + result.success = self._handler.set_ugi(args.user_name, args.group_names) + except MetaException as o1: + result.o1 = o1 + oprot.writeMessageBegin("set_ugi", TMessageType.REPLY, seqid) + result.write(oprot) + oprot.writeMessageEnd() + oprot.trans.flush() + + def process_get_delegation_token(self, seqid, iprot, oprot): + args = get_delegation_token_args() + args.read(iprot) + iprot.readMessageEnd() + result = get_delegation_token_result() + try: + result.success = self._handler.get_delegation_token(args.token_owner, args.renewer_kerberos_principal_name) + except MetaException as o1: + result.o1 = o1 + oprot.writeMessageBegin("get_delegation_token", TMessageType.REPLY, seqid) + result.write(oprot) + oprot.writeMessageEnd() + oprot.trans.flush() + + def process_renew_delegation_token(self, seqid, iprot, oprot): + args = renew_delegation_token_args() + args.read(iprot) + iprot.readMessageEnd() + result = renew_delegation_token_result() + try: + result.success = self._handler.renew_delegation_token(args.token_str_form) + except MetaException as o1: + result.o1 = o1 + oprot.writeMessageBegin("renew_delegation_token", TMessageType.REPLY, seqid) + result.write(oprot) + oprot.writeMessageEnd() + oprot.trans.flush() + + def process_cancel_delegation_token(self, seqid, iprot, oprot): + args = cancel_delegation_token_args() + args.read(iprot) + iprot.readMessageEnd() + result = cancel_delegation_token_result() + try: + self._handler.cancel_delegation_token(args.token_str_form) + except MetaException as o1: + result.o1 = o1 + oprot.writeMessageBegin("cancel_delegation_token", TMessageType.REPLY, seqid) + result.write(oprot) + oprot.writeMessageEnd() + oprot.trans.flush() + + def process_get_open_txns(self, seqid, iprot, oprot): + args = get_open_txns_args() + args.read(iprot) + iprot.readMessageEnd() + result = get_open_txns_result() + result.success = self._handler.get_open_txns() + oprot.writeMessageBegin("get_open_txns", TMessageType.REPLY, seqid) + result.write(oprot) + oprot.writeMessageEnd() + oprot.trans.flush() + + def process_get_open_txns_info(self, seqid, iprot, oprot): + args = get_open_txns_info_args() + args.read(iprot) + iprot.readMessageEnd() + result = get_open_txns_info_result() + result.success = self._handler.get_open_txns_info() + oprot.writeMessageBegin("get_open_txns_info", TMessageType.REPLY, seqid) + result.write(oprot) + oprot.writeMessageEnd() + oprot.trans.flush() + + def process_open_txns(self, seqid, iprot, oprot): + args = open_txns_args() + args.read(iprot) + iprot.readMessageEnd() + result = open_txns_result() + result.success = self._handler.open_txns(args.rqst) + oprot.writeMessageBegin("open_txns", TMessageType.REPLY, seqid) + result.write(oprot) + oprot.writeMessageEnd() + oprot.trans.flush() + + def process_abort_txn(self, seqid, iprot, oprot): + args = abort_txn_args() + args.read(iprot) + iprot.readMessageEnd() + result = abort_txn_result() + try: + self._handler.abort_txn(args.rqst) + except NoSuchTxnException as o1: + result.o1 = o1 + oprot.writeMessageBegin("abort_txn", TMessageType.REPLY, seqid) + result.write(oprot) + oprot.writeMessageEnd() + oprot.trans.flush() + + def process_commit_txn(self, seqid, iprot, oprot): + args = commit_txn_args() + args.read(iprot) + iprot.readMessageEnd() + result = commit_txn_result() + try: + self._handler.commit_txn(args.rqst) + except NoSuchTxnException as o1: + result.o1 = o1 + except TxnAbortedException as o2: + result.o2 = o2 + oprot.writeMessageBegin("commit_txn", TMessageType.REPLY, seqid) + result.write(oprot) + oprot.writeMessageEnd() + oprot.trans.flush() + + def process_lock(self, seqid, iprot, oprot): + args = lock_args() + args.read(iprot) + iprot.readMessageEnd() + result = lock_result() + try: + result.success = self._handler.lock(args.rqst) + except NoSuchTxnException as o1: + result.o1 = o1 + except TxnAbortedException as o2: + result.o2 = o2 + oprot.writeMessageBegin("lock", TMessageType.REPLY, seqid) + result.write(oprot) + oprot.writeMessageEnd() + oprot.trans.flush() + + def process_check_lock(self, seqid, iprot, oprot): + args = check_lock_args() + args.read(iprot) + iprot.readMessageEnd() + result = check_lock_result() + try: + result.success = self._handler.check_lock(args.rqst) + except NoSuchTxnException as o1: + result.o1 = o1 + except TxnAbortedException as o2: + result.o2 = o2 + except NoSuchLockException as o3: + result.o3 = o3 + oprot.writeMessageBegin("check_lock", TMessageType.REPLY, seqid) + result.write(oprot) + oprot.writeMessageEnd() + oprot.trans.flush() + + def process_unlock(self, seqid, iprot, oprot): + args = unlock_args() + args.read(iprot) + iprot.readMessageEnd() + result = unlock_result() + try: + self._handler.unlock(args.rqst) + except NoSuchLockException as o1: + result.o1 = o1 + except TxnOpenException as o2: + result.o2 = o2 + oprot.writeMessageBegin("unlock", TMessageType.REPLY, seqid) + result.write(oprot) + oprot.writeMessageEnd() + oprot.trans.flush() + + def process_show_locks(self, seqid, iprot, oprot): + args = show_locks_args() + args.read(iprot) + iprot.readMessageEnd() + result = show_locks_result() + result.success = self._handler.show_locks(args.rqst) + oprot.writeMessageBegin("show_locks", TMessageType.REPLY, seqid) + result.write(oprot) + oprot.writeMessageEnd() + oprot.trans.flush() + + def process_heartbeat(self, seqid, iprot, oprot): + args = heartbeat_args() + args.read(iprot) + iprot.readMessageEnd() + result = heartbeat_result() + try: + self._handler.heartbeat(args.ids) + except NoSuchLockException as o1: + result.o1 = o1 + except NoSuchTxnException as o2: + result.o2 = o2 + except TxnAbortedException as o3: + result.o3 = o3 + oprot.writeMessageBegin("heartbeat", TMessageType.REPLY, seqid) + result.write(oprot) + oprot.writeMessageEnd() + oprot.trans.flush() + + def process_compact(self, seqid, iprot, oprot): + args = compact_args() + args.read(iprot) + iprot.readMessageEnd() + result = compact_result() + self._handler.compact(args.rqst) + oprot.writeMessageBegin("compact", TMessageType.REPLY, seqid) + result.write(oprot) + oprot.writeMessageEnd() + oprot.trans.flush() + + def process_show_compact(self, seqid, iprot, oprot): + args = show_compact_args() + args.read(iprot) + iprot.readMessageEnd() + result = show_compact_result() + result.success = self._handler.show_compact(args.rqst) + oprot.writeMessageBegin("show_compact", TMessageType.REPLY, seqid) + result.write(oprot) + oprot.writeMessageEnd() + oprot.trans.flush() + + +# HELPER FUNCTIONS AND STRUCTURES + +class create_database_args: + """ + Attributes: + - database + """ + + thrift_spec = ( + None, # 0 + (1, TType.STRUCT, 'database', (Database, Database.thrift_spec), None, ), # 1 + ) + + def __init__(self, database=None,): + self.database = database + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.STRUCT: + self.database = Database() + self.database.read(iprot) + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('create_database_args') + if self.database is not None: + oprot.writeFieldBegin('database', TType.STRUCT, 1) + self.database.write(oprot) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class create_database_result: + """ + Attributes: + - o1 + - o2 + - o3 + """ + + thrift_spec = ( + None, # 0 + (1, TType.STRUCT, 'o1', (AlreadyExistsException, AlreadyExistsException.thrift_spec), None, ), # 1 + (2, TType.STRUCT, 'o2', (InvalidObjectException, InvalidObjectException.thrift_spec), None, ), # 2 + (3, TType.STRUCT, 'o3', (MetaException, MetaException.thrift_spec), None, ), # 3 + ) + + def __init__(self, o1=None, o2=None, o3=None,): + self.o1 = o1 + self.o2 = o2 + self.o3 = o3 + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.STRUCT: + self.o1 = AlreadyExistsException() + self.o1.read(iprot) + else: + iprot.skip(ftype) + elif fid == 2: + if ftype == TType.STRUCT: + self.o2 = InvalidObjectException() + self.o2.read(iprot) + else: + iprot.skip(ftype) + elif fid == 3: + if ftype == TType.STRUCT: + self.o3 = MetaException() + self.o3.read(iprot) + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('create_database_result') + if self.o1 is not None: + oprot.writeFieldBegin('o1', TType.STRUCT, 1) + self.o1.write(oprot) + oprot.writeFieldEnd() + if self.o2 is not None: + oprot.writeFieldBegin('o2', TType.STRUCT, 2) + self.o2.write(oprot) + oprot.writeFieldEnd() + if self.o3 is not None: + oprot.writeFieldBegin('o3', TType.STRUCT, 3) + self.o3.write(oprot) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class get_database_args: + """ + Attributes: + - name + """ + + thrift_spec = ( + None, # 0 + (1, TType.STRING, 'name', None, None, ), # 1 + ) + + def __init__(self, name=None,): + self.name = name + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.STRING: + self.name = iprot.readString(); + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('get_database_args') + if self.name is not None: + oprot.writeFieldBegin('name', TType.STRING, 1) + oprot.writeString(self.name) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class get_database_result: + """ + Attributes: + - success + - o1 + - o2 + """ + + thrift_spec = ( + (0, TType.STRUCT, 'success', (Database, Database.thrift_spec), None, ), # 0 + (1, TType.STRUCT, 'o1', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 1 + (2, TType.STRUCT, 'o2', (MetaException, MetaException.thrift_spec), None, ), # 2 + ) + + def __init__(self, success=None, o1=None, o2=None,): + self.success = success + self.o1 = o1 + self.o2 = o2 + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 0: + if ftype == TType.STRUCT: + self.success = Database() + self.success.read(iprot) + else: + iprot.skip(ftype) + elif fid == 1: + if ftype == TType.STRUCT: + self.o1 = NoSuchObjectException() + self.o1.read(iprot) + else: + iprot.skip(ftype) + elif fid == 2: + if ftype == TType.STRUCT: + self.o2 = MetaException() + self.o2.read(iprot) + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('get_database_result') + if self.success is not None: + oprot.writeFieldBegin('success', TType.STRUCT, 0) + self.success.write(oprot) + oprot.writeFieldEnd() + if self.o1 is not None: + oprot.writeFieldBegin('o1', TType.STRUCT, 1) + self.o1.write(oprot) + oprot.writeFieldEnd() + if self.o2 is not None: + oprot.writeFieldBegin('o2', TType.STRUCT, 2) + self.o2.write(oprot) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class drop_database_args: + """ + Attributes: + - name + - deleteData + - cascade + """ + + thrift_spec = ( + None, # 0 + (1, TType.STRING, 'name', None, None, ), # 1 + (2, TType.BOOL, 'deleteData', None, None, ), # 2 + (3, TType.BOOL, 'cascade', None, None, ), # 3 + ) + + def __init__(self, name=None, deleteData=None, cascade=None,): + self.name = name + self.deleteData = deleteData + self.cascade = cascade + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.STRING: + self.name = iprot.readString(); + else: + iprot.skip(ftype) + elif fid == 2: + if ftype == TType.BOOL: + self.deleteData = iprot.readBool(); + else: + iprot.skip(ftype) + elif fid == 3: + if ftype == TType.BOOL: + self.cascade = iprot.readBool(); + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('drop_database_args') + if self.name is not None: + oprot.writeFieldBegin('name', TType.STRING, 1) + oprot.writeString(self.name) + oprot.writeFieldEnd() + if self.deleteData is not None: + oprot.writeFieldBegin('deleteData', TType.BOOL, 2) + oprot.writeBool(self.deleteData) + oprot.writeFieldEnd() + if self.cascade is not None: + oprot.writeFieldBegin('cascade', TType.BOOL, 3) + oprot.writeBool(self.cascade) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class drop_database_result: + """ + Attributes: + - o1 + - o2 + - o3 + """ + + thrift_spec = ( + None, # 0 + (1, TType.STRUCT, 'o1', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 1 + (2, TType.STRUCT, 'o2', (InvalidOperationException, InvalidOperationException.thrift_spec), None, ), # 2 + (3, TType.STRUCT, 'o3', (MetaException, MetaException.thrift_spec), None, ), # 3 + ) + + def __init__(self, o1=None, o2=None, o3=None,): + self.o1 = o1 + self.o2 = o2 + self.o3 = o3 + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.STRUCT: + self.o1 = NoSuchObjectException() + self.o1.read(iprot) + else: + iprot.skip(ftype) + elif fid == 2: + if ftype == TType.STRUCT: + self.o2 = InvalidOperationException() + self.o2.read(iprot) + else: + iprot.skip(ftype) + elif fid == 3: + if ftype == TType.STRUCT: + self.o3 = MetaException() + self.o3.read(iprot) + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('drop_database_result') + if self.o1 is not None: + oprot.writeFieldBegin('o1', TType.STRUCT, 1) + self.o1.write(oprot) + oprot.writeFieldEnd() + if self.o2 is not None: + oprot.writeFieldBegin('o2', TType.STRUCT, 2) + self.o2.write(oprot) + oprot.writeFieldEnd() + if self.o3 is not None: + oprot.writeFieldBegin('o3', TType.STRUCT, 3) + self.o3.write(oprot) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class get_databases_args: + """ + Attributes: + - pattern + """ + + thrift_spec = ( + None, # 0 + (1, TType.STRING, 'pattern', None, None, ), # 1 + ) + + def __init__(self, pattern=None,): + self.pattern = pattern + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.STRING: + self.pattern = iprot.readString(); + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('get_databases_args') + if self.pattern is not None: + oprot.writeFieldBegin('pattern', TType.STRING, 1) + oprot.writeString(self.pattern) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class get_databases_result: + """ + Attributes: + - success + - o1 + """ + + thrift_spec = ( + (0, TType.LIST, 'success', (TType.STRING,None), None, ), # 0 + (1, TType.STRUCT, 'o1', (MetaException, MetaException.thrift_spec), None, ), # 1 + ) + + def __init__(self, success=None, o1=None,): + self.success = success + self.o1 = o1 + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 0: + if ftype == TType.LIST: + self.success = [] + (_etype365, _size362) = iprot.readListBegin() + for _i366 in xrange(_size362): + _elem367 = iprot.readString(); + self.success.append(_elem367) + iprot.readListEnd() + else: + iprot.skip(ftype) + elif fid == 1: + if ftype == TType.STRUCT: + self.o1 = MetaException() + self.o1.read(iprot) + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('get_databases_result') + if self.success is not None: + oprot.writeFieldBegin('success', TType.LIST, 0) + oprot.writeListBegin(TType.STRING, len(self.success)) + for iter368 in self.success: + oprot.writeString(iter368) + oprot.writeListEnd() + oprot.writeFieldEnd() + if self.o1 is not None: + oprot.writeFieldBegin('o1', TType.STRUCT, 1) + self.o1.write(oprot) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class get_all_databases_args: + + thrift_spec = ( + ) + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('get_all_databases_args') + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class get_all_databases_result: + """ + Attributes: + - success + - o1 + """ + + thrift_spec = ( + (0, TType.LIST, 'success', (TType.STRING,None), None, ), # 0 + (1, TType.STRUCT, 'o1', (MetaException, MetaException.thrift_spec), None, ), # 1 + ) + + def __init__(self, success=None, o1=None,): + self.success = success + self.o1 = o1 + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 0: + if ftype == TType.LIST: + self.success = [] + (_etype372, _size369) = iprot.readListBegin() + for _i373 in xrange(_size369): + _elem374 = iprot.readString(); + self.success.append(_elem374) + iprot.readListEnd() + else: + iprot.skip(ftype) + elif fid == 1: + if ftype == TType.STRUCT: + self.o1 = MetaException() + self.o1.read(iprot) + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('get_all_databases_result') + if self.success is not None: + oprot.writeFieldBegin('success', TType.LIST, 0) + oprot.writeListBegin(TType.STRING, len(self.success)) + for iter375 in self.success: + oprot.writeString(iter375) + oprot.writeListEnd() + oprot.writeFieldEnd() + if self.o1 is not None: + oprot.writeFieldBegin('o1', TType.STRUCT, 1) + self.o1.write(oprot) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class alter_database_args: + """ + Attributes: + - dbname + - db + """ + + thrift_spec = ( + None, # 0 + (1, TType.STRING, 'dbname', None, None, ), # 1 + (2, TType.STRUCT, 'db', (Database, Database.thrift_spec), None, ), # 2 + ) + + def __init__(self, dbname=None, db=None,): + self.dbname = dbname + self.db = db + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.STRING: + self.dbname = iprot.readString(); + else: + iprot.skip(ftype) + elif fid == 2: + if ftype == TType.STRUCT: + self.db = Database() + self.db.read(iprot) + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('alter_database_args') + if self.dbname is not None: + oprot.writeFieldBegin('dbname', TType.STRING, 1) + oprot.writeString(self.dbname) + oprot.writeFieldEnd() + if self.db is not None: + oprot.writeFieldBegin('db', TType.STRUCT, 2) + self.db.write(oprot) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class alter_database_result: + """ + Attributes: + - o1 + - o2 + """ + + thrift_spec = ( + None, # 0 + (1, TType.STRUCT, 'o1', (MetaException, MetaException.thrift_spec), None, ), # 1 + (2, TType.STRUCT, 'o2', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 2 + ) + + def __init__(self, o1=None, o2=None,): + self.o1 = o1 + self.o2 = o2 + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.STRUCT: + self.o1 = MetaException() + self.o1.read(iprot) + else: + iprot.skip(ftype) + elif fid == 2: + if ftype == TType.STRUCT: + self.o2 = NoSuchObjectException() + self.o2.read(iprot) + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('alter_database_result') + if self.o1 is not None: + oprot.writeFieldBegin('o1', TType.STRUCT, 1) + self.o1.write(oprot) + oprot.writeFieldEnd() + if self.o2 is not None: + oprot.writeFieldBegin('o2', TType.STRUCT, 2) + self.o2.write(oprot) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class get_type_args: + """ + Attributes: + - name + """ + + thrift_spec = ( + None, # 0 + (1, TType.STRING, 'name', None, None, ), # 1 + ) + + def __init__(self, name=None,): + self.name = name + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.STRING: + self.name = iprot.readString(); + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('get_type_args') + if self.name is not None: + oprot.writeFieldBegin('name', TType.STRING, 1) + oprot.writeString(self.name) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class get_type_result: + """ + Attributes: + - success + - o1 + - o2 + """ + + thrift_spec = ( + (0, TType.STRUCT, 'success', (Type, Type.thrift_spec), None, ), # 0 + (1, TType.STRUCT, 'o1', (MetaException, MetaException.thrift_spec), None, ), # 1 + (2, TType.STRUCT, 'o2', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 2 + ) + + def __init__(self, success=None, o1=None, o2=None,): + self.success = success + self.o1 = o1 + self.o2 = o2 + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 0: + if ftype == TType.STRUCT: + self.success = Type() + self.success.read(iprot) + else: + iprot.skip(ftype) + elif fid == 1: + if ftype == TType.STRUCT: + self.o1 = MetaException() + self.o1.read(iprot) + else: + iprot.skip(ftype) + elif fid == 2: + if ftype == TType.STRUCT: + self.o2 = NoSuchObjectException() + self.o2.read(iprot) + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('get_type_result') + if self.success is not None: + oprot.writeFieldBegin('success', TType.STRUCT, 0) + self.success.write(oprot) + oprot.writeFieldEnd() + if self.o1 is not None: + oprot.writeFieldBegin('o1', TType.STRUCT, 1) + self.o1.write(oprot) + oprot.writeFieldEnd() + if self.o2 is not None: + oprot.writeFieldBegin('o2', TType.STRUCT, 2) + self.o2.write(oprot) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class create_type_args: + """ + Attributes: + - type + """ + + thrift_spec = ( + None, # 0 + (1, TType.STRUCT, 'type', (Type, Type.thrift_spec), None, ), # 1 + ) + + def __init__(self, type=None,): + self.type = type + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.STRUCT: + self.type = Type() + self.type.read(iprot) + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('create_type_args') + if self.type is not None: + oprot.writeFieldBegin('type', TType.STRUCT, 1) + self.type.write(oprot) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class create_type_result: + """ + Attributes: + - success + - o1 + - o2 + - o3 + """ + + thrift_spec = ( + (0, TType.BOOL, 'success', None, None, ), # 0 + (1, TType.STRUCT, 'o1', (AlreadyExistsException, AlreadyExistsException.thrift_spec), None, ), # 1 + (2, TType.STRUCT, 'o2', (InvalidObjectException, InvalidObjectException.thrift_spec), None, ), # 2 + (3, TType.STRUCT, 'o3', (MetaException, MetaException.thrift_spec), None, ), # 3 + ) + + def __init__(self, success=None, o1=None, o2=None, o3=None,): + self.success = success + self.o1 = o1 + self.o2 = o2 + self.o3 = o3 + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 0: + if ftype == TType.BOOL: + self.success = iprot.readBool(); + else: + iprot.skip(ftype) + elif fid == 1: + if ftype == TType.STRUCT: + self.o1 = AlreadyExistsException() + self.o1.read(iprot) + else: + iprot.skip(ftype) + elif fid == 2: + if ftype == TType.STRUCT: + self.o2 = InvalidObjectException() + self.o2.read(iprot) + else: + iprot.skip(ftype) + elif fid == 3: + if ftype == TType.STRUCT: + self.o3 = MetaException() + self.o3.read(iprot) + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('create_type_result') + if self.success is not None: + oprot.writeFieldBegin('success', TType.BOOL, 0) + oprot.writeBool(self.success) + oprot.writeFieldEnd() + if self.o1 is not None: + oprot.writeFieldBegin('o1', TType.STRUCT, 1) + self.o1.write(oprot) + oprot.writeFieldEnd() + if self.o2 is not None: + oprot.writeFieldBegin('o2', TType.STRUCT, 2) + self.o2.write(oprot) + oprot.writeFieldEnd() + if self.o3 is not None: + oprot.writeFieldBegin('o3', TType.STRUCT, 3) + self.o3.write(oprot) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class drop_type_args: + """ + Attributes: + - type + """ + + thrift_spec = ( + None, # 0 + (1, TType.STRING, 'type', None, None, ), # 1 + ) + + def __init__(self, type=None,): + self.type = type + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.STRING: + self.type = iprot.readString(); + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('drop_type_args') + if self.type is not None: + oprot.writeFieldBegin('type', TType.STRING, 1) + oprot.writeString(self.type) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class drop_type_result: + """ + Attributes: + - success + - o1 + - o2 + """ + + thrift_spec = ( + (0, TType.BOOL, 'success', None, None, ), # 0 + (1, TType.STRUCT, 'o1', (MetaException, MetaException.thrift_spec), None, ), # 1 + (2, TType.STRUCT, 'o2', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 2 + ) + + def __init__(self, success=None, o1=None, o2=None,): + self.success = success + self.o1 = o1 + self.o2 = o2 + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 0: + if ftype == TType.BOOL: + self.success = iprot.readBool(); + else: + iprot.skip(ftype) + elif fid == 1: + if ftype == TType.STRUCT: + self.o1 = MetaException() + self.o1.read(iprot) + else: + iprot.skip(ftype) + elif fid == 2: + if ftype == TType.STRUCT: + self.o2 = NoSuchObjectException() + self.o2.read(iprot) + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('drop_type_result') + if self.success is not None: + oprot.writeFieldBegin('success', TType.BOOL, 0) + oprot.writeBool(self.success) + oprot.writeFieldEnd() + if self.o1 is not None: + oprot.writeFieldBegin('o1', TType.STRUCT, 1) + self.o1.write(oprot) + oprot.writeFieldEnd() + if self.o2 is not None: + oprot.writeFieldBegin('o2', TType.STRUCT, 2) + self.o2.write(oprot) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class get_type_all_args: + """ + Attributes: + - name + """ + + thrift_spec = ( + None, # 0 + (1, TType.STRING, 'name', None, None, ), # 1 + ) + + def __init__(self, name=None,): + self.name = name + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.STRING: + self.name = iprot.readString(); + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('get_type_all_args') + if self.name is not None: + oprot.writeFieldBegin('name', TType.STRING, 1) + oprot.writeString(self.name) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class get_type_all_result: + """ + Attributes: + - success + - o2 + """ + + thrift_spec = ( + (0, TType.MAP, 'success', (TType.STRING,None,TType.STRUCT,(Type, Type.thrift_spec)), None, ), # 0 + (1, TType.STRUCT, 'o2', (MetaException, MetaException.thrift_spec), None, ), # 1 + ) + + def __init__(self, success=None, o2=None,): + self.success = success + self.o2 = o2 + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 0: + if ftype == TType.MAP: + self.success = {} + (_ktype377, _vtype378, _size376 ) = iprot.readMapBegin() + for _i380 in xrange(_size376): + _key381 = iprot.readString(); + _val382 = Type() + _val382.read(iprot) + self.success[_key381] = _val382 + iprot.readMapEnd() + else: + iprot.skip(ftype) + elif fid == 1: + if ftype == TType.STRUCT: + self.o2 = MetaException() + self.o2.read(iprot) + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('get_type_all_result') + if self.success is not None: + oprot.writeFieldBegin('success', TType.MAP, 0) + oprot.writeMapBegin(TType.STRING, TType.STRUCT, len(self.success)) + for kiter383,viter384 in self.success.items(): + oprot.writeString(kiter383) + viter384.write(oprot) + oprot.writeMapEnd() + oprot.writeFieldEnd() + if self.o2 is not None: + oprot.writeFieldBegin('o2', TType.STRUCT, 1) + self.o2.write(oprot) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class get_fields_args: + """ + Attributes: + - db_name + - table_name + """ + + thrift_spec = ( + None, # 0 + (1, TType.STRING, 'db_name', None, None, ), # 1 + (2, TType.STRING, 'table_name', None, None, ), # 2 + ) + + def __init__(self, db_name=None, table_name=None,): + self.db_name = db_name + self.table_name = table_name + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.STRING: + self.db_name = iprot.readString(); + else: + iprot.skip(ftype) + elif fid == 2: + if ftype == TType.STRING: + self.table_name = iprot.readString(); + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('get_fields_args') + if self.db_name is not None: + oprot.writeFieldBegin('db_name', TType.STRING, 1) + oprot.writeString(self.db_name) + oprot.writeFieldEnd() + if self.table_name is not None: + oprot.writeFieldBegin('table_name', TType.STRING, 2) + oprot.writeString(self.table_name) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class get_fields_result: + """ + Attributes: + - success + - o1 + - o2 + - o3 + """ + + thrift_spec = ( + (0, TType.LIST, 'success', (TType.STRUCT,(FieldSchema, FieldSchema.thrift_spec)), None, ), # 0 + (1, TType.STRUCT, 'o1', (MetaException, MetaException.thrift_spec), None, ), # 1 + (2, TType.STRUCT, 'o2', (UnknownTableException, UnknownTableException.thrift_spec), None, ), # 2 + (3, TType.STRUCT, 'o3', (UnknownDBException, UnknownDBException.thrift_spec), None, ), # 3 + ) + + def __init__(self, success=None, o1=None, o2=None, o3=None,): + self.success = success + self.o1 = o1 + self.o2 = o2 + self.o3 = o3 + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 0: + if ftype == TType.LIST: + self.success = [] + (_etype388, _size385) = iprot.readListBegin() + for _i389 in xrange(_size385): + _elem390 = FieldSchema() + _elem390.read(iprot) + self.success.append(_elem390) + iprot.readListEnd() + else: + iprot.skip(ftype) + elif fid == 1: + if ftype == TType.STRUCT: + self.o1 = MetaException() + self.o1.read(iprot) + else: + iprot.skip(ftype) + elif fid == 2: + if ftype == TType.STRUCT: + self.o2 = UnknownTableException() + self.o2.read(iprot) + else: + iprot.skip(ftype) + elif fid == 3: + if ftype == TType.STRUCT: + self.o3 = UnknownDBException() + self.o3.read(iprot) + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('get_fields_result') + if self.success is not None: + oprot.writeFieldBegin('success', TType.LIST, 0) + oprot.writeListBegin(TType.STRUCT, len(self.success)) + for iter391 in self.success: + iter391.write(oprot) + oprot.writeListEnd() + oprot.writeFieldEnd() + if self.o1 is not None: + oprot.writeFieldBegin('o1', TType.STRUCT, 1) + self.o1.write(oprot) + oprot.writeFieldEnd() + if self.o2 is not None: + oprot.writeFieldBegin('o2', TType.STRUCT, 2) + self.o2.write(oprot) + oprot.writeFieldEnd() + if self.o3 is not None: + oprot.writeFieldBegin('o3', TType.STRUCT, 3) + self.o3.write(oprot) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return - def process_create_function(self, seqid, iprot, oprot): - args = create_function_args() - args.read(iprot) - iprot.readMessageEnd() - result = create_function_result() - try: - self._handler.create_function(args.func) - except AlreadyExistsException as o1: - result.o1 = o1 - except InvalidObjectException as o2: - result.o2 = o2 - except MetaException as o3: - result.o3 = o3 - except NoSuchObjectException as o4: - result.o4 = o4 - oprot.writeMessageBegin("create_function", TMessageType.REPLY, seqid) - result.write(oprot) - oprot.writeMessageEnd() - oprot.trans.flush() - def process_drop_function(self, seqid, iprot, oprot): - args = drop_function_args() - args.read(iprot) - iprot.readMessageEnd() - result = drop_function_result() - try: - self._handler.drop_function(args.dbName, args.funcName) - except NoSuchObjectException as o1: - result.o1 = o1 - except MetaException as o3: - result.o3 = o3 - oprot.writeMessageBegin("drop_function", TMessageType.REPLY, seqid) - result.write(oprot) - oprot.writeMessageEnd() - oprot.trans.flush() + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) - def process_alter_function(self, seqid, iprot, oprot): - args = alter_function_args() - args.read(iprot) - iprot.readMessageEnd() - result = alter_function_result() - try: - self._handler.alter_function(args.dbName, args.funcName, args.newFunc) - except InvalidOperationException as o1: - result.o1 = o1 - except MetaException as o2: - result.o2 = o2 - oprot.writeMessageBegin("alter_function", TMessageType.REPLY, seqid) - result.write(oprot) - oprot.writeMessageEnd() - oprot.trans.flush() + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ - def process_get_functions(self, seqid, iprot, oprot): - args = get_functions_args() - args.read(iprot) - iprot.readMessageEnd() - result = get_functions_result() - try: - result.success = self._handler.get_functions(args.dbName, args.pattern) - except MetaException as o1: - result.o1 = o1 - oprot.writeMessageBegin("get_functions", TMessageType.REPLY, seqid) - result.write(oprot) - oprot.writeMessageEnd() - oprot.trans.flush() + def __ne__(self, other): + return not (self == other) - def process_get_function(self, seqid, iprot, oprot): - args = get_function_args() - args.read(iprot) - iprot.readMessageEnd() - result = get_function_result() - try: - result.success = self._handler.get_function(args.dbName, args.funcName) - except MetaException as o1: - result.o1 = o1 - except NoSuchObjectException as o2: - result.o2 = o2 - oprot.writeMessageBegin("get_function", TMessageType.REPLY, seqid) - result.write(oprot) - oprot.writeMessageEnd() - oprot.trans.flush() +class get_schema_args: + """ + Attributes: + - db_name + - table_name + """ - def process_create_role(self, seqid, iprot, oprot): - args = create_role_args() - args.read(iprot) - iprot.readMessageEnd() - result = create_role_result() - try: - result.success = self._handler.create_role(args.role) - except MetaException as o1: - result.o1 = o1 - oprot.writeMessageBegin("create_role", TMessageType.REPLY, seqid) - result.write(oprot) - oprot.writeMessageEnd() - oprot.trans.flush() + thrift_spec = ( + None, # 0 + (1, TType.STRING, 'db_name', None, None, ), # 1 + (2, TType.STRING, 'table_name', None, None, ), # 2 + ) - def process_drop_role(self, seqid, iprot, oprot): - args = drop_role_args() - args.read(iprot) - iprot.readMessageEnd() - result = drop_role_result() - try: - result.success = self._handler.drop_role(args.role_name) - except MetaException as o1: - result.o1 = o1 - oprot.writeMessageBegin("drop_role", TMessageType.REPLY, seqid) - result.write(oprot) - oprot.writeMessageEnd() - oprot.trans.flush() + def __init__(self, db_name=None, table_name=None,): + self.db_name = db_name + self.table_name = table_name - def process_get_role_names(self, seqid, iprot, oprot): - args = get_role_names_args() - args.read(iprot) - iprot.readMessageEnd() - result = get_role_names_result() - try: - result.success = self._handler.get_role_names() - except MetaException as o1: - result.o1 = o1 - oprot.writeMessageBegin("get_role_names", TMessageType.REPLY, seqid) - result.write(oprot) - oprot.writeMessageEnd() - oprot.trans.flush() + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.STRING: + self.db_name = iprot.readString(); + else: + iprot.skip(ftype) + elif fid == 2: + if ftype == TType.STRING: + self.table_name = iprot.readString(); + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() - def process_grant_role(self, seqid, iprot, oprot): - args = grant_role_args() - args.read(iprot) - iprot.readMessageEnd() - result = grant_role_result() - try: - result.success = self._handler.grant_role(args.role_name, args.principal_name, args.principal_type, args.grantor, args.grantorType, args.grant_option) - except MetaException as o1: - result.o1 = o1 - oprot.writeMessageBegin("grant_role", TMessageType.REPLY, seqid) - result.write(oprot) - oprot.writeMessageEnd() - oprot.trans.flush() + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('get_schema_args') + if self.db_name is not None: + oprot.writeFieldBegin('db_name', TType.STRING, 1) + oprot.writeString(self.db_name) + oprot.writeFieldEnd() + if self.table_name is not None: + oprot.writeFieldBegin('table_name', TType.STRING, 2) + oprot.writeString(self.table_name) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() - def process_revoke_role(self, seqid, iprot, oprot): - args = revoke_role_args() - args.read(iprot) - iprot.readMessageEnd() - result = revoke_role_result() - try: - result.success = self._handler.revoke_role(args.role_name, args.principal_name, args.principal_type) - except MetaException as o1: - result.o1 = o1 - oprot.writeMessageBegin("revoke_role", TMessageType.REPLY, seqid) - result.write(oprot) - oprot.writeMessageEnd() - oprot.trans.flush() + def validate(self): + return - def process_list_roles(self, seqid, iprot, oprot): - args = list_roles_args() - args.read(iprot) - iprot.readMessageEnd() - result = list_roles_result() - try: - result.success = self._handler.list_roles(args.principal_name, args.principal_type) - except MetaException as o1: - result.o1 = o1 - oprot.writeMessageBegin("list_roles", TMessageType.REPLY, seqid) - result.write(oprot) - oprot.writeMessageEnd() - oprot.trans.flush() - def process_get_privilege_set(self, seqid, iprot, oprot): - args = get_privilege_set_args() - args.read(iprot) - iprot.readMessageEnd() - result = get_privilege_set_result() - try: - result.success = self._handler.get_privilege_set(args.hiveObject, args.user_name, args.group_names) - except MetaException as o1: - result.o1 = o1 - oprot.writeMessageBegin("get_privilege_set", TMessageType.REPLY, seqid) - result.write(oprot) - oprot.writeMessageEnd() - oprot.trans.flush() + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) - def process_list_privileges(self, seqid, iprot, oprot): - args = list_privileges_args() - args.read(iprot) - iprot.readMessageEnd() - result = list_privileges_result() - try: - result.success = self._handler.list_privileges(args.principal_name, args.principal_type, args.hiveObject) - except MetaException as o1: - result.o1 = o1 - oprot.writeMessageBegin("list_privileges", TMessageType.REPLY, seqid) - result.write(oprot) - oprot.writeMessageEnd() - oprot.trans.flush() + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ - def process_grant_privileges(self, seqid, iprot, oprot): - args = grant_privileges_args() - args.read(iprot) - iprot.readMessageEnd() - result = grant_privileges_result() - try: - result.success = self._handler.grant_privileges(args.privileges) - except MetaException as o1: - result.o1 = o1 - oprot.writeMessageBegin("grant_privileges", TMessageType.REPLY, seqid) - result.write(oprot) - oprot.writeMessageEnd() - oprot.trans.flush() + def __ne__(self, other): + return not (self == other) - def process_revoke_privileges(self, seqid, iprot, oprot): - args = revoke_privileges_args() - args.read(iprot) - iprot.readMessageEnd() - result = revoke_privileges_result() - try: - result.success = self._handler.revoke_privileges(args.privileges) - except MetaException as o1: - result.o1 = o1 - oprot.writeMessageBegin("revoke_privileges", TMessageType.REPLY, seqid) - result.write(oprot) - oprot.writeMessageEnd() - oprot.trans.flush() +class get_schema_result: + """ + Attributes: + - success + - o1 + - o2 + - o3 + """ - def process_set_ugi(self, seqid, iprot, oprot): - args = set_ugi_args() - args.read(iprot) - iprot.readMessageEnd() - result = set_ugi_result() - try: - result.success = self._handler.set_ugi(args.user_name, args.group_names) - except MetaException as o1: - result.o1 = o1 - oprot.writeMessageBegin("set_ugi", TMessageType.REPLY, seqid) - result.write(oprot) - oprot.writeMessageEnd() - oprot.trans.flush() + thrift_spec = ( + (0, TType.LIST, 'success', (TType.STRUCT,(FieldSchema, FieldSchema.thrift_spec)), None, ), # 0 + (1, TType.STRUCT, 'o1', (MetaException, MetaException.thrift_spec), None, ), # 1 + (2, TType.STRUCT, 'o2', (UnknownTableException, UnknownTableException.thrift_spec), None, ), # 2 + (3, TType.STRUCT, 'o3', (UnknownDBException, UnknownDBException.thrift_spec), None, ), # 3 + ) - def process_get_delegation_token(self, seqid, iprot, oprot): - args = get_delegation_token_args() - args.read(iprot) - iprot.readMessageEnd() - result = get_delegation_token_result() - try: - result.success = self._handler.get_delegation_token(args.token_owner, args.renewer_kerberos_principal_name) - except MetaException as o1: - result.o1 = o1 - oprot.writeMessageBegin("get_delegation_token", TMessageType.REPLY, seqid) - result.write(oprot) - oprot.writeMessageEnd() - oprot.trans.flush() + def __init__(self, success=None, o1=None, o2=None, o3=None,): + self.success = success + self.o1 = o1 + self.o2 = o2 + self.o3 = o3 - def process_renew_delegation_token(self, seqid, iprot, oprot): - args = renew_delegation_token_args() - args.read(iprot) - iprot.readMessageEnd() - result = renew_delegation_token_result() - try: - result.success = self._handler.renew_delegation_token(args.token_str_form) - except MetaException as o1: - result.o1 = o1 - oprot.writeMessageBegin("renew_delegation_token", TMessageType.REPLY, seqid) - result.write(oprot) - oprot.writeMessageEnd() - oprot.trans.flush() + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 0: + if ftype == TType.LIST: + self.success = [] + (_etype395, _size392) = iprot.readListBegin() + for _i396 in xrange(_size392): + _elem397 = FieldSchema() + _elem397.read(iprot) + self.success.append(_elem397) + iprot.readListEnd() + else: + iprot.skip(ftype) + elif fid == 1: + if ftype == TType.STRUCT: + self.o1 = MetaException() + self.o1.read(iprot) + else: + iprot.skip(ftype) + elif fid == 2: + if ftype == TType.STRUCT: + self.o2 = UnknownTableException() + self.o2.read(iprot) + else: + iprot.skip(ftype) + elif fid == 3: + if ftype == TType.STRUCT: + self.o3 = UnknownDBException() + self.o3.read(iprot) + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() - def process_cancel_delegation_token(self, seqid, iprot, oprot): - args = cancel_delegation_token_args() - args.read(iprot) - iprot.readMessageEnd() - result = cancel_delegation_token_result() - try: - self._handler.cancel_delegation_token(args.token_str_form) - except MetaException as o1: - result.o1 = o1 - oprot.writeMessageBegin("cancel_delegation_token", TMessageType.REPLY, seqid) - result.write(oprot) - oprot.writeMessageEnd() - oprot.trans.flush() + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('get_schema_result') + if self.success is not None: + oprot.writeFieldBegin('success', TType.LIST, 0) + oprot.writeListBegin(TType.STRUCT, len(self.success)) + for iter398 in self.success: + iter398.write(oprot) + oprot.writeListEnd() + oprot.writeFieldEnd() + if self.o1 is not None: + oprot.writeFieldBegin('o1', TType.STRUCT, 1) + self.o1.write(oprot) + oprot.writeFieldEnd() + if self.o2 is not None: + oprot.writeFieldBegin('o2', TType.STRUCT, 2) + self.o2.write(oprot) + oprot.writeFieldEnd() + if self.o3 is not None: + oprot.writeFieldBegin('o3', TType.STRUCT, 3) + self.o3.write(oprot) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return -# HELPER FUNCTIONS AND STRUCTURES + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) -class create_database_args: + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class create_table_args: """ Attributes: - - database + - tbl """ thrift_spec = ( None, # 0 - (1, TType.STRUCT, 'database', (Database, Database.thrift_spec), None, ), # 1 + (1, TType.STRUCT, 'tbl', (Table, Table.thrift_spec), None, ), # 1 ) - def __init__(self, database=None,): - self.database = database + def __init__(self, tbl=None,): + self.tbl = tbl def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -5856,8 +8299,8 @@ def read(self, iprot): break if fid == 1: if ftype == TType.STRUCT: - self.database = Database() - self.database.read(iprot) + self.tbl = Table() + self.tbl.read(iprot) else: iprot.skip(ftype) else: @@ -5869,10 +8312,10 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('create_database_args') - if self.database is not None: - oprot.writeFieldBegin('database', TType.STRUCT, 1) - self.database.write(oprot) + oprot.writeStructBegin('create_table_args') + if self.tbl is not None: + oprot.writeFieldBegin('tbl', TType.STRUCT, 1) + self.tbl.write(oprot) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -5892,12 +8335,13 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class create_database_result: +class create_table_result: """ Attributes: - o1 - o2 - o3 + - o4 """ thrift_spec = ( @@ -5905,12 +8349,14 @@ class create_database_result: (1, TType.STRUCT, 'o1', (AlreadyExistsException, AlreadyExistsException.thrift_spec), None, ), # 1 (2, TType.STRUCT, 'o2', (InvalidObjectException, InvalidObjectException.thrift_spec), None, ), # 2 (3, TType.STRUCT, 'o3', (MetaException, MetaException.thrift_spec), None, ), # 3 + (4, TType.STRUCT, 'o4', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 4 ) - def __init__(self, o1=None, o2=None, o3=None,): + def __init__(self, o1=None, o2=None, o3=None, o4=None,): self.o1 = o1 self.o2 = o2 self.o3 = o3 + self.o4 = o4 def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -5939,6 +8385,12 @@ def read(self, iprot): self.o3.read(iprot) else: iprot.skip(ftype) + elif fid == 4: + if ftype == TType.STRUCT: + self.o4 = NoSuchObjectException() + self.o4.read(iprot) + else: + iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() @@ -5948,7 +8400,7 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('create_database_result') + oprot.writeStructBegin('create_table_result') if self.o1 is not None: oprot.writeFieldBegin('o1', TType.STRUCT, 1) self.o1.write(oprot) @@ -5961,6 +8413,10 @@ def write(self, oprot): oprot.writeFieldBegin('o3', TType.STRUCT, 3) self.o3.write(oprot) oprot.writeFieldEnd() + if self.o4 is not None: + oprot.writeFieldBegin('o4', TType.STRUCT, 4) + self.o4.write(oprot) + oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -5979,19 +8435,22 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class get_database_args: +class create_table_with_environment_context_args: """ Attributes: - - name + - tbl + - environment_context """ thrift_spec = ( None, # 0 - (1, TType.STRING, 'name', None, None, ), # 1 + (1, TType.STRUCT, 'tbl', (Table, Table.thrift_spec), None, ), # 1 + (2, TType.STRUCT, 'environment_context', (EnvironmentContext, EnvironmentContext.thrift_spec), None, ), # 2 ) - def __init__(self, name=None,): - self.name = name + def __init__(self, tbl=None, environment_context=None,): + self.tbl = tbl + self.environment_context = environment_context def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -6003,8 +8462,15 @@ def read(self, iprot): if ftype == TType.STOP: break if fid == 1: - if ftype == TType.STRING: - self.name = iprot.readString(); + if ftype == TType.STRUCT: + self.tbl = Table() + self.tbl.read(iprot) + else: + iprot.skip(ftype) + elif fid == 2: + if ftype == TType.STRUCT: + self.environment_context = EnvironmentContext() + self.environment_context.read(iprot) else: iprot.skip(ftype) else: @@ -6016,10 +8482,14 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('get_database_args') - if self.name is not None: - oprot.writeFieldBegin('name', TType.STRING, 1) - oprot.writeString(self.name) + oprot.writeStructBegin('create_table_with_environment_context_args') + if self.tbl is not None: + oprot.writeFieldBegin('tbl', TType.STRUCT, 1) + self.tbl.write(oprot) + oprot.writeFieldEnd() + if self.environment_context is not None: + oprot.writeFieldBegin('environment_context', TType.STRUCT, 2) + self.environment_context.write(oprot) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -6039,24 +8509,28 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class get_database_result: +class create_table_with_environment_context_result: """ Attributes: - - success - o1 - o2 + - o3 + - o4 """ thrift_spec = ( - (0, TType.STRUCT, 'success', (Database, Database.thrift_spec), None, ), # 0 - (1, TType.STRUCT, 'o1', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 1 - (2, TType.STRUCT, 'o2', (MetaException, MetaException.thrift_spec), None, ), # 2 + None, # 0 + (1, TType.STRUCT, 'o1', (AlreadyExistsException, AlreadyExistsException.thrift_spec), None, ), # 1 + (2, TType.STRUCT, 'o2', (InvalidObjectException, InvalidObjectException.thrift_spec), None, ), # 2 + (3, TType.STRUCT, 'o3', (MetaException, MetaException.thrift_spec), None, ), # 3 + (4, TType.STRUCT, 'o4', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 4 ) - def __init__(self, success=None, o1=None, o2=None,): - self.success = success + def __init__(self, o1=None, o2=None, o3=None, o4=None,): self.o1 = o1 self.o2 = o2 + self.o3 = o3 + self.o4 = o4 def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -6067,24 +8541,30 @@ def read(self, iprot): (fname, ftype, fid) = iprot.readFieldBegin() if ftype == TType.STOP: break - if fid == 0: - if ftype == TType.STRUCT: - self.success = Database() - self.success.read(iprot) - else: - iprot.skip(ftype) - elif fid == 1: + if fid == 1: if ftype == TType.STRUCT: - self.o1 = NoSuchObjectException() + self.o1 = AlreadyExistsException() self.o1.read(iprot) else: iprot.skip(ftype) elif fid == 2: if ftype == TType.STRUCT: - self.o2 = MetaException() + self.o2 = InvalidObjectException() self.o2.read(iprot) else: iprot.skip(ftype) + elif fid == 3: + if ftype == TType.STRUCT: + self.o3 = MetaException() + self.o3.read(iprot) + else: + iprot.skip(ftype) + elif fid == 4: + if ftype == TType.STRUCT: + self.o4 = NoSuchObjectException() + self.o4.read(iprot) + else: + iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() @@ -6094,11 +8574,7 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('get_database_result') - if self.success is not None: - oprot.writeFieldBegin('success', TType.STRUCT, 0) - self.success.write(oprot) - oprot.writeFieldEnd() + oprot.writeStructBegin('create_table_with_environment_context_result') if self.o1 is not None: oprot.writeFieldBegin('o1', TType.STRUCT, 1) self.o1.write(oprot) @@ -6107,6 +8583,14 @@ def write(self, oprot): oprot.writeFieldBegin('o2', TType.STRUCT, 2) self.o2.write(oprot) oprot.writeFieldEnd() + if self.o3 is not None: + oprot.writeFieldBegin('o3', TType.STRUCT, 3) + self.o3.write(oprot) + oprot.writeFieldEnd() + if self.o4 is not None: + oprot.writeFieldBegin('o4', TType.STRUCT, 4) + self.o4.write(oprot) + oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -6125,25 +8609,25 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class drop_database_args: +class drop_table_args: """ Attributes: + - dbname - name - deleteData - - cascade """ thrift_spec = ( None, # 0 - (1, TType.STRING, 'name', None, None, ), # 1 - (2, TType.BOOL, 'deleteData', None, None, ), # 2 - (3, TType.BOOL, 'cascade', None, None, ), # 3 + (1, TType.STRING, 'dbname', None, None, ), # 1 + (2, TType.STRING, 'name', None, None, ), # 2 + (3, TType.BOOL, 'deleteData', None, None, ), # 3 ) - def __init__(self, name=None, deleteData=None, cascade=None,): + def __init__(self, dbname=None, name=None, deleteData=None,): + self.dbname = dbname self.name = name self.deleteData = deleteData - self.cascade = cascade def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -6156,17 +8640,17 @@ def read(self, iprot): break if fid == 1: if ftype == TType.STRING: - self.name = iprot.readString(); + self.dbname = iprot.readString(); else: iprot.skip(ftype) elif fid == 2: - if ftype == TType.BOOL: - self.deleteData = iprot.readBool(); + if ftype == TType.STRING: + self.name = iprot.readString(); else: iprot.skip(ftype) elif fid == 3: if ftype == TType.BOOL: - self.cascade = iprot.readBool(); + self.deleteData = iprot.readBool(); else: iprot.skip(ftype) else: @@ -6178,19 +8662,19 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('drop_database_args') + oprot.writeStructBegin('drop_table_args') + if self.dbname is not None: + oprot.writeFieldBegin('dbname', TType.STRING, 1) + oprot.writeString(self.dbname) + oprot.writeFieldEnd() if self.name is not None: - oprot.writeFieldBegin('name', TType.STRING, 1) + oprot.writeFieldBegin('name', TType.STRING, 2) oprot.writeString(self.name) oprot.writeFieldEnd() if self.deleteData is not None: - oprot.writeFieldBegin('deleteData', TType.BOOL, 2) + oprot.writeFieldBegin('deleteData', TType.BOOL, 3) oprot.writeBool(self.deleteData) oprot.writeFieldEnd() - if self.cascade is not None: - oprot.writeFieldBegin('cascade', TType.BOOL, 3) - oprot.writeBool(self.cascade) - oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -6209,24 +8693,21 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class drop_database_result: +class drop_table_result: """ Attributes: - o1 - - o2 - o3 """ thrift_spec = ( None, # 0 (1, TType.STRUCT, 'o1', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 1 - (2, TType.STRUCT, 'o2', (InvalidOperationException, InvalidOperationException.thrift_spec), None, ), # 2 - (3, TType.STRUCT, 'o3', (MetaException, MetaException.thrift_spec), None, ), # 3 + (2, TType.STRUCT, 'o3', (MetaException, MetaException.thrift_spec), None, ), # 2 ) - def __init__(self, o1=None, o2=None, o3=None,): + def __init__(self, o1=None, o3=None,): self.o1 = o1 - self.o2 = o2 self.o3 = o3 def read(self, iprot): @@ -6246,12 +8727,6 @@ def read(self, iprot): iprot.skip(ftype) elif fid == 2: if ftype == TType.STRUCT: - self.o2 = InvalidOperationException() - self.o2.read(iprot) - else: - iprot.skip(ftype) - elif fid == 3: - if ftype == TType.STRUCT: self.o3 = MetaException() self.o3.read(iprot) else: @@ -6265,17 +8740,13 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('drop_database_result') + oprot.writeStructBegin('drop_table_result') if self.o1 is not None: oprot.writeFieldBegin('o1', TType.STRUCT, 1) self.o1.write(oprot) oprot.writeFieldEnd() - if self.o2 is not None: - oprot.writeFieldBegin('o2', TType.STRUCT, 2) - self.o2.write(oprot) - oprot.writeFieldEnd() if self.o3 is not None: - oprot.writeFieldBegin('o3', TType.STRUCT, 3) + oprot.writeFieldBegin('o3', TType.STRUCT, 2) self.o3.write(oprot) oprot.writeFieldEnd() oprot.writeFieldStop() @@ -6296,19 +8767,28 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class get_databases_args: +class drop_table_with_environment_context_args: """ Attributes: - - pattern + - dbname + - name + - deleteData + - environment_context """ thrift_spec = ( None, # 0 - (1, TType.STRING, 'pattern', None, None, ), # 1 + (1, TType.STRING, 'dbname', None, None, ), # 1 + (2, TType.STRING, 'name', None, None, ), # 2 + (3, TType.BOOL, 'deleteData', None, None, ), # 3 + (4, TType.STRUCT, 'environment_context', (EnvironmentContext, EnvironmentContext.thrift_spec), None, ), # 4 ) - def __init__(self, pattern=None,): - self.pattern = pattern + def __init__(self, dbname=None, name=None, deleteData=None, environment_context=None,): + self.dbname = dbname + self.name = name + self.deleteData = deleteData + self.environment_context = environment_context def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -6321,7 +8801,23 @@ def read(self, iprot): break if fid == 1: if ftype == TType.STRING: - self.pattern = iprot.readString(); + self.dbname = iprot.readString(); + else: + iprot.skip(ftype) + elif fid == 2: + if ftype == TType.STRING: + self.name = iprot.readString(); + else: + iprot.skip(ftype) + elif fid == 3: + if ftype == TType.BOOL: + self.deleteData = iprot.readBool(); + else: + iprot.skip(ftype) + elif fid == 4: + if ftype == TType.STRUCT: + self.environment_context = EnvironmentContext() + self.environment_context.read(iprot) else: iprot.skip(ftype) else: @@ -6333,10 +8829,22 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('get_databases_args') - if self.pattern is not None: - oprot.writeFieldBegin('pattern', TType.STRING, 1) - oprot.writeString(self.pattern) + oprot.writeStructBegin('drop_table_with_environment_context_args') + if self.dbname is not None: + oprot.writeFieldBegin('dbname', TType.STRING, 1) + oprot.writeString(self.dbname) + oprot.writeFieldEnd() + if self.name is not None: + oprot.writeFieldBegin('name', TType.STRING, 2) + oprot.writeString(self.name) + oprot.writeFieldEnd() + if self.deleteData is not None: + oprot.writeFieldBegin('deleteData', TType.BOOL, 3) + oprot.writeBool(self.deleteData) + oprot.writeFieldEnd() + if self.environment_context is not None: + oprot.writeFieldBegin('environment_context', TType.STRUCT, 4) + self.environment_context.write(oprot) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -6356,21 +8864,22 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class get_databases_result: +class drop_table_with_environment_context_result: """ Attributes: - - success - o1 + - o3 """ thrift_spec = ( - (0, TType.LIST, 'success', (TType.STRING,None), None, ), # 0 - (1, TType.STRUCT, 'o1', (MetaException, MetaException.thrift_spec), None, ), # 1 + None, # 0 + (1, TType.STRUCT, 'o1', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 1 + (2, TType.STRUCT, 'o3', (MetaException, MetaException.thrift_spec), None, ), # 2 ) - def __init__(self, success=None, o1=None,): - self.success = success + def __init__(self, o1=None, o3=None,): self.o1 = o1 + self.o3 = o3 def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -6381,20 +8890,16 @@ def read(self, iprot): (fname, ftype, fid) = iprot.readFieldBegin() if ftype == TType.STOP: break - if fid == 0: - if ftype == TType.LIST: - self.success = [] - (_etype323, _size320) = iprot.readListBegin() - for _i324 in xrange(_size320): - _elem325 = iprot.readString(); - self.success.append(_elem325) - iprot.readListEnd() + if fid == 1: + if ftype == TType.STRUCT: + self.o1 = NoSuchObjectException() + self.o1.read(iprot) else: iprot.skip(ftype) - elif fid == 1: + elif fid == 2: if ftype == TType.STRUCT: - self.o1 = MetaException() - self.o1.read(iprot) + self.o3 = MetaException() + self.o3.read(iprot) else: iprot.skip(ftype) else: @@ -6406,18 +8911,15 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('get_databases_result') - if self.success is not None: - oprot.writeFieldBegin('success', TType.LIST, 0) - oprot.writeListBegin(TType.STRING, len(self.success)) - for iter326 in self.success: - oprot.writeString(iter326) - oprot.writeListEnd() - oprot.writeFieldEnd() + oprot.writeStructBegin('drop_table_with_environment_context_result') if self.o1 is not None: oprot.writeFieldBegin('o1', TType.STRUCT, 1) self.o1.write(oprot) oprot.writeFieldEnd() + if self.o3 is not None: + oprot.writeFieldBegin('o3', TType.STRUCT, 2) + self.o3.write(oprot) + oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -6436,11 +8938,23 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class get_all_databases_args: +class get_tables_args: + """ + Attributes: + - db_name + - pattern + """ thrift_spec = ( + None, # 0 + (1, TType.STRING, 'db_name', None, None, ), # 1 + (2, TType.STRING, 'pattern', None, None, ), # 2 ) + def __init__(self, db_name=None, pattern=None,): + self.db_name = db_name + self.pattern = pattern + def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) @@ -6450,6 +8964,16 @@ def read(self, iprot): (fname, ftype, fid) = iprot.readFieldBegin() if ftype == TType.STOP: break + if fid == 1: + if ftype == TType.STRING: + self.db_name = iprot.readString(); + else: + iprot.skip(ftype) + elif fid == 2: + if ftype == TType.STRING: + self.pattern = iprot.readString(); + else: + iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() @@ -6459,7 +8983,15 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('get_all_databases_args') + oprot.writeStructBegin('get_tables_args') + if self.db_name is not None: + oprot.writeFieldBegin('db_name', TType.STRING, 1) + oprot.writeString(self.db_name) + oprot.writeFieldEnd() + if self.pattern is not None: + oprot.writeFieldBegin('pattern', TType.STRING, 2) + oprot.writeString(self.pattern) + oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -6478,7 +9010,7 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class get_all_databases_result: +class get_tables_result: """ Attributes: - success @@ -6506,10 +9038,10 @@ def read(self, iprot): if fid == 0: if ftype == TType.LIST: self.success = [] - (_etype330, _size327) = iprot.readListBegin() - for _i331 in xrange(_size327): - _elem332 = iprot.readString(); - self.success.append(_elem332) + (_etype402, _size399) = iprot.readListBegin() + for _i403 in xrange(_size399): + _elem404 = iprot.readString(); + self.success.append(_elem404) iprot.readListEnd() else: iprot.skip(ftype) @@ -6528,12 +9060,12 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('get_all_databases_result') + oprot.writeStructBegin('get_tables_result') if self.success is not None: oprot.writeFieldBegin('success', TType.LIST, 0) oprot.writeListBegin(TType.STRING, len(self.success)) - for iter333 in self.success: - oprot.writeString(iter333) + for iter405 in self.success: + oprot.writeString(iter405) oprot.writeListEnd() oprot.writeFieldEnd() if self.o1 is not None: @@ -6558,22 +9090,19 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class alter_database_args: +class get_all_tables_args: """ Attributes: - - dbname - - db + - db_name """ thrift_spec = ( None, # 0 - (1, TType.STRING, 'dbname', None, None, ), # 1 - (2, TType.STRUCT, 'db', (Database, Database.thrift_spec), None, ), # 2 + (1, TType.STRING, 'db_name', None, None, ), # 1 ) - def __init__(self, dbname=None, db=None,): - self.dbname = dbname - self.db = db + def __init__(self, db_name=None,): + self.db_name = db_name def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -6586,13 +9115,7 @@ def read(self, iprot): break if fid == 1: if ftype == TType.STRING: - self.dbname = iprot.readString(); - else: - iprot.skip(ftype) - elif fid == 2: - if ftype == TType.STRUCT: - self.db = Database() - self.db.read(iprot) + self.db_name = iprot.readString(); else: iprot.skip(ftype) else: @@ -6604,14 +9127,10 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('alter_database_args') - if self.dbname is not None: - oprot.writeFieldBegin('dbname', TType.STRING, 1) - oprot.writeString(self.dbname) - oprot.writeFieldEnd() - if self.db is not None: - oprot.writeFieldBegin('db', TType.STRUCT, 2) - self.db.write(oprot) + oprot.writeStructBegin('get_all_tables_args') + if self.db_name is not None: + oprot.writeFieldBegin('db_name', TType.STRING, 1) + oprot.writeString(self.db_name) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -6631,22 +9150,21 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class alter_database_result: +class get_all_tables_result: """ Attributes: + - success - o1 - - o2 """ thrift_spec = ( - None, # 0 + (0, TType.LIST, 'success', (TType.STRING,None), None, ), # 0 (1, TType.STRUCT, 'o1', (MetaException, MetaException.thrift_spec), None, ), # 1 - (2, TType.STRUCT, 'o2', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 2 ) - def __init__(self, o1=None, o2=None,): + def __init__(self, success=None, o1=None,): + self.success = success self.o1 = o1 - self.o2 = o2 def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -6657,16 +9175,20 @@ def read(self, iprot): (fname, ftype, fid) = iprot.readFieldBegin() if ftype == TType.STOP: break - if fid == 1: - if ftype == TType.STRUCT: - self.o1 = MetaException() - self.o1.read(iprot) + if fid == 0: + if ftype == TType.LIST: + self.success = [] + (_etype409, _size406) = iprot.readListBegin() + for _i410 in xrange(_size406): + _elem411 = iprot.readString(); + self.success.append(_elem411) + iprot.readListEnd() else: iprot.skip(ftype) - elif fid == 2: + elif fid == 1: if ftype == TType.STRUCT: - self.o2 = NoSuchObjectException() - self.o2.read(iprot) + self.o1 = MetaException() + self.o1.read(iprot) else: iprot.skip(ftype) else: @@ -6678,15 +9200,18 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('alter_database_result') + oprot.writeStructBegin('get_all_tables_result') + if self.success is not None: + oprot.writeFieldBegin('success', TType.LIST, 0) + oprot.writeListBegin(TType.STRING, len(self.success)) + for iter412 in self.success: + oprot.writeString(iter412) + oprot.writeListEnd() + oprot.writeFieldEnd() if self.o1 is not None: oprot.writeFieldBegin('o1', TType.STRUCT, 1) self.o1.write(oprot) oprot.writeFieldEnd() - if self.o2 is not None: - oprot.writeFieldBegin('o2', TType.STRUCT, 2) - self.o2.write(oprot) - oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -6705,19 +9230,22 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class get_type_args: +class get_table_args: """ Attributes: - - name + - dbname + - tbl_name """ thrift_spec = ( None, # 0 - (1, TType.STRING, 'name', None, None, ), # 1 + (1, TType.STRING, 'dbname', None, None, ), # 1 + (2, TType.STRING, 'tbl_name', None, None, ), # 2 ) - def __init__(self, name=None,): - self.name = name + def __init__(self, dbname=None, tbl_name=None,): + self.dbname = dbname + self.tbl_name = tbl_name def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -6730,7 +9258,12 @@ def read(self, iprot): break if fid == 1: if ftype == TType.STRING: - self.name = iprot.readString(); + self.dbname = iprot.readString(); + else: + iprot.skip(ftype) + elif fid == 2: + if ftype == TType.STRING: + self.tbl_name = iprot.readString(); else: iprot.skip(ftype) else: @@ -6742,10 +9275,14 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('get_type_args') - if self.name is not None: - oprot.writeFieldBegin('name', TType.STRING, 1) - oprot.writeString(self.name) + oprot.writeStructBegin('get_table_args') + if self.dbname is not None: + oprot.writeFieldBegin('dbname', TType.STRING, 1) + oprot.writeString(self.dbname) + oprot.writeFieldEnd() + if self.tbl_name is not None: + oprot.writeFieldBegin('tbl_name', TType.STRING, 2) + oprot.writeString(self.tbl_name) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -6765,7 +9302,7 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class get_type_result: +class get_table_result: """ Attributes: - success @@ -6774,7 +9311,7 @@ class get_type_result: """ thrift_spec = ( - (0, TType.STRUCT, 'success', (Type, Type.thrift_spec), None, ), # 0 + (0, TType.STRUCT, 'success', (Table, Table.thrift_spec), None, ), # 0 (1, TType.STRUCT, 'o1', (MetaException, MetaException.thrift_spec), None, ), # 1 (2, TType.STRUCT, 'o2', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 2 ) @@ -6795,7 +9332,7 @@ def read(self, iprot): break if fid == 0: if ftype == TType.STRUCT: - self.success = Type() + self.success = Table() self.success.read(iprot) else: iprot.skip(ftype) @@ -6820,7 +9357,7 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('get_type_result') + oprot.writeStructBegin('get_table_result') if self.success is not None: oprot.writeFieldBegin('success', TType.STRUCT, 0) self.success.write(oprot) @@ -6851,19 +9388,22 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class create_type_args: +class get_table_objects_by_name_args: """ Attributes: - - type + - dbname + - tbl_names """ thrift_spec = ( None, # 0 - (1, TType.STRUCT, 'type', (Type, Type.thrift_spec), None, ), # 1 + (1, TType.STRING, 'dbname', None, None, ), # 1 + (2, TType.LIST, 'tbl_names', (TType.STRING,None), None, ), # 2 ) - def __init__(self, type=None,): - self.type = type + def __init__(self, dbname=None, tbl_names=None,): + self.dbname = dbname + self.tbl_names = tbl_names def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -6875,9 +9415,18 @@ def read(self, iprot): if ftype == TType.STOP: break if fid == 1: - if ftype == TType.STRUCT: - self.type = Type() - self.type.read(iprot) + if ftype == TType.STRING: + self.dbname = iprot.readString(); + else: + iprot.skip(ftype) + elif fid == 2: + if ftype == TType.LIST: + self.tbl_names = [] + (_etype416, _size413) = iprot.readListBegin() + for _i417 in xrange(_size413): + _elem418 = iprot.readString(); + self.tbl_names.append(_elem418) + iprot.readListEnd() else: iprot.skip(ftype) else: @@ -6889,10 +9438,17 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('create_type_args') - if self.type is not None: - oprot.writeFieldBegin('type', TType.STRUCT, 1) - self.type.write(oprot) + oprot.writeStructBegin('get_table_objects_by_name_args') + if self.dbname is not None: + oprot.writeFieldBegin('dbname', TType.STRING, 1) + oprot.writeString(self.dbname) + oprot.writeFieldEnd() + if self.tbl_names is not None: + oprot.writeFieldBegin('tbl_names', TType.LIST, 2) + oprot.writeListBegin(TType.STRING, len(self.tbl_names)) + for iter419 in self.tbl_names: + oprot.writeString(iter419) + oprot.writeListEnd() oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -6912,7 +9468,7 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class create_type_result: +class get_table_objects_by_name_result: """ Attributes: - success @@ -6922,10 +9478,10 @@ class create_type_result: """ thrift_spec = ( - (0, TType.BOOL, 'success', None, None, ), # 0 - (1, TType.STRUCT, 'o1', (AlreadyExistsException, AlreadyExistsException.thrift_spec), None, ), # 1 - (2, TType.STRUCT, 'o2', (InvalidObjectException, InvalidObjectException.thrift_spec), None, ), # 2 - (3, TType.STRUCT, 'o3', (MetaException, MetaException.thrift_spec), None, ), # 3 + (0, TType.LIST, 'success', (TType.STRUCT,(Table, Table.thrift_spec)), None, ), # 0 + (1, TType.STRUCT, 'o1', (MetaException, MetaException.thrift_spec), None, ), # 1 + (2, TType.STRUCT, 'o2', (InvalidOperationException, InvalidOperationException.thrift_spec), None, ), # 2 + (3, TType.STRUCT, 'o3', (UnknownDBException, UnknownDBException.thrift_spec), None, ), # 3 ) def __init__(self, success=None, o1=None, o2=None, o3=None,): @@ -6944,25 +9500,31 @@ def read(self, iprot): if ftype == TType.STOP: break if fid == 0: - if ftype == TType.BOOL: - self.success = iprot.readBool(); + if ftype == TType.LIST: + self.success = [] + (_etype423, _size420) = iprot.readListBegin() + for _i424 in xrange(_size420): + _elem425 = Table() + _elem425.read(iprot) + self.success.append(_elem425) + iprot.readListEnd() else: iprot.skip(ftype) elif fid == 1: if ftype == TType.STRUCT: - self.o1 = AlreadyExistsException() + self.o1 = MetaException() self.o1.read(iprot) else: iprot.skip(ftype) elif fid == 2: if ftype == TType.STRUCT: - self.o2 = InvalidObjectException() + self.o2 = InvalidOperationException() self.o2.read(iprot) else: iprot.skip(ftype) elif fid == 3: if ftype == TType.STRUCT: - self.o3 = MetaException() + self.o3 = UnknownDBException() self.o3.read(iprot) else: iprot.skip(ftype) @@ -6975,10 +9537,13 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('create_type_result') + oprot.writeStructBegin('get_table_objects_by_name_result') if self.success is not None: - oprot.writeFieldBegin('success', TType.BOOL, 0) - oprot.writeBool(self.success) + oprot.writeFieldBegin('success', TType.LIST, 0) + oprot.writeListBegin(TType.STRUCT, len(self.success)) + for iter426 in self.success: + iter426.write(oprot) + oprot.writeListEnd() oprot.writeFieldEnd() if self.o1 is not None: oprot.writeFieldBegin('o1', TType.STRUCT, 1) @@ -7010,19 +9575,25 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class drop_type_args: +class get_table_names_by_filter_args: """ Attributes: - - type + - dbname + - filter + - max_tables """ thrift_spec = ( None, # 0 - (1, TType.STRING, 'type', None, None, ), # 1 + (1, TType.STRING, 'dbname', None, None, ), # 1 + (2, TType.STRING, 'filter', None, None, ), # 2 + (3, TType.I16, 'max_tables', None, -1, ), # 3 ) - def __init__(self, type=None,): - self.type = type + def __init__(self, dbname=None, filter=None, max_tables=thrift_spec[3][4],): + self.dbname = dbname + self.filter = filter + self.max_tables = max_tables def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -7035,7 +9606,17 @@ def read(self, iprot): break if fid == 1: if ftype == TType.STRING: - self.type = iprot.readString(); + self.dbname = iprot.readString(); + else: + iprot.skip(ftype) + elif fid == 2: + if ftype == TType.STRING: + self.filter = iprot.readString(); + else: + iprot.skip(ftype) + elif fid == 3: + if ftype == TType.I16: + self.max_tables = iprot.readI16(); else: iprot.skip(ftype) else: @@ -7047,10 +9628,18 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('drop_type_args') - if self.type is not None: - oprot.writeFieldBegin('type', TType.STRING, 1) - oprot.writeString(self.type) + oprot.writeStructBegin('get_table_names_by_filter_args') + if self.dbname is not None: + oprot.writeFieldBegin('dbname', TType.STRING, 1) + oprot.writeString(self.dbname) + oprot.writeFieldEnd() + if self.filter is not None: + oprot.writeFieldBegin('filter', TType.STRING, 2) + oprot.writeString(self.filter) + oprot.writeFieldEnd() + if self.max_tables is not None: + oprot.writeFieldBegin('max_tables', TType.I16, 3) + oprot.writeI16(self.max_tables) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -7070,24 +9659,27 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class drop_type_result: +class get_table_names_by_filter_result: """ Attributes: - success - o1 - o2 + - o3 """ thrift_spec = ( - (0, TType.BOOL, 'success', None, None, ), # 0 + (0, TType.LIST, 'success', (TType.STRING,None), None, ), # 0 (1, TType.STRUCT, 'o1', (MetaException, MetaException.thrift_spec), None, ), # 1 - (2, TType.STRUCT, 'o2', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 2 + (2, TType.STRUCT, 'o2', (InvalidOperationException, InvalidOperationException.thrift_spec), None, ), # 2 + (3, TType.STRUCT, 'o3', (UnknownDBException, UnknownDBException.thrift_spec), None, ), # 3 ) - def __init__(self, success=None, o1=None, o2=None,): + def __init__(self, success=None, o1=None, o2=None, o3=None,): self.success = success self.o1 = o1 self.o2 = o2 + self.o3 = o3 def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -7099,8 +9691,13 @@ def read(self, iprot): if ftype == TType.STOP: break if fid == 0: - if ftype == TType.BOOL: - self.success = iprot.readBool(); + if ftype == TType.LIST: + self.success = [] + (_etype430, _size427) = iprot.readListBegin() + for _i431 in xrange(_size427): + _elem432 = iprot.readString(); + self.success.append(_elem432) + iprot.readListEnd() else: iprot.skip(ftype) elif fid == 1: @@ -7111,10 +9708,16 @@ def read(self, iprot): iprot.skip(ftype) elif fid == 2: if ftype == TType.STRUCT: - self.o2 = NoSuchObjectException() + self.o2 = InvalidOperationException() self.o2.read(iprot) else: iprot.skip(ftype) + elif fid == 3: + if ftype == TType.STRUCT: + self.o3 = UnknownDBException() + self.o3.read(iprot) + else: + iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() @@ -7124,10 +9727,13 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('drop_type_result') + oprot.writeStructBegin('get_table_names_by_filter_result') if self.success is not None: - oprot.writeFieldBegin('success', TType.BOOL, 0) - oprot.writeBool(self.success) + oprot.writeFieldBegin('success', TType.LIST, 0) + oprot.writeListBegin(TType.STRING, len(self.success)) + for iter433 in self.success: + oprot.writeString(iter433) + oprot.writeListEnd() oprot.writeFieldEnd() if self.o1 is not None: oprot.writeFieldBegin('o1', TType.STRUCT, 1) @@ -7137,6 +9743,10 @@ def write(self, oprot): oprot.writeFieldBegin('o2', TType.STRUCT, 2) self.o2.write(oprot) oprot.writeFieldEnd() + if self.o3 is not None: + oprot.writeFieldBegin('o3', TType.STRUCT, 3) + self.o3.write(oprot) + oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -7155,19 +9765,25 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class get_type_all_args: +class alter_table_args: """ Attributes: - - name + - dbname + - tbl_name + - new_tbl """ thrift_spec = ( None, # 0 - (1, TType.STRING, 'name', None, None, ), # 1 + (1, TType.STRING, 'dbname', None, None, ), # 1 + (2, TType.STRING, 'tbl_name', None, None, ), # 2 + (3, TType.STRUCT, 'new_tbl', (Table, Table.thrift_spec), None, ), # 3 ) - def __init__(self, name=None,): - self.name = name + def __init__(self, dbname=None, tbl_name=None, new_tbl=None,): + self.dbname = dbname + self.tbl_name = tbl_name + self.new_tbl = new_tbl def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -7180,7 +9796,18 @@ def read(self, iprot): break if fid == 1: if ftype == TType.STRING: - self.name = iprot.readString(); + self.dbname = iprot.readString(); + else: + iprot.skip(ftype) + elif fid == 2: + if ftype == TType.STRING: + self.tbl_name = iprot.readString(); + else: + iprot.skip(ftype) + elif fid == 3: + if ftype == TType.STRUCT: + self.new_tbl = Table() + self.new_tbl.read(iprot) else: iprot.skip(ftype) else: @@ -7192,10 +9819,18 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('get_type_all_args') - if self.name is not None: - oprot.writeFieldBegin('name', TType.STRING, 1) - oprot.writeString(self.name) + oprot.writeStructBegin('alter_table_args') + if self.dbname is not None: + oprot.writeFieldBegin('dbname', TType.STRING, 1) + oprot.writeString(self.dbname) + oprot.writeFieldEnd() + if self.tbl_name is not None: + oprot.writeFieldBegin('tbl_name', TType.STRING, 2) + oprot.writeString(self.tbl_name) + oprot.writeFieldEnd() + if self.new_tbl is not None: + oprot.writeFieldBegin('new_tbl', TType.STRUCT, 3) + self.new_tbl.write(oprot) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -7215,20 +9850,21 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class get_type_all_result: +class alter_table_result: """ Attributes: - - success + - o1 - o2 """ thrift_spec = ( - (0, TType.MAP, 'success', (TType.STRING,None,TType.STRUCT,(Type, Type.thrift_spec)), None, ), # 0 - (1, TType.STRUCT, 'o2', (MetaException, MetaException.thrift_spec), None, ), # 1 + None, # 0 + (1, TType.STRUCT, 'o1', (InvalidOperationException, InvalidOperationException.thrift_spec), None, ), # 1 + (2, TType.STRUCT, 'o2', (MetaException, MetaException.thrift_spec), None, ), # 2 ) - def __init__(self, success=None, o2=None,): - self.success = success + def __init__(self, o1=None, o2=None,): + self.o1 = o1 self.o2 = o2 def read(self, iprot): @@ -7240,19 +9876,13 @@ def read(self, iprot): (fname, ftype, fid) = iprot.readFieldBegin() if ftype == TType.STOP: break - if fid == 0: - if ftype == TType.MAP: - self.success = {} - (_ktype335, _vtype336, _size334 ) = iprot.readMapBegin() - for _i338 in xrange(_size334): - _key339 = iprot.readString(); - _val340 = Type() - _val340.read(iprot) - self.success[_key339] = _val340 - iprot.readMapEnd() + if fid == 1: + if ftype == TType.STRUCT: + self.o1 = InvalidOperationException() + self.o1.read(iprot) else: iprot.skip(ftype) - elif fid == 1: + elif fid == 2: if ftype == TType.STRUCT: self.o2 = MetaException() self.o2.read(iprot) @@ -7267,17 +9897,13 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('get_type_all_result') - if self.success is not None: - oprot.writeFieldBegin('success', TType.MAP, 0) - oprot.writeMapBegin(TType.STRING, TType.STRUCT, len(self.success)) - for kiter341,viter342 in self.success.items(): - oprot.writeString(kiter341) - viter342.write(oprot) - oprot.writeMapEnd() + oprot.writeStructBegin('alter_table_result') + if self.o1 is not None: + oprot.writeFieldBegin('o1', TType.STRUCT, 1) + self.o1.write(oprot) oprot.writeFieldEnd() if self.o2 is not None: - oprot.writeFieldBegin('o2', TType.STRUCT, 1) + oprot.writeFieldBegin('o2', TType.STRUCT, 2) self.o2.write(oprot) oprot.writeFieldEnd() oprot.writeFieldStop() @@ -7298,22 +9924,28 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class get_fields_args: +class alter_table_with_environment_context_args: """ Attributes: - - db_name - - table_name + - dbname + - tbl_name + - new_tbl + - environment_context """ thrift_spec = ( None, # 0 - (1, TType.STRING, 'db_name', None, None, ), # 1 - (2, TType.STRING, 'table_name', None, None, ), # 2 + (1, TType.STRING, 'dbname', None, None, ), # 1 + (2, TType.STRING, 'tbl_name', None, None, ), # 2 + (3, TType.STRUCT, 'new_tbl', (Table, Table.thrift_spec), None, ), # 3 + (4, TType.STRUCT, 'environment_context', (EnvironmentContext, EnvironmentContext.thrift_spec), None, ), # 4 ) - def __init__(self, db_name=None, table_name=None,): - self.db_name = db_name - self.table_name = table_name + def __init__(self, dbname=None, tbl_name=None, new_tbl=None, environment_context=None,): + self.dbname = dbname + self.tbl_name = tbl_name + self.new_tbl = new_tbl + self.environment_context = environment_context def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -7326,12 +9958,24 @@ def read(self, iprot): break if fid == 1: if ftype == TType.STRING: - self.db_name = iprot.readString(); + self.dbname = iprot.readString(); else: iprot.skip(ftype) elif fid == 2: if ftype == TType.STRING: - self.table_name = iprot.readString(); + self.tbl_name = iprot.readString(); + else: + iprot.skip(ftype) + elif fid == 3: + if ftype == TType.STRUCT: + self.new_tbl = Table() + self.new_tbl.read(iprot) + else: + iprot.skip(ftype) + elif fid == 4: + if ftype == TType.STRUCT: + self.environment_context = EnvironmentContext() + self.environment_context.read(iprot) else: iprot.skip(ftype) else: @@ -7343,14 +9987,22 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('get_fields_args') - if self.db_name is not None: - oprot.writeFieldBegin('db_name', TType.STRING, 1) - oprot.writeString(self.db_name) + oprot.writeStructBegin('alter_table_with_environment_context_args') + if self.dbname is not None: + oprot.writeFieldBegin('dbname', TType.STRING, 1) + oprot.writeString(self.dbname) oprot.writeFieldEnd() - if self.table_name is not None: - oprot.writeFieldBegin('table_name', TType.STRING, 2) - oprot.writeString(self.table_name) + if self.tbl_name is not None: + oprot.writeFieldBegin('tbl_name', TType.STRING, 2) + oprot.writeString(self.tbl_name) + oprot.writeFieldEnd() + if self.new_tbl is not None: + oprot.writeFieldBegin('new_tbl', TType.STRUCT, 3) + self.new_tbl.write(oprot) + oprot.writeFieldEnd() + if self.environment_context is not None: + oprot.writeFieldBegin('environment_context', TType.STRUCT, 4) + self.environment_context.write(oprot) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -7370,27 +10022,22 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class get_fields_result: +class alter_table_with_environment_context_result: """ Attributes: - - success - o1 - o2 - - o3 """ thrift_spec = ( - (0, TType.LIST, 'success', (TType.STRUCT,(FieldSchema, FieldSchema.thrift_spec)), None, ), # 0 - (1, TType.STRUCT, 'o1', (MetaException, MetaException.thrift_spec), None, ), # 1 - (2, TType.STRUCT, 'o2', (UnknownTableException, UnknownTableException.thrift_spec), None, ), # 2 - (3, TType.STRUCT, 'o3', (UnknownDBException, UnknownDBException.thrift_spec), None, ), # 3 + None, # 0 + (1, TType.STRUCT, 'o1', (InvalidOperationException, InvalidOperationException.thrift_spec), None, ), # 1 + (2, TType.STRUCT, 'o2', (MetaException, MetaException.thrift_spec), None, ), # 2 ) - def __init__(self, success=None, o1=None, o2=None, o3=None,): - self.success = success + def __init__(self, o1=None, o2=None,): self.o1 = o1 self.o2 = o2 - self.o3 = o3 def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -7401,35 +10048,18 @@ def read(self, iprot): (fname, ftype, fid) = iprot.readFieldBegin() if ftype == TType.STOP: break - if fid == 0: - if ftype == TType.LIST: - self.success = [] - (_etype346, _size343) = iprot.readListBegin() - for _i347 in xrange(_size343): - _elem348 = FieldSchema() - _elem348.read(iprot) - self.success.append(_elem348) - iprot.readListEnd() - else: - iprot.skip(ftype) - elif fid == 1: + if fid == 1: if ftype == TType.STRUCT: - self.o1 = MetaException() + self.o1 = InvalidOperationException() self.o1.read(iprot) else: iprot.skip(ftype) elif fid == 2: if ftype == TType.STRUCT: - self.o2 = UnknownTableException() + self.o2 = MetaException() self.o2.read(iprot) else: iprot.skip(ftype) - elif fid == 3: - if ftype == TType.STRUCT: - self.o3 = UnknownDBException() - self.o3.read(iprot) - else: - iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() @@ -7439,14 +10069,7 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('get_fields_result') - if self.success is not None: - oprot.writeFieldBegin('success', TType.LIST, 0) - oprot.writeListBegin(TType.STRUCT, len(self.success)) - for iter349 in self.success: - iter349.write(oprot) - oprot.writeListEnd() - oprot.writeFieldEnd() + oprot.writeStructBegin('alter_table_with_environment_context_result') if self.o1 is not None: oprot.writeFieldBegin('o1', TType.STRUCT, 1) self.o1.write(oprot) @@ -7455,10 +10078,6 @@ def write(self, oprot): oprot.writeFieldBegin('o2', TType.STRUCT, 2) self.o2.write(oprot) oprot.writeFieldEnd() - if self.o3 is not None: - oprot.writeFieldBegin('o3', TType.STRUCT, 3) - self.o3.write(oprot) - oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -7477,22 +10096,19 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class get_schema_args: +class add_partition_args: """ Attributes: - - db_name - - table_name + - new_part """ thrift_spec = ( None, # 0 - (1, TType.STRING, 'db_name', None, None, ), # 1 - (2, TType.STRING, 'table_name', None, None, ), # 2 + (1, TType.STRUCT, 'new_part', (Partition, Partition.thrift_spec), None, ), # 1 ) - def __init__(self, db_name=None, table_name=None,): - self.db_name = db_name - self.table_name = table_name + def __init__(self, new_part=None,): + self.new_part = new_part def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -7504,13 +10120,9 @@ def read(self, iprot): if ftype == TType.STOP: break if fid == 1: - if ftype == TType.STRING: - self.db_name = iprot.readString(); - else: - iprot.skip(ftype) - elif fid == 2: - if ftype == TType.STRING: - self.table_name = iprot.readString(); + if ftype == TType.STRUCT: + self.new_part = Partition() + self.new_part.read(iprot) else: iprot.skip(ftype) else: @@ -7522,14 +10134,10 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('get_schema_args') - if self.db_name is not None: - oprot.writeFieldBegin('db_name', TType.STRING, 1) - oprot.writeString(self.db_name) - oprot.writeFieldEnd() - if self.table_name is not None: - oprot.writeFieldBegin('table_name', TType.STRING, 2) - oprot.writeString(self.table_name) + oprot.writeStructBegin('add_partition_args') + if self.new_part is not None: + oprot.writeFieldBegin('new_part', TType.STRUCT, 1) + self.new_part.write(oprot) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -7549,7 +10157,7 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class get_schema_result: +class add_partition_result: """ Attributes: - success @@ -7559,10 +10167,10 @@ class get_schema_result: """ thrift_spec = ( - (0, TType.LIST, 'success', (TType.STRUCT,(FieldSchema, FieldSchema.thrift_spec)), None, ), # 0 - (1, TType.STRUCT, 'o1', (MetaException, MetaException.thrift_spec), None, ), # 1 - (2, TType.STRUCT, 'o2', (UnknownTableException, UnknownTableException.thrift_spec), None, ), # 2 - (3, TType.STRUCT, 'o3', (UnknownDBException, UnknownDBException.thrift_spec), None, ), # 3 + (0, TType.STRUCT, 'success', (Partition, Partition.thrift_spec), None, ), # 0 + (1, TType.STRUCT, 'o1', (InvalidObjectException, InvalidObjectException.thrift_spec), None, ), # 1 + (2, TType.STRUCT, 'o2', (AlreadyExistsException, AlreadyExistsException.thrift_spec), None, ), # 2 + (3, TType.STRUCT, 'o3', (MetaException, MetaException.thrift_spec), None, ), # 3 ) def __init__(self, success=None, o1=None, o2=None, o3=None,): @@ -7581,31 +10189,26 @@ def read(self, iprot): if ftype == TType.STOP: break if fid == 0: - if ftype == TType.LIST: - self.success = [] - (_etype353, _size350) = iprot.readListBegin() - for _i354 in xrange(_size350): - _elem355 = FieldSchema() - _elem355.read(iprot) - self.success.append(_elem355) - iprot.readListEnd() + if ftype == TType.STRUCT: + self.success = Partition() + self.success.read(iprot) else: iprot.skip(ftype) elif fid == 1: if ftype == TType.STRUCT: - self.o1 = MetaException() + self.o1 = InvalidObjectException() self.o1.read(iprot) else: iprot.skip(ftype) elif fid == 2: if ftype == TType.STRUCT: - self.o2 = UnknownTableException() + self.o2 = AlreadyExistsException() self.o2.read(iprot) else: iprot.skip(ftype) elif fid == 3: if ftype == TType.STRUCT: - self.o3 = UnknownDBException() + self.o3 = MetaException() self.o3.read(iprot) else: iprot.skip(ftype) @@ -7618,13 +10221,10 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('get_schema_result') + oprot.writeStructBegin('add_partition_result') if self.success is not None: - oprot.writeFieldBegin('success', TType.LIST, 0) - oprot.writeListBegin(TType.STRUCT, len(self.success)) - for iter356 in self.success: - iter356.write(oprot) - oprot.writeListEnd() + oprot.writeFieldBegin('success', TType.STRUCT, 0) + self.success.write(oprot) oprot.writeFieldEnd() if self.o1 is not None: oprot.writeFieldBegin('o1', TType.STRUCT, 1) @@ -7656,19 +10256,22 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class create_table_args: +class add_partition_with_environment_context_args: """ Attributes: - - tbl + - new_part + - environment_context """ thrift_spec = ( None, # 0 - (1, TType.STRUCT, 'tbl', (Table, Table.thrift_spec), None, ), # 1 + (1, TType.STRUCT, 'new_part', (Partition, Partition.thrift_spec), None, ), # 1 + (2, TType.STRUCT, 'environment_context', (EnvironmentContext, EnvironmentContext.thrift_spec), None, ), # 2 ) - def __init__(self, tbl=None,): - self.tbl = tbl + def __init__(self, new_part=None, environment_context=None,): + self.new_part = new_part + self.environment_context = environment_context def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -7681,8 +10284,14 @@ def read(self, iprot): break if fid == 1: if ftype == TType.STRUCT: - self.tbl = Table() - self.tbl.read(iprot) + self.new_part = Partition() + self.new_part.read(iprot) + else: + iprot.skip(ftype) + elif fid == 2: + if ftype == TType.STRUCT: + self.environment_context = EnvironmentContext() + self.environment_context.read(iprot) else: iprot.skip(ftype) else: @@ -7694,10 +10303,14 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('create_table_args') - if self.tbl is not None: - oprot.writeFieldBegin('tbl', TType.STRUCT, 1) - self.tbl.write(oprot) + oprot.writeStructBegin('add_partition_with_environment_context_args') + if self.new_part is not None: + oprot.writeFieldBegin('new_part', TType.STRUCT, 1) + self.new_part.write(oprot) + oprot.writeFieldEnd() + if self.environment_context is not None: + oprot.writeFieldBegin('environment_context', TType.STRUCT, 2) + self.environment_context.write(oprot) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -7717,28 +10330,27 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class create_table_result: +class add_partition_with_environment_context_result: """ Attributes: + - success - o1 - o2 - o3 - - o4 """ thrift_spec = ( - None, # 0 - (1, TType.STRUCT, 'o1', (AlreadyExistsException, AlreadyExistsException.thrift_spec), None, ), # 1 - (2, TType.STRUCT, 'o2', (InvalidObjectException, InvalidObjectException.thrift_spec), None, ), # 2 + (0, TType.STRUCT, 'success', (Partition, Partition.thrift_spec), None, ), # 0 + (1, TType.STRUCT, 'o1', (InvalidObjectException, InvalidObjectException.thrift_spec), None, ), # 1 + (2, TType.STRUCT, 'o2', (AlreadyExistsException, AlreadyExistsException.thrift_spec), None, ), # 2 (3, TType.STRUCT, 'o3', (MetaException, MetaException.thrift_spec), None, ), # 3 - (4, TType.STRUCT, 'o4', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 4 ) - def __init__(self, o1=None, o2=None, o3=None, o4=None,): + def __init__(self, success=None, o1=None, o2=None, o3=None,): + self.success = success self.o1 = o1 self.o2 = o2 self.o3 = o3 - self.o4 = o4 def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -7749,15 +10361,21 @@ def read(self, iprot): (fname, ftype, fid) = iprot.readFieldBegin() if ftype == TType.STOP: break - if fid == 1: + if fid == 0: if ftype == TType.STRUCT: - self.o1 = AlreadyExistsException() + self.success = Partition() + self.success.read(iprot) + else: + iprot.skip(ftype) + elif fid == 1: + if ftype == TType.STRUCT: + self.o1 = InvalidObjectException() self.o1.read(iprot) else: iprot.skip(ftype) elif fid == 2: if ftype == TType.STRUCT: - self.o2 = InvalidObjectException() + self.o2 = AlreadyExistsException() self.o2.read(iprot) else: iprot.skip(ftype) @@ -7767,12 +10385,6 @@ def read(self, iprot): self.o3.read(iprot) else: iprot.skip(ftype) - elif fid == 4: - if ftype == TType.STRUCT: - self.o4 = NoSuchObjectException() - self.o4.read(iprot) - else: - iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() @@ -7782,7 +10394,11 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('create_table_result') + oprot.writeStructBegin('add_partition_with_environment_context_result') + if self.success is not None: + oprot.writeFieldBegin('success', TType.STRUCT, 0) + self.success.write(oprot) + oprot.writeFieldEnd() if self.o1 is not None: oprot.writeFieldBegin('o1', TType.STRUCT, 1) self.o1.write(oprot) @@ -7795,10 +10411,6 @@ def write(self, oprot): oprot.writeFieldBegin('o3', TType.STRUCT, 3) self.o3.write(oprot) oprot.writeFieldEnd() - if self.o4 is not None: - oprot.writeFieldBegin('o4', TType.STRUCT, 4) - self.o4.write(oprot) - oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -7817,22 +10429,19 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class create_table_with_environment_context_args: +class add_partitions_args: """ Attributes: - - tbl - - environment_context + - new_parts """ thrift_spec = ( None, # 0 - (1, TType.STRUCT, 'tbl', (Table, Table.thrift_spec), None, ), # 1 - (2, TType.STRUCT, 'environment_context', (EnvironmentContext, EnvironmentContext.thrift_spec), None, ), # 2 + (1, TType.LIST, 'new_parts', (TType.STRUCT,(Partition, Partition.thrift_spec)), None, ), # 1 ) - def __init__(self, tbl=None, environment_context=None,): - self.tbl = tbl - self.environment_context = environment_context + def __init__(self, new_parts=None,): + self.new_parts = new_parts def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -7844,15 +10453,14 @@ def read(self, iprot): if ftype == TType.STOP: break if fid == 1: - if ftype == TType.STRUCT: - self.tbl = Table() - self.tbl.read(iprot) - else: - iprot.skip(ftype) - elif fid == 2: - if ftype == TType.STRUCT: - self.environment_context = EnvironmentContext() - self.environment_context.read(iprot) + if ftype == TType.LIST: + self.new_parts = [] + (_etype437, _size434) = iprot.readListBegin() + for _i438 in xrange(_size434): + _elem439 = Partition() + _elem439.read(iprot) + self.new_parts.append(_elem439) + iprot.readListEnd() else: iprot.skip(ftype) else: @@ -7864,14 +10472,13 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('create_table_with_environment_context_args') - if self.tbl is not None: - oprot.writeFieldBegin('tbl', TType.STRUCT, 1) - self.tbl.write(oprot) - oprot.writeFieldEnd() - if self.environment_context is not None: - oprot.writeFieldBegin('environment_context', TType.STRUCT, 2) - self.environment_context.write(oprot) + oprot.writeStructBegin('add_partitions_args') + if self.new_parts is not None: + oprot.writeFieldBegin('new_parts', TType.LIST, 1) + oprot.writeListBegin(TType.STRUCT, len(self.new_parts)) + for iter440 in self.new_parts: + iter440.write(oprot) + oprot.writeListEnd() oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -7891,28 +10498,27 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class create_table_with_environment_context_result: +class add_partitions_result: """ Attributes: + - success - o1 - o2 - o3 - - o4 """ thrift_spec = ( - None, # 0 - (1, TType.STRUCT, 'o1', (AlreadyExistsException, AlreadyExistsException.thrift_spec), None, ), # 1 - (2, TType.STRUCT, 'o2', (InvalidObjectException, InvalidObjectException.thrift_spec), None, ), # 2 + (0, TType.I32, 'success', None, None, ), # 0 + (1, TType.STRUCT, 'o1', (InvalidObjectException, InvalidObjectException.thrift_spec), None, ), # 1 + (2, TType.STRUCT, 'o2', (AlreadyExistsException, AlreadyExistsException.thrift_spec), None, ), # 2 (3, TType.STRUCT, 'o3', (MetaException, MetaException.thrift_spec), None, ), # 3 - (4, TType.STRUCT, 'o4', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 4 ) - def __init__(self, o1=None, o2=None, o3=None, o4=None,): + def __init__(self, success=None, o1=None, o2=None, o3=None,): + self.success = success self.o1 = o1 self.o2 = o2 self.o3 = o3 - self.o4 = o4 def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -7923,15 +10529,20 @@ def read(self, iprot): (fname, ftype, fid) = iprot.readFieldBegin() if ftype == TType.STOP: break - if fid == 1: + if fid == 0: + if ftype == TType.I32: + self.success = iprot.readI32(); + else: + iprot.skip(ftype) + elif fid == 1: if ftype == TType.STRUCT: - self.o1 = AlreadyExistsException() + self.o1 = InvalidObjectException() self.o1.read(iprot) else: iprot.skip(ftype) elif fid == 2: if ftype == TType.STRUCT: - self.o2 = InvalidObjectException() + self.o2 = AlreadyExistsException() self.o2.read(iprot) else: iprot.skip(ftype) @@ -7941,12 +10552,6 @@ def read(self, iprot): self.o3.read(iprot) else: iprot.skip(ftype) - elif fid == 4: - if ftype == TType.STRUCT: - self.o4 = NoSuchObjectException() - self.o4.read(iprot) - else: - iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() @@ -7956,7 +10561,11 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('create_table_with_environment_context_result') + oprot.writeStructBegin('add_partitions_result') + if self.success is not None: + oprot.writeFieldBegin('success', TType.I32, 0) + oprot.writeI32(self.success) + oprot.writeFieldEnd() if self.o1 is not None: oprot.writeFieldBegin('o1', TType.STRUCT, 1) self.o1.write(oprot) @@ -7969,10 +10578,6 @@ def write(self, oprot): oprot.writeFieldBegin('o3', TType.STRUCT, 3) self.o3.write(oprot) oprot.writeFieldEnd() - if self.o4 is not None: - oprot.writeFieldBegin('o4', TType.STRUCT, 4) - self.o4.write(oprot) - oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -7991,25 +10596,25 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class drop_table_args: +class append_partition_args: """ Attributes: - - dbname - - name - - deleteData + - db_name + - tbl_name + - part_vals """ thrift_spec = ( None, # 0 - (1, TType.STRING, 'dbname', None, None, ), # 1 - (2, TType.STRING, 'name', None, None, ), # 2 - (3, TType.BOOL, 'deleteData', None, None, ), # 3 + (1, TType.STRING, 'db_name', None, None, ), # 1 + (2, TType.STRING, 'tbl_name', None, None, ), # 2 + (3, TType.LIST, 'part_vals', (TType.STRING,None), None, ), # 3 ) - def __init__(self, dbname=None, name=None, deleteData=None,): - self.dbname = dbname - self.name = name - self.deleteData = deleteData + def __init__(self, db_name=None, tbl_name=None, part_vals=None,): + self.db_name = db_name + self.tbl_name = tbl_name + self.part_vals = part_vals def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -8022,17 +10627,22 @@ def read(self, iprot): break if fid == 1: if ftype == TType.STRING: - self.dbname = iprot.readString(); + self.db_name = iprot.readString(); else: iprot.skip(ftype) elif fid == 2: if ftype == TType.STRING: - self.name = iprot.readString(); + self.tbl_name = iprot.readString(); else: iprot.skip(ftype) elif fid == 3: - if ftype == TType.BOOL: - self.deleteData = iprot.readBool(); + if ftype == TType.LIST: + self.part_vals = [] + (_etype444, _size441) = iprot.readListBegin() + for _i445 in xrange(_size441): + _elem446 = iprot.readString(); + self.part_vals.append(_elem446) + iprot.readListEnd() else: iprot.skip(ftype) else: @@ -8044,18 +10654,21 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('drop_table_args') - if self.dbname is not None: - oprot.writeFieldBegin('dbname', TType.STRING, 1) - oprot.writeString(self.dbname) + oprot.writeStructBegin('append_partition_args') + if self.db_name is not None: + oprot.writeFieldBegin('db_name', TType.STRING, 1) + oprot.writeString(self.db_name) oprot.writeFieldEnd() - if self.name is not None: - oprot.writeFieldBegin('name', TType.STRING, 2) - oprot.writeString(self.name) + if self.tbl_name is not None: + oprot.writeFieldBegin('tbl_name', TType.STRING, 2) + oprot.writeString(self.tbl_name) oprot.writeFieldEnd() - if self.deleteData is not None: - oprot.writeFieldBegin('deleteData', TType.BOOL, 3) - oprot.writeBool(self.deleteData) + if self.part_vals is not None: + oprot.writeFieldBegin('part_vals', TType.LIST, 3) + oprot.writeListBegin(TType.STRING, len(self.part_vals)) + for iter447 in self.part_vals: + oprot.writeString(iter447) + oprot.writeListEnd() oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -8075,21 +10688,26 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class drop_table_result: +class append_partition_result: """ Attributes: + - success - o1 + - o2 - o3 """ thrift_spec = ( - None, # 0 - (1, TType.STRUCT, 'o1', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 1 - (2, TType.STRUCT, 'o3', (MetaException, MetaException.thrift_spec), None, ), # 2 + (0, TType.STRUCT, 'success', (Partition, Partition.thrift_spec), None, ), # 0 + (1, TType.STRUCT, 'o1', (InvalidObjectException, InvalidObjectException.thrift_spec), None, ), # 1 + (2, TType.STRUCT, 'o2', (AlreadyExistsException, AlreadyExistsException.thrift_spec), None, ), # 2 + (3, TType.STRUCT, 'o3', (MetaException, MetaException.thrift_spec), None, ), # 3 ) - def __init__(self, o1=None, o3=None,): + def __init__(self, success=None, o1=None, o2=None, o3=None,): + self.success = success self.o1 = o1 + self.o2 = o2 self.o3 = o3 def read(self, iprot): @@ -8101,13 +10719,25 @@ def read(self, iprot): (fname, ftype, fid) = iprot.readFieldBegin() if ftype == TType.STOP: break - if fid == 1: + if fid == 0: if ftype == TType.STRUCT: - self.o1 = NoSuchObjectException() + self.success = Partition() + self.success.read(iprot) + else: + iprot.skip(ftype) + elif fid == 1: + if ftype == TType.STRUCT: + self.o1 = InvalidObjectException() self.o1.read(iprot) else: iprot.skip(ftype) - elif fid == 2: + elif fid == 2: + if ftype == TType.STRUCT: + self.o2 = AlreadyExistsException() + self.o2.read(iprot) + else: + iprot.skip(ftype) + elif fid == 3: if ftype == TType.STRUCT: self.o3 = MetaException() self.o3.read(iprot) @@ -8122,13 +10752,21 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('drop_table_result') + oprot.writeStructBegin('append_partition_result') + if self.success is not None: + oprot.writeFieldBegin('success', TType.STRUCT, 0) + self.success.write(oprot) + oprot.writeFieldEnd() if self.o1 is not None: oprot.writeFieldBegin('o1', TType.STRUCT, 1) self.o1.write(oprot) oprot.writeFieldEnd() + if self.o2 is not None: + oprot.writeFieldBegin('o2', TType.STRUCT, 2) + self.o2.write(oprot) + oprot.writeFieldEnd() if self.o3 is not None: - oprot.writeFieldBegin('o3', TType.STRUCT, 2) + oprot.writeFieldBegin('o3', TType.STRUCT, 3) self.o3.write(oprot) oprot.writeFieldEnd() oprot.writeFieldStop() @@ -8149,28 +10787,19 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class drop_table_with_environment_context_args: +class add_partitions_req_args: """ Attributes: - - dbname - - name - - deleteData - - environment_context + - request """ thrift_spec = ( None, # 0 - (1, TType.STRING, 'dbname', None, None, ), # 1 - (2, TType.STRING, 'name', None, None, ), # 2 - (3, TType.BOOL, 'deleteData', None, None, ), # 3 - (4, TType.STRUCT, 'environment_context', (EnvironmentContext, EnvironmentContext.thrift_spec), None, ), # 4 + (1, TType.STRUCT, 'request', (AddPartitionsRequest, AddPartitionsRequest.thrift_spec), None, ), # 1 ) - def __init__(self, dbname=None, name=None, deleteData=None, environment_context=None,): - self.dbname = dbname - self.name = name - self.deleteData = deleteData - self.environment_context = environment_context + def __init__(self, request=None,): + self.request = request def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -8182,24 +10811,9 @@ def read(self, iprot): if ftype == TType.STOP: break if fid == 1: - if ftype == TType.STRING: - self.dbname = iprot.readString(); - else: - iprot.skip(ftype) - elif fid == 2: - if ftype == TType.STRING: - self.name = iprot.readString(); - else: - iprot.skip(ftype) - elif fid == 3: - if ftype == TType.BOOL: - self.deleteData = iprot.readBool(); - else: - iprot.skip(ftype) - elif fid == 4: if ftype == TType.STRUCT: - self.environment_context = EnvironmentContext() - self.environment_context.read(iprot) + self.request = AddPartitionsRequest() + self.request.read(iprot) else: iprot.skip(ftype) else: @@ -8211,22 +10825,10 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('drop_table_with_environment_context_args') - if self.dbname is not None: - oprot.writeFieldBegin('dbname', TType.STRING, 1) - oprot.writeString(self.dbname) - oprot.writeFieldEnd() - if self.name is not None: - oprot.writeFieldBegin('name', TType.STRING, 2) - oprot.writeString(self.name) - oprot.writeFieldEnd() - if self.deleteData is not None: - oprot.writeFieldBegin('deleteData', TType.BOOL, 3) - oprot.writeBool(self.deleteData) - oprot.writeFieldEnd() - if self.environment_context is not None: - oprot.writeFieldBegin('environment_context', TType.STRUCT, 4) - self.environment_context.write(oprot) + oprot.writeStructBegin('add_partitions_req_args') + if self.request is not None: + oprot.writeFieldBegin('request', TType.STRUCT, 1) + self.request.write(oprot) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -8246,21 +10848,26 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class drop_table_with_environment_context_result: +class add_partitions_req_result: """ Attributes: + - success - o1 + - o2 - o3 """ thrift_spec = ( - None, # 0 - (1, TType.STRUCT, 'o1', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 1 - (2, TType.STRUCT, 'o3', (MetaException, MetaException.thrift_spec), None, ), # 2 + (0, TType.STRUCT, 'success', (AddPartitionsResult, AddPartitionsResult.thrift_spec), None, ), # 0 + (1, TType.STRUCT, 'o1', (InvalidObjectException, InvalidObjectException.thrift_spec), None, ), # 1 + (2, TType.STRUCT, 'o2', (AlreadyExistsException, AlreadyExistsException.thrift_spec), None, ), # 2 + (3, TType.STRUCT, 'o3', (MetaException, MetaException.thrift_spec), None, ), # 3 ) - def __init__(self, o1=None, o3=None,): + def __init__(self, success=None, o1=None, o2=None, o3=None,): + self.success = success self.o1 = o1 + self.o2 = o2 self.o3 = o3 def read(self, iprot): @@ -8272,14 +10879,26 @@ def read(self, iprot): (fname, ftype, fid) = iprot.readFieldBegin() if ftype == TType.STOP: break - if fid == 1: + if fid == 0: if ftype == TType.STRUCT: - self.o1 = NoSuchObjectException() + self.success = AddPartitionsResult() + self.success.read(iprot) + else: + iprot.skip(ftype) + elif fid == 1: + if ftype == TType.STRUCT: + self.o1 = InvalidObjectException() self.o1.read(iprot) else: iprot.skip(ftype) elif fid == 2: if ftype == TType.STRUCT: + self.o2 = AlreadyExistsException() + self.o2.read(iprot) + else: + iprot.skip(ftype) + elif fid == 3: + if ftype == TType.STRUCT: self.o3 = MetaException() self.o3.read(iprot) else: @@ -8293,13 +10912,21 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('drop_table_with_environment_context_result') + oprot.writeStructBegin('add_partitions_req_result') + if self.success is not None: + oprot.writeFieldBegin('success', TType.STRUCT, 0) + self.success.write(oprot) + oprot.writeFieldEnd() if self.o1 is not None: oprot.writeFieldBegin('o1', TType.STRUCT, 1) self.o1.write(oprot) oprot.writeFieldEnd() + if self.o2 is not None: + oprot.writeFieldBegin('o2', TType.STRUCT, 2) + self.o2.write(oprot) + oprot.writeFieldEnd() if self.o3 is not None: - oprot.writeFieldBegin('o3', TType.STRUCT, 2) + oprot.writeFieldBegin('o3', TType.STRUCT, 3) self.o3.write(oprot) oprot.writeFieldEnd() oprot.writeFieldStop() @@ -8320,22 +10947,28 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class get_tables_args: +class append_partition_with_environment_context_args: """ Attributes: - db_name - - pattern + - tbl_name + - part_vals + - environment_context """ thrift_spec = ( None, # 0 (1, TType.STRING, 'db_name', None, None, ), # 1 - (2, TType.STRING, 'pattern', None, None, ), # 2 + (2, TType.STRING, 'tbl_name', None, None, ), # 2 + (3, TType.LIST, 'part_vals', (TType.STRING,None), None, ), # 3 + (4, TType.STRUCT, 'environment_context', (EnvironmentContext, EnvironmentContext.thrift_spec), None, ), # 4 ) - def __init__(self, db_name=None, pattern=None,): + def __init__(self, db_name=None, tbl_name=None, part_vals=None, environment_context=None,): self.db_name = db_name - self.pattern = pattern + self.tbl_name = tbl_name + self.part_vals = part_vals + self.environment_context = environment_context def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -8353,7 +10986,23 @@ def read(self, iprot): iprot.skip(ftype) elif fid == 2: if ftype == TType.STRING: - self.pattern = iprot.readString(); + self.tbl_name = iprot.readString(); + else: + iprot.skip(ftype) + elif fid == 3: + if ftype == TType.LIST: + self.part_vals = [] + (_etype451, _size448) = iprot.readListBegin() + for _i452 in xrange(_size448): + _elem453 = iprot.readString(); + self.part_vals.append(_elem453) + iprot.readListEnd() + else: + iprot.skip(ftype) + elif fid == 4: + if ftype == TType.STRUCT: + self.environment_context = EnvironmentContext() + self.environment_context.read(iprot) else: iprot.skip(ftype) else: @@ -8365,14 +11014,25 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('get_tables_args') + oprot.writeStructBegin('append_partition_with_environment_context_args') if self.db_name is not None: oprot.writeFieldBegin('db_name', TType.STRING, 1) oprot.writeString(self.db_name) oprot.writeFieldEnd() - if self.pattern is not None: - oprot.writeFieldBegin('pattern', TType.STRING, 2) - oprot.writeString(self.pattern) + if self.tbl_name is not None: + oprot.writeFieldBegin('tbl_name', TType.STRING, 2) + oprot.writeString(self.tbl_name) + oprot.writeFieldEnd() + if self.part_vals is not None: + oprot.writeFieldBegin('part_vals', TType.LIST, 3) + oprot.writeListBegin(TType.STRING, len(self.part_vals)) + for iter454 in self.part_vals: + oprot.writeString(iter454) + oprot.writeListEnd() + oprot.writeFieldEnd() + if self.environment_context is not None: + oprot.writeFieldBegin('environment_context', TType.STRUCT, 4) + self.environment_context.write(oprot) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -8392,21 +11052,27 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class get_tables_result: +class append_partition_with_environment_context_result: """ Attributes: - success - o1 + - o2 + - o3 """ thrift_spec = ( - (0, TType.LIST, 'success', (TType.STRING,None), None, ), # 0 - (1, TType.STRUCT, 'o1', (MetaException, MetaException.thrift_spec), None, ), # 1 + (0, TType.STRUCT, 'success', (Partition, Partition.thrift_spec), None, ), # 0 + (1, TType.STRUCT, 'o1', (InvalidObjectException, InvalidObjectException.thrift_spec), None, ), # 1 + (2, TType.STRUCT, 'o2', (AlreadyExistsException, AlreadyExistsException.thrift_spec), None, ), # 2 + (3, TType.STRUCT, 'o3', (MetaException, MetaException.thrift_spec), None, ), # 3 ) - def __init__(self, success=None, o1=None,): + def __init__(self, success=None, o1=None, o2=None, o3=None,): self.success = success self.o1 = o1 + self.o2 = o2 + self.o3 = o3 def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -8418,21 +11084,29 @@ def read(self, iprot): if ftype == TType.STOP: break if fid == 0: - if ftype == TType.LIST: - self.success = [] - (_etype360, _size357) = iprot.readListBegin() - for _i361 in xrange(_size357): - _elem362 = iprot.readString(); - self.success.append(_elem362) - iprot.readListEnd() + if ftype == TType.STRUCT: + self.success = Partition() + self.success.read(iprot) else: iprot.skip(ftype) elif fid == 1: if ftype == TType.STRUCT: - self.o1 = MetaException() + self.o1 = InvalidObjectException() self.o1.read(iprot) else: iprot.skip(ftype) + elif fid == 2: + if ftype == TType.STRUCT: + self.o2 = AlreadyExistsException() + self.o2.read(iprot) + else: + iprot.skip(ftype) + elif fid == 3: + if ftype == TType.STRUCT: + self.o3 = MetaException() + self.o3.read(iprot) + else: + iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() @@ -8442,18 +11116,23 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('get_tables_result') + oprot.writeStructBegin('append_partition_with_environment_context_result') if self.success is not None: - oprot.writeFieldBegin('success', TType.LIST, 0) - oprot.writeListBegin(TType.STRING, len(self.success)) - for iter363 in self.success: - oprot.writeString(iter363) - oprot.writeListEnd() + oprot.writeFieldBegin('success', TType.STRUCT, 0) + self.success.write(oprot) oprot.writeFieldEnd() if self.o1 is not None: oprot.writeFieldBegin('o1', TType.STRUCT, 1) self.o1.write(oprot) oprot.writeFieldEnd() + if self.o2 is not None: + oprot.writeFieldBegin('o2', TType.STRUCT, 2) + self.o2.write(oprot) + oprot.writeFieldEnd() + if self.o3 is not None: + oprot.writeFieldBegin('o3', TType.STRUCT, 3) + self.o3.write(oprot) + oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -8472,19 +11151,25 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class get_all_tables_args: +class append_partition_by_name_args: """ Attributes: - db_name + - tbl_name + - part_name """ thrift_spec = ( None, # 0 (1, TType.STRING, 'db_name', None, None, ), # 1 + (2, TType.STRING, 'tbl_name', None, None, ), # 2 + (3, TType.STRING, 'part_name', None, None, ), # 3 ) - def __init__(self, db_name=None,): + def __init__(self, db_name=None, tbl_name=None, part_name=None,): self.db_name = db_name + self.tbl_name = tbl_name + self.part_name = part_name def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -8500,6 +11185,16 @@ def read(self, iprot): self.db_name = iprot.readString(); else: iprot.skip(ftype) + elif fid == 2: + if ftype == TType.STRING: + self.tbl_name = iprot.readString(); + else: + iprot.skip(ftype) + elif fid == 3: + if ftype == TType.STRING: + self.part_name = iprot.readString(); + else: + iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() @@ -8509,11 +11204,19 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('get_all_tables_args') + oprot.writeStructBegin('append_partition_by_name_args') if self.db_name is not None: oprot.writeFieldBegin('db_name', TType.STRING, 1) oprot.writeString(self.db_name) oprot.writeFieldEnd() + if self.tbl_name is not None: + oprot.writeFieldBegin('tbl_name', TType.STRING, 2) + oprot.writeString(self.tbl_name) + oprot.writeFieldEnd() + if self.part_name is not None: + oprot.writeFieldBegin('part_name', TType.STRING, 3) + oprot.writeString(self.part_name) + oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -8532,21 +11235,27 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class get_all_tables_result: +class append_partition_by_name_result: """ Attributes: - success - o1 + - o2 + - o3 """ thrift_spec = ( - (0, TType.LIST, 'success', (TType.STRING,None), None, ), # 0 - (1, TType.STRUCT, 'o1', (MetaException, MetaException.thrift_spec), None, ), # 1 + (0, TType.STRUCT, 'success', (Partition, Partition.thrift_spec), None, ), # 0 + (1, TType.STRUCT, 'o1', (InvalidObjectException, InvalidObjectException.thrift_spec), None, ), # 1 + (2, TType.STRUCT, 'o2', (AlreadyExistsException, AlreadyExistsException.thrift_spec), None, ), # 2 + (3, TType.STRUCT, 'o3', (MetaException, MetaException.thrift_spec), None, ), # 3 ) - def __init__(self, success=None, o1=None,): + def __init__(self, success=None, o1=None, o2=None, o3=None,): self.success = success self.o1 = o1 + self.o2 = o2 + self.o3 = o3 def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -8558,21 +11267,29 @@ def read(self, iprot): if ftype == TType.STOP: break if fid == 0: - if ftype == TType.LIST: - self.success = [] - (_etype367, _size364) = iprot.readListBegin() - for _i368 in xrange(_size364): - _elem369 = iprot.readString(); - self.success.append(_elem369) - iprot.readListEnd() + if ftype == TType.STRUCT: + self.success = Partition() + self.success.read(iprot) else: iprot.skip(ftype) elif fid == 1: if ftype == TType.STRUCT: - self.o1 = MetaException() + self.o1 = InvalidObjectException() self.o1.read(iprot) else: iprot.skip(ftype) + elif fid == 2: + if ftype == TType.STRUCT: + self.o2 = AlreadyExistsException() + self.o2.read(iprot) + else: + iprot.skip(ftype) + elif fid == 3: + if ftype == TType.STRUCT: + self.o3 = MetaException() + self.o3.read(iprot) + else: + iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() @@ -8582,18 +11299,23 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('get_all_tables_result') + oprot.writeStructBegin('append_partition_by_name_result') if self.success is not None: - oprot.writeFieldBegin('success', TType.LIST, 0) - oprot.writeListBegin(TType.STRING, len(self.success)) - for iter370 in self.success: - oprot.writeString(iter370) - oprot.writeListEnd() + oprot.writeFieldBegin('success', TType.STRUCT, 0) + self.success.write(oprot) oprot.writeFieldEnd() if self.o1 is not None: oprot.writeFieldBegin('o1', TType.STRUCT, 1) self.o1.write(oprot) oprot.writeFieldEnd() + if self.o2 is not None: + oprot.writeFieldBegin('o2', TType.STRUCT, 2) + self.o2.write(oprot) + oprot.writeFieldEnd() + if self.o3 is not None: + oprot.writeFieldBegin('o3', TType.STRUCT, 3) + self.o3.write(oprot) + oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -8612,22 +11334,28 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class get_table_args: +class append_partition_by_name_with_environment_context_args: """ Attributes: - - dbname + - db_name - tbl_name + - part_name + - environment_context """ thrift_spec = ( None, # 0 - (1, TType.STRING, 'dbname', None, None, ), # 1 + (1, TType.STRING, 'db_name', None, None, ), # 1 (2, TType.STRING, 'tbl_name', None, None, ), # 2 + (3, TType.STRING, 'part_name', None, None, ), # 3 + (4, TType.STRUCT, 'environment_context', (EnvironmentContext, EnvironmentContext.thrift_spec), None, ), # 4 ) - def __init__(self, dbname=None, tbl_name=None,): - self.dbname = dbname + def __init__(self, db_name=None, tbl_name=None, part_name=None, environment_context=None,): + self.db_name = db_name self.tbl_name = tbl_name + self.part_name = part_name + self.environment_context = environment_context def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -8640,7 +11368,7 @@ def read(self, iprot): break if fid == 1: if ftype == TType.STRING: - self.dbname = iprot.readString(); + self.db_name = iprot.readString(); else: iprot.skip(ftype) elif fid == 2: @@ -8648,6 +11376,17 @@ def read(self, iprot): self.tbl_name = iprot.readString(); else: iprot.skip(ftype) + elif fid == 3: + if ftype == TType.STRING: + self.part_name = iprot.readString(); + else: + iprot.skip(ftype) + elif fid == 4: + if ftype == TType.STRUCT: + self.environment_context = EnvironmentContext() + self.environment_context.read(iprot) + else: + iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() @@ -8657,15 +11396,23 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('get_table_args') - if self.dbname is not None: - oprot.writeFieldBegin('dbname', TType.STRING, 1) - oprot.writeString(self.dbname) + oprot.writeStructBegin('append_partition_by_name_with_environment_context_args') + if self.db_name is not None: + oprot.writeFieldBegin('db_name', TType.STRING, 1) + oprot.writeString(self.db_name) oprot.writeFieldEnd() if self.tbl_name is not None: oprot.writeFieldBegin('tbl_name', TType.STRING, 2) oprot.writeString(self.tbl_name) oprot.writeFieldEnd() + if self.part_name is not None: + oprot.writeFieldBegin('part_name', TType.STRING, 3) + oprot.writeString(self.part_name) + oprot.writeFieldEnd() + if self.environment_context is not None: + oprot.writeFieldBegin('environment_context', TType.STRUCT, 4) + self.environment_context.write(oprot) + oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -8684,24 +11431,27 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class get_table_result: +class append_partition_by_name_with_environment_context_result: """ Attributes: - success - o1 - o2 + - o3 """ thrift_spec = ( - (0, TType.STRUCT, 'success', (Table, Table.thrift_spec), None, ), # 0 - (1, TType.STRUCT, 'o1', (MetaException, MetaException.thrift_spec), None, ), # 1 - (2, TType.STRUCT, 'o2', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 2 + (0, TType.STRUCT, 'success', (Partition, Partition.thrift_spec), None, ), # 0 + (1, TType.STRUCT, 'o1', (InvalidObjectException, InvalidObjectException.thrift_spec), None, ), # 1 + (2, TType.STRUCT, 'o2', (AlreadyExistsException, AlreadyExistsException.thrift_spec), None, ), # 2 + (3, TType.STRUCT, 'o3', (MetaException, MetaException.thrift_spec), None, ), # 3 ) - def __init__(self, success=None, o1=None, o2=None,): + def __init__(self, success=None, o1=None, o2=None, o3=None,): self.success = success self.o1 = o1 self.o2 = o2 + self.o3 = o3 def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -8714,22 +11464,28 @@ def read(self, iprot): break if fid == 0: if ftype == TType.STRUCT: - self.success = Table() + self.success = Partition() self.success.read(iprot) else: iprot.skip(ftype) elif fid == 1: if ftype == TType.STRUCT: - self.o1 = MetaException() + self.o1 = InvalidObjectException() self.o1.read(iprot) else: iprot.skip(ftype) elif fid == 2: if ftype == TType.STRUCT: - self.o2 = NoSuchObjectException() + self.o2 = AlreadyExistsException() self.o2.read(iprot) else: iprot.skip(ftype) + elif fid == 3: + if ftype == TType.STRUCT: + self.o3 = MetaException() + self.o3.read(iprot) + else: + iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() @@ -8739,7 +11495,7 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('get_table_result') + oprot.writeStructBegin('append_partition_by_name_with_environment_context_result') if self.success is not None: oprot.writeFieldBegin('success', TType.STRUCT, 0) self.success.write(oprot) @@ -8752,6 +11508,10 @@ def write(self, oprot): oprot.writeFieldBegin('o2', TType.STRUCT, 2) self.o2.write(oprot) oprot.writeFieldEnd() + if self.o3 is not None: + oprot.writeFieldBegin('o3', TType.STRUCT, 3) + self.o3.write(oprot) + oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -8770,22 +11530,28 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class get_table_objects_by_name_args: +class drop_partition_args: """ Attributes: - - dbname - - tbl_names + - db_name + - tbl_name + - part_vals + - deleteData """ thrift_spec = ( None, # 0 - (1, TType.STRING, 'dbname', None, None, ), # 1 - (2, TType.LIST, 'tbl_names', (TType.STRING,None), None, ), # 2 + (1, TType.STRING, 'db_name', None, None, ), # 1 + (2, TType.STRING, 'tbl_name', None, None, ), # 2 + (3, TType.LIST, 'part_vals', (TType.STRING,None), None, ), # 3 + (4, TType.BOOL, 'deleteData', None, None, ), # 4 ) - def __init__(self, dbname=None, tbl_names=None,): - self.dbname = dbname - self.tbl_names = tbl_names + def __init__(self, db_name=None, tbl_name=None, part_vals=None, deleteData=None,): + self.db_name = db_name + self.tbl_name = tbl_name + self.part_vals = part_vals + self.deleteData = deleteData def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -8798,19 +11564,29 @@ def read(self, iprot): break if fid == 1: if ftype == TType.STRING: - self.dbname = iprot.readString(); + self.db_name = iprot.readString(); else: iprot.skip(ftype) elif fid == 2: + if ftype == TType.STRING: + self.tbl_name = iprot.readString(); + else: + iprot.skip(ftype) + elif fid == 3: if ftype == TType.LIST: - self.tbl_names = [] - (_etype374, _size371) = iprot.readListBegin() - for _i375 in xrange(_size371): - _elem376 = iprot.readString(); - self.tbl_names.append(_elem376) + self.part_vals = [] + (_etype458, _size455) = iprot.readListBegin() + for _i459 in xrange(_size455): + _elem460 = iprot.readString(); + self.part_vals.append(_elem460) iprot.readListEnd() else: iprot.skip(ftype) + elif fid == 4: + if ftype == TType.BOOL: + self.deleteData = iprot.readBool(); + else: + iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() @@ -8820,18 +11596,26 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('get_table_objects_by_name_args') - if self.dbname is not None: - oprot.writeFieldBegin('dbname', TType.STRING, 1) - oprot.writeString(self.dbname) + oprot.writeStructBegin('drop_partition_args') + if self.db_name is not None: + oprot.writeFieldBegin('db_name', TType.STRING, 1) + oprot.writeString(self.db_name) oprot.writeFieldEnd() - if self.tbl_names is not None: - oprot.writeFieldBegin('tbl_names', TType.LIST, 2) - oprot.writeListBegin(TType.STRING, len(self.tbl_names)) - for iter377 in self.tbl_names: - oprot.writeString(iter377) + if self.tbl_name is not None: + oprot.writeFieldBegin('tbl_name', TType.STRING, 2) + oprot.writeString(self.tbl_name) + oprot.writeFieldEnd() + if self.part_vals is not None: + oprot.writeFieldBegin('part_vals', TType.LIST, 3) + oprot.writeListBegin(TType.STRING, len(self.part_vals)) + for iter461 in self.part_vals: + oprot.writeString(iter461) oprot.writeListEnd() oprot.writeFieldEnd() + if self.deleteData is not None: + oprot.writeFieldBegin('deleteData', TType.BOOL, 4) + oprot.writeBool(self.deleteData) + oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -8850,27 +11634,24 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class get_table_objects_by_name_result: +class drop_partition_result: """ Attributes: - success - o1 - o2 - - o3 """ thrift_spec = ( - (0, TType.LIST, 'success', (TType.STRUCT,(Table, Table.thrift_spec)), None, ), # 0 - (1, TType.STRUCT, 'o1', (MetaException, MetaException.thrift_spec), None, ), # 1 - (2, TType.STRUCT, 'o2', (InvalidOperationException, InvalidOperationException.thrift_spec), None, ), # 2 - (3, TType.STRUCT, 'o3', (UnknownDBException, UnknownDBException.thrift_spec), None, ), # 3 + (0, TType.BOOL, 'success', None, None, ), # 0 + (1, TType.STRUCT, 'o1', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 1 + (2, TType.STRUCT, 'o2', (MetaException, MetaException.thrift_spec), None, ), # 2 ) - def __init__(self, success=None, o1=None, o2=None, o3=None,): + def __init__(self, success=None, o1=None, o2=None,): self.success = success self.o1 = o1 self.o2 = o2 - self.o3 = o3 def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -8882,34 +11663,22 @@ def read(self, iprot): if ftype == TType.STOP: break if fid == 0: - if ftype == TType.LIST: - self.success = [] - (_etype381, _size378) = iprot.readListBegin() - for _i382 in xrange(_size378): - _elem383 = Table() - _elem383.read(iprot) - self.success.append(_elem383) - iprot.readListEnd() + if ftype == TType.BOOL: + self.success = iprot.readBool(); else: iprot.skip(ftype) elif fid == 1: if ftype == TType.STRUCT: - self.o1 = MetaException() + self.o1 = NoSuchObjectException() self.o1.read(iprot) else: iprot.skip(ftype) elif fid == 2: if ftype == TType.STRUCT: - self.o2 = InvalidOperationException() + self.o2 = MetaException() self.o2.read(iprot) else: iprot.skip(ftype) - elif fid == 3: - if ftype == TType.STRUCT: - self.o3 = UnknownDBException() - self.o3.read(iprot) - else: - iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() @@ -8919,13 +11688,10 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('get_table_objects_by_name_result') + oprot.writeStructBegin('drop_partition_result') if self.success is not None: - oprot.writeFieldBegin('success', TType.LIST, 0) - oprot.writeListBegin(TType.STRUCT, len(self.success)) - for iter384 in self.success: - iter384.write(oprot) - oprot.writeListEnd() + oprot.writeFieldBegin('success', TType.BOOL, 0) + oprot.writeBool(self.success) oprot.writeFieldEnd() if self.o1 is not None: oprot.writeFieldBegin('o1', TType.STRUCT, 1) @@ -8935,10 +11701,6 @@ def write(self, oprot): oprot.writeFieldBegin('o2', TType.STRUCT, 2) self.o2.write(oprot) oprot.writeFieldEnd() - if self.o3 is not None: - oprot.writeFieldBegin('o3', TType.STRUCT, 3) - self.o3.write(oprot) - oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -8957,25 +11719,31 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class get_table_names_by_filter_args: +class drop_partition_with_environment_context_args: """ Attributes: - - dbname - - filter - - max_tables + - db_name + - tbl_name + - part_vals + - deleteData + - environment_context """ thrift_spec = ( None, # 0 - (1, TType.STRING, 'dbname', None, None, ), # 1 - (2, TType.STRING, 'filter', None, None, ), # 2 - (3, TType.I16, 'max_tables', None, -1, ), # 3 + (1, TType.STRING, 'db_name', None, None, ), # 1 + (2, TType.STRING, 'tbl_name', None, None, ), # 2 + (3, TType.LIST, 'part_vals', (TType.STRING,None), None, ), # 3 + (4, TType.BOOL, 'deleteData', None, None, ), # 4 + (5, TType.STRUCT, 'environment_context', (EnvironmentContext, EnvironmentContext.thrift_spec), None, ), # 5 ) - def __init__(self, dbname=None, filter=None, max_tables=thrift_spec[3][4],): - self.dbname = dbname - self.filter = filter - self.max_tables = max_tables + def __init__(self, db_name=None, tbl_name=None, part_vals=None, deleteData=None, environment_context=None,): + self.db_name = db_name + self.tbl_name = tbl_name + self.part_vals = part_vals + self.deleteData = deleteData + self.environment_context = environment_context def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -8988,17 +11756,33 @@ def read(self, iprot): break if fid == 1: if ftype == TType.STRING: - self.dbname = iprot.readString(); + self.db_name = iprot.readString(); else: iprot.skip(ftype) elif fid == 2: if ftype == TType.STRING: - self.filter = iprot.readString(); + self.tbl_name = iprot.readString(); else: iprot.skip(ftype) elif fid == 3: - if ftype == TType.I16: - self.max_tables = iprot.readI16(); + if ftype == TType.LIST: + self.part_vals = [] + (_etype465, _size462) = iprot.readListBegin() + for _i466 in xrange(_size462): + _elem467 = iprot.readString(); + self.part_vals.append(_elem467) + iprot.readListEnd() + else: + iprot.skip(ftype) + elif fid == 4: + if ftype == TType.BOOL: + self.deleteData = iprot.readBool(); + else: + iprot.skip(ftype) + elif fid == 5: + if ftype == TType.STRUCT: + self.environment_context = EnvironmentContext() + self.environment_context.read(iprot) else: iprot.skip(ftype) else: @@ -9010,18 +11794,29 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('get_table_names_by_filter_args') - if self.dbname is not None: - oprot.writeFieldBegin('dbname', TType.STRING, 1) - oprot.writeString(self.dbname) + oprot.writeStructBegin('drop_partition_with_environment_context_args') + if self.db_name is not None: + oprot.writeFieldBegin('db_name', TType.STRING, 1) + oprot.writeString(self.db_name) oprot.writeFieldEnd() - if self.filter is not None: - oprot.writeFieldBegin('filter', TType.STRING, 2) - oprot.writeString(self.filter) + if self.tbl_name is not None: + oprot.writeFieldBegin('tbl_name', TType.STRING, 2) + oprot.writeString(self.tbl_name) oprot.writeFieldEnd() - if self.max_tables is not None: - oprot.writeFieldBegin('max_tables', TType.I16, 3) - oprot.writeI16(self.max_tables) + if self.part_vals is not None: + oprot.writeFieldBegin('part_vals', TType.LIST, 3) + oprot.writeListBegin(TType.STRING, len(self.part_vals)) + for iter468 in self.part_vals: + oprot.writeString(iter468) + oprot.writeListEnd() + oprot.writeFieldEnd() + if self.deleteData is not None: + oprot.writeFieldBegin('deleteData', TType.BOOL, 4) + oprot.writeBool(self.deleteData) + oprot.writeFieldEnd() + if self.environment_context is not None: + oprot.writeFieldBegin('environment_context', TType.STRUCT, 5) + self.environment_context.write(oprot) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -9041,27 +11836,24 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class get_table_names_by_filter_result: +class drop_partition_with_environment_context_result: """ Attributes: - success - o1 - o2 - - o3 """ thrift_spec = ( - (0, TType.LIST, 'success', (TType.STRING,None), None, ), # 0 - (1, TType.STRUCT, 'o1', (MetaException, MetaException.thrift_spec), None, ), # 1 - (2, TType.STRUCT, 'o2', (InvalidOperationException, InvalidOperationException.thrift_spec), None, ), # 2 - (3, TType.STRUCT, 'o3', (UnknownDBException, UnknownDBException.thrift_spec), None, ), # 3 + (0, TType.BOOL, 'success', None, None, ), # 0 + (1, TType.STRUCT, 'o1', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 1 + (2, TType.STRUCT, 'o2', (MetaException, MetaException.thrift_spec), None, ), # 2 ) - def __init__(self, success=None, o1=None, o2=None, o3=None,): + def __init__(self, success=None, o1=None, o2=None,): self.success = success self.o1 = o1 self.o2 = o2 - self.o3 = o3 def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -9073,33 +11865,22 @@ def read(self, iprot): if ftype == TType.STOP: break if fid == 0: - if ftype == TType.LIST: - self.success = [] - (_etype388, _size385) = iprot.readListBegin() - for _i389 in xrange(_size385): - _elem390 = iprot.readString(); - self.success.append(_elem390) - iprot.readListEnd() + if ftype == TType.BOOL: + self.success = iprot.readBool(); else: iprot.skip(ftype) elif fid == 1: if ftype == TType.STRUCT: - self.o1 = MetaException() + self.o1 = NoSuchObjectException() self.o1.read(iprot) else: iprot.skip(ftype) elif fid == 2: if ftype == TType.STRUCT: - self.o2 = InvalidOperationException() + self.o2 = MetaException() self.o2.read(iprot) else: iprot.skip(ftype) - elif fid == 3: - if ftype == TType.STRUCT: - self.o3 = UnknownDBException() - self.o3.read(iprot) - else: - iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() @@ -9109,13 +11890,10 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('get_table_names_by_filter_result') + oprot.writeStructBegin('drop_partition_with_environment_context_result') if self.success is not None: - oprot.writeFieldBegin('success', TType.LIST, 0) - oprot.writeListBegin(TType.STRING, len(self.success)) - for iter391 in self.success: - oprot.writeString(iter391) - oprot.writeListEnd() + oprot.writeFieldBegin('success', TType.BOOL, 0) + oprot.writeBool(self.success) oprot.writeFieldEnd() if self.o1 is not None: oprot.writeFieldBegin('o1', TType.STRUCT, 1) @@ -9125,10 +11903,6 @@ def write(self, oprot): oprot.writeFieldBegin('o2', TType.STRUCT, 2) self.o2.write(oprot) oprot.writeFieldEnd() - if self.o3 is not None: - oprot.writeFieldBegin('o3', TType.STRUCT, 3) - self.o3.write(oprot) - oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -9147,25 +11921,28 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class alter_table_args: +class drop_partition_by_name_args: """ Attributes: - - dbname + - db_name - tbl_name - - new_tbl + - part_name + - deleteData """ thrift_spec = ( None, # 0 - (1, TType.STRING, 'dbname', None, None, ), # 1 + (1, TType.STRING, 'db_name', None, None, ), # 1 (2, TType.STRING, 'tbl_name', None, None, ), # 2 - (3, TType.STRUCT, 'new_tbl', (Table, Table.thrift_spec), None, ), # 3 + (3, TType.STRING, 'part_name', None, None, ), # 3 + (4, TType.BOOL, 'deleteData', None, None, ), # 4 ) - def __init__(self, dbname=None, tbl_name=None, new_tbl=None,): - self.dbname = dbname + def __init__(self, db_name=None, tbl_name=None, part_name=None, deleteData=None,): + self.db_name = db_name self.tbl_name = tbl_name - self.new_tbl = new_tbl + self.part_name = part_name + self.deleteData = deleteData def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -9178,7 +11955,7 @@ def read(self, iprot): break if fid == 1: if ftype == TType.STRING: - self.dbname = iprot.readString(); + self.db_name = iprot.readString(); else: iprot.skip(ftype) elif fid == 2: @@ -9187,9 +11964,13 @@ def read(self, iprot): else: iprot.skip(ftype) elif fid == 3: - if ftype == TType.STRUCT: - self.new_tbl = Table() - self.new_tbl.read(iprot) + if ftype == TType.STRING: + self.part_name = iprot.readString(); + else: + iprot.skip(ftype) + elif fid == 4: + if ftype == TType.BOOL: + self.deleteData = iprot.readBool(); else: iprot.skip(ftype) else: @@ -9201,18 +11982,22 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('alter_table_args') - if self.dbname is not None: - oprot.writeFieldBegin('dbname', TType.STRING, 1) - oprot.writeString(self.dbname) + oprot.writeStructBegin('drop_partition_by_name_args') + if self.db_name is not None: + oprot.writeFieldBegin('db_name', TType.STRING, 1) + oprot.writeString(self.db_name) oprot.writeFieldEnd() if self.tbl_name is not None: oprot.writeFieldBegin('tbl_name', TType.STRING, 2) oprot.writeString(self.tbl_name) oprot.writeFieldEnd() - if self.new_tbl is not None: - oprot.writeFieldBegin('new_tbl', TType.STRUCT, 3) - self.new_tbl.write(oprot) + if self.part_name is not None: + oprot.writeFieldBegin('part_name', TType.STRING, 3) + oprot.writeString(self.part_name) + oprot.writeFieldEnd() + if self.deleteData is not None: + oprot.writeFieldBegin('deleteData', TType.BOOL, 4) + oprot.writeBool(self.deleteData) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -9232,20 +12017,22 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class alter_table_result: +class drop_partition_by_name_result: """ Attributes: + - success - o1 - o2 """ thrift_spec = ( - None, # 0 - (1, TType.STRUCT, 'o1', (InvalidOperationException, InvalidOperationException.thrift_spec), None, ), # 1 + (0, TType.BOOL, 'success', None, None, ), # 0 + (1, TType.STRUCT, 'o1', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 1 (2, TType.STRUCT, 'o2', (MetaException, MetaException.thrift_spec), None, ), # 2 ) - def __init__(self, o1=None, o2=None,): + def __init__(self, success=None, o1=None, o2=None,): + self.success = success self.o1 = o1 self.o2 = o2 @@ -9258,9 +12045,14 @@ def read(self, iprot): (fname, ftype, fid) = iprot.readFieldBegin() if ftype == TType.STOP: break - if fid == 1: + if fid == 0: + if ftype == TType.BOOL: + self.success = iprot.readBool(); + else: + iprot.skip(ftype) + elif fid == 1: if ftype == TType.STRUCT: - self.o1 = InvalidOperationException() + self.o1 = NoSuchObjectException() self.o1.read(iprot) else: iprot.skip(ftype) @@ -9279,7 +12071,11 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('alter_table_result') + oprot.writeStructBegin('drop_partition_by_name_result') + if self.success is not None: + oprot.writeFieldBegin('success', TType.BOOL, 0) + oprot.writeBool(self.success) + oprot.writeFieldEnd() if self.o1 is not None: oprot.writeFieldBegin('o1', TType.STRUCT, 1) self.o1.write(oprot) @@ -9306,27 +12102,30 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class alter_table_with_environment_context_args: +class drop_partition_by_name_with_environment_context_args: """ Attributes: - - dbname + - db_name - tbl_name - - new_tbl + - part_name + - deleteData - environment_context """ thrift_spec = ( None, # 0 - (1, TType.STRING, 'dbname', None, None, ), # 1 + (1, TType.STRING, 'db_name', None, None, ), # 1 (2, TType.STRING, 'tbl_name', None, None, ), # 2 - (3, TType.STRUCT, 'new_tbl', (Table, Table.thrift_spec), None, ), # 3 - (4, TType.STRUCT, 'environment_context', (EnvironmentContext, EnvironmentContext.thrift_spec), None, ), # 4 + (3, TType.STRING, 'part_name', None, None, ), # 3 + (4, TType.BOOL, 'deleteData', None, None, ), # 4 + (5, TType.STRUCT, 'environment_context', (EnvironmentContext, EnvironmentContext.thrift_spec), None, ), # 5 ) - def __init__(self, dbname=None, tbl_name=None, new_tbl=None, environment_context=None,): - self.dbname = dbname + def __init__(self, db_name=None, tbl_name=None, part_name=None, deleteData=None, environment_context=None,): + self.db_name = db_name self.tbl_name = tbl_name - self.new_tbl = new_tbl + self.part_name = part_name + self.deleteData = deleteData self.environment_context = environment_context def read(self, iprot): @@ -9340,7 +12139,7 @@ def read(self, iprot): break if fid == 1: if ftype == TType.STRING: - self.dbname = iprot.readString(); + self.db_name = iprot.readString(); else: iprot.skip(ftype) elif fid == 2: @@ -9349,12 +12148,16 @@ def read(self, iprot): else: iprot.skip(ftype) elif fid == 3: - if ftype == TType.STRUCT: - self.new_tbl = Table() - self.new_tbl.read(iprot) + if ftype == TType.STRING: + self.part_name = iprot.readString(); else: iprot.skip(ftype) elif fid == 4: + if ftype == TType.BOOL: + self.deleteData = iprot.readBool(); + else: + iprot.skip(ftype) + elif fid == 5: if ftype == TType.STRUCT: self.environment_context = EnvironmentContext() self.environment_context.read(iprot) @@ -9369,21 +12172,25 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('alter_table_with_environment_context_args') - if self.dbname is not None: - oprot.writeFieldBegin('dbname', TType.STRING, 1) - oprot.writeString(self.dbname) + oprot.writeStructBegin('drop_partition_by_name_with_environment_context_args') + if self.db_name is not None: + oprot.writeFieldBegin('db_name', TType.STRING, 1) + oprot.writeString(self.db_name) oprot.writeFieldEnd() if self.tbl_name is not None: oprot.writeFieldBegin('tbl_name', TType.STRING, 2) oprot.writeString(self.tbl_name) oprot.writeFieldEnd() - if self.new_tbl is not None: - oprot.writeFieldBegin('new_tbl', TType.STRUCT, 3) - self.new_tbl.write(oprot) + if self.part_name is not None: + oprot.writeFieldBegin('part_name', TType.STRING, 3) + oprot.writeString(self.part_name) + oprot.writeFieldEnd() + if self.deleteData is not None: + oprot.writeFieldBegin('deleteData', TType.BOOL, 4) + oprot.writeBool(self.deleteData) oprot.writeFieldEnd() if self.environment_context is not None: - oprot.writeFieldBegin('environment_context', TType.STRUCT, 4) + oprot.writeFieldBegin('environment_context', TType.STRUCT, 5) self.environment_context.write(oprot) oprot.writeFieldEnd() oprot.writeFieldStop() @@ -9404,20 +12211,22 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class alter_table_with_environment_context_result: +class drop_partition_by_name_with_environment_context_result: """ Attributes: + - success - o1 - o2 """ thrift_spec = ( - None, # 0 - (1, TType.STRUCT, 'o1', (InvalidOperationException, InvalidOperationException.thrift_spec), None, ), # 1 + (0, TType.BOOL, 'success', None, None, ), # 0 + (1, TType.STRUCT, 'o1', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 1 (2, TType.STRUCT, 'o2', (MetaException, MetaException.thrift_spec), None, ), # 2 ) - def __init__(self, o1=None, o2=None,): + def __init__(self, success=None, o1=None, o2=None,): + self.success = success self.o1 = o1 self.o2 = o2 @@ -9430,9 +12239,14 @@ def read(self, iprot): (fname, ftype, fid) = iprot.readFieldBegin() if ftype == TType.STOP: break - if fid == 1: + if fid == 0: + if ftype == TType.BOOL: + self.success = iprot.readBool(); + else: + iprot.skip(ftype) + elif fid == 1: if ftype == TType.STRUCT: - self.o1 = InvalidOperationException() + self.o1 = NoSuchObjectException() self.o1.read(iprot) else: iprot.skip(ftype) @@ -9451,7 +12265,11 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('alter_table_with_environment_context_result') + oprot.writeStructBegin('drop_partition_by_name_with_environment_context_result') + if self.success is not None: + oprot.writeFieldBegin('success', TType.BOOL, 0) + oprot.writeBool(self.success) + oprot.writeFieldEnd() if self.o1 is not None: oprot.writeFieldBegin('o1', TType.STRUCT, 1) self.o1.write(oprot) @@ -9478,19 +12296,19 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class add_partition_args: +class drop_partitions_req_args: """ Attributes: - - new_part + - req """ thrift_spec = ( None, # 0 - (1, TType.STRUCT, 'new_part', (Partition, Partition.thrift_spec), None, ), # 1 + (1, TType.STRUCT, 'req', (DropPartitionsRequest, DropPartitionsRequest.thrift_spec), None, ), # 1 ) - def __init__(self, new_part=None,): - self.new_part = new_part + def __init__(self, req=None,): + self.req = req def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -9503,8 +12321,8 @@ def read(self, iprot): break if fid == 1: if ftype == TType.STRUCT: - self.new_part = Partition() - self.new_part.read(iprot) + self.req = DropPartitionsRequest() + self.req.read(iprot) else: iprot.skip(ftype) else: @@ -9516,10 +12334,10 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('add_partition_args') - if self.new_part is not None: - oprot.writeFieldBegin('new_part', TType.STRUCT, 1) - self.new_part.write(oprot) + oprot.writeStructBegin('drop_partitions_req_args') + if self.req is not None: + oprot.writeFieldBegin('req', TType.STRUCT, 1) + self.req.write(oprot) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -9539,27 +12357,24 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class add_partition_result: +class drop_partitions_req_result: """ Attributes: - success - o1 - o2 - - o3 - """ - - thrift_spec = ( - (0, TType.STRUCT, 'success', (Partition, Partition.thrift_spec), None, ), # 0 - (1, TType.STRUCT, 'o1', (InvalidObjectException, InvalidObjectException.thrift_spec), None, ), # 1 - (2, TType.STRUCT, 'o2', (AlreadyExistsException, AlreadyExistsException.thrift_spec), None, ), # 2 - (3, TType.STRUCT, 'o3', (MetaException, MetaException.thrift_spec), None, ), # 3 + """ + + thrift_spec = ( + (0, TType.STRUCT, 'success', (DropPartitionsResult, DropPartitionsResult.thrift_spec), None, ), # 0 + (1, TType.STRUCT, 'o1', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 1 + (2, TType.STRUCT, 'o2', (MetaException, MetaException.thrift_spec), None, ), # 2 ) - def __init__(self, success=None, o1=None, o2=None, o3=None,): + def __init__(self, success=None, o1=None, o2=None,): self.success = success self.o1 = o1 self.o2 = o2 - self.o3 = o3 def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -9572,28 +12387,22 @@ def read(self, iprot): break if fid == 0: if ftype == TType.STRUCT: - self.success = Partition() + self.success = DropPartitionsResult() self.success.read(iprot) else: iprot.skip(ftype) elif fid == 1: if ftype == TType.STRUCT: - self.o1 = InvalidObjectException() + self.o1 = NoSuchObjectException() self.o1.read(iprot) else: iprot.skip(ftype) elif fid == 2: if ftype == TType.STRUCT: - self.o2 = AlreadyExistsException() + self.o2 = MetaException() self.o2.read(iprot) else: iprot.skip(ftype) - elif fid == 3: - if ftype == TType.STRUCT: - self.o3 = MetaException() - self.o3.read(iprot) - else: - iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() @@ -9603,7 +12412,7 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('add_partition_result') + oprot.writeStructBegin('drop_partitions_req_result') if self.success is not None: oprot.writeFieldBegin('success', TType.STRUCT, 0) self.success.write(oprot) @@ -9616,10 +12425,6 @@ def write(self, oprot): oprot.writeFieldBegin('o2', TType.STRUCT, 2) self.o2.write(oprot) oprot.writeFieldEnd() - if self.o3 is not None: - oprot.writeFieldBegin('o3', TType.STRUCT, 3) - self.o3.write(oprot) - oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -9638,22 +12443,25 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class add_partition_with_environment_context_args: +class get_partition_args: """ Attributes: - - new_part - - environment_context + - db_name + - tbl_name + - part_vals """ thrift_spec = ( None, # 0 - (1, TType.STRUCT, 'new_part', (Partition, Partition.thrift_spec), None, ), # 1 - (2, TType.STRUCT, 'environment_context', (EnvironmentContext, EnvironmentContext.thrift_spec), None, ), # 2 + (1, TType.STRING, 'db_name', None, None, ), # 1 + (2, TType.STRING, 'tbl_name', None, None, ), # 2 + (3, TType.LIST, 'part_vals', (TType.STRING,None), None, ), # 3 ) - def __init__(self, new_part=None, environment_context=None,): - self.new_part = new_part - self.environment_context = environment_context + def __init__(self, db_name=None, tbl_name=None, part_vals=None,): + self.db_name = db_name + self.tbl_name = tbl_name + self.part_vals = part_vals def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -9665,15 +12473,23 @@ def read(self, iprot): if ftype == TType.STOP: break if fid == 1: - if ftype == TType.STRUCT: - self.new_part = Partition() - self.new_part.read(iprot) + if ftype == TType.STRING: + self.db_name = iprot.readString(); else: iprot.skip(ftype) elif fid == 2: - if ftype == TType.STRUCT: - self.environment_context = EnvironmentContext() - self.environment_context.read(iprot) + if ftype == TType.STRING: + self.tbl_name = iprot.readString(); + else: + iprot.skip(ftype) + elif fid == 3: + if ftype == TType.LIST: + self.part_vals = [] + (_etype472, _size469) = iprot.readListBegin() + for _i473 in xrange(_size469): + _elem474 = iprot.readString(); + self.part_vals.append(_elem474) + iprot.readListEnd() else: iprot.skip(ftype) else: @@ -9685,14 +12501,21 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('add_partition_with_environment_context_args') - if self.new_part is not None: - oprot.writeFieldBegin('new_part', TType.STRUCT, 1) - self.new_part.write(oprot) + oprot.writeStructBegin('get_partition_args') + if self.db_name is not None: + oprot.writeFieldBegin('db_name', TType.STRING, 1) + oprot.writeString(self.db_name) oprot.writeFieldEnd() - if self.environment_context is not None: - oprot.writeFieldBegin('environment_context', TType.STRUCT, 2) - self.environment_context.write(oprot) + if self.tbl_name is not None: + oprot.writeFieldBegin('tbl_name', TType.STRING, 2) + oprot.writeString(self.tbl_name) + oprot.writeFieldEnd() + if self.part_vals is not None: + oprot.writeFieldBegin('part_vals', TType.LIST, 3) + oprot.writeListBegin(TType.STRING, len(self.part_vals)) + for iter475 in self.part_vals: + oprot.writeString(iter475) + oprot.writeListEnd() oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -9712,27 +12535,24 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class add_partition_with_environment_context_result: +class get_partition_result: """ Attributes: - success - o1 - o2 - - o3 """ thrift_spec = ( (0, TType.STRUCT, 'success', (Partition, Partition.thrift_spec), None, ), # 0 - (1, TType.STRUCT, 'o1', (InvalidObjectException, InvalidObjectException.thrift_spec), None, ), # 1 - (2, TType.STRUCT, 'o2', (AlreadyExistsException, AlreadyExistsException.thrift_spec), None, ), # 2 - (3, TType.STRUCT, 'o3', (MetaException, MetaException.thrift_spec), None, ), # 3 + (1, TType.STRUCT, 'o1', (MetaException, MetaException.thrift_spec), None, ), # 1 + (2, TType.STRUCT, 'o2', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 2 ) - def __init__(self, success=None, o1=None, o2=None, o3=None,): + def __init__(self, success=None, o1=None, o2=None,): self.success = success self.o1 = o1 self.o2 = o2 - self.o3 = o3 def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -9751,22 +12571,16 @@ def read(self, iprot): iprot.skip(ftype) elif fid == 1: if ftype == TType.STRUCT: - self.o1 = InvalidObjectException() + self.o1 = MetaException() self.o1.read(iprot) else: iprot.skip(ftype) elif fid == 2: if ftype == TType.STRUCT: - self.o2 = AlreadyExistsException() + self.o2 = NoSuchObjectException() self.o2.read(iprot) else: iprot.skip(ftype) - elif fid == 3: - if ftype == TType.STRUCT: - self.o3 = MetaException() - self.o3.read(iprot) - else: - iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() @@ -9776,7 +12590,7 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('add_partition_with_environment_context_result') + oprot.writeStructBegin('get_partition_result') if self.success is not None: oprot.writeFieldBegin('success', TType.STRUCT, 0) self.success.write(oprot) @@ -9789,10 +12603,6 @@ def write(self, oprot): oprot.writeFieldBegin('o2', TType.STRUCT, 2) self.o2.write(oprot) oprot.writeFieldEnd() - if self.o3 is not None: - oprot.writeFieldBegin('o3', TType.STRUCT, 3) - self.o3.write(oprot) - oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -9811,19 +12621,31 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class add_partitions_args: +class exchange_partition_args: """ Attributes: - - new_parts + - partitionSpecs + - source_db + - source_table_name + - dest_db + - dest_table_name """ thrift_spec = ( None, # 0 - (1, TType.LIST, 'new_parts', (TType.STRUCT,(Partition, Partition.thrift_spec)), None, ), # 1 + (1, TType.MAP, 'partitionSpecs', (TType.STRING,None,TType.STRING,None), None, ), # 1 + (2, TType.STRING, 'source_db', None, None, ), # 2 + (3, TType.STRING, 'source_table_name', None, None, ), # 3 + (4, TType.STRING, 'dest_db', None, None, ), # 4 + (5, TType.STRING, 'dest_table_name', None, None, ), # 5 ) - def __init__(self, new_parts=None,): - self.new_parts = new_parts + def __init__(self, partitionSpecs=None, source_db=None, source_table_name=None, dest_db=None, dest_table_name=None,): + self.partitionSpecs = partitionSpecs + self.source_db = source_db + self.source_table_name = source_table_name + self.dest_db = dest_db + self.dest_table_name = dest_table_name def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -9835,14 +12657,34 @@ def read(self, iprot): if ftype == TType.STOP: break if fid == 1: - if ftype == TType.LIST: - self.new_parts = [] - (_etype395, _size392) = iprot.readListBegin() - for _i396 in xrange(_size392): - _elem397 = Partition() - _elem397.read(iprot) - self.new_parts.append(_elem397) - iprot.readListEnd() + if ftype == TType.MAP: + self.partitionSpecs = {} + (_ktype477, _vtype478, _size476 ) = iprot.readMapBegin() + for _i480 in xrange(_size476): + _key481 = iprot.readString(); + _val482 = iprot.readString(); + self.partitionSpecs[_key481] = _val482 + iprot.readMapEnd() + else: + iprot.skip(ftype) + elif fid == 2: + if ftype == TType.STRING: + self.source_db = iprot.readString(); + else: + iprot.skip(ftype) + elif fid == 3: + if ftype == TType.STRING: + self.source_table_name = iprot.readString(); + else: + iprot.skip(ftype) + elif fid == 4: + if ftype == TType.STRING: + self.dest_db = iprot.readString(); + else: + iprot.skip(ftype) + elif fid == 5: + if ftype == TType.STRING: + self.dest_table_name = iprot.readString(); else: iprot.skip(ftype) else: @@ -9854,13 +12696,30 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('add_partitions_args') - if self.new_parts is not None: - oprot.writeFieldBegin('new_parts', TType.LIST, 1) - oprot.writeListBegin(TType.STRUCT, len(self.new_parts)) - for iter398 in self.new_parts: - iter398.write(oprot) - oprot.writeListEnd() + oprot.writeStructBegin('exchange_partition_args') + if self.partitionSpecs is not None: + oprot.writeFieldBegin('partitionSpecs', TType.MAP, 1) + oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.partitionSpecs)) + for kiter483,viter484 in self.partitionSpecs.items(): + oprot.writeString(kiter483) + oprot.writeString(viter484) + oprot.writeMapEnd() + oprot.writeFieldEnd() + if self.source_db is not None: + oprot.writeFieldBegin('source_db', TType.STRING, 2) + oprot.writeString(self.source_db) + oprot.writeFieldEnd() + if self.source_table_name is not None: + oprot.writeFieldBegin('source_table_name', TType.STRING, 3) + oprot.writeString(self.source_table_name) + oprot.writeFieldEnd() + if self.dest_db is not None: + oprot.writeFieldBegin('dest_db', TType.STRING, 4) + oprot.writeString(self.dest_db) + oprot.writeFieldEnd() + if self.dest_table_name is not None: + oprot.writeFieldBegin('dest_table_name', TType.STRING, 5) + oprot.writeString(self.dest_table_name) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -9880,27 +12739,30 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class add_partitions_result: +class exchange_partition_result: """ Attributes: - success - o1 - o2 - o3 + - o4 """ thrift_spec = ( - (0, TType.I32, 'success', None, None, ), # 0 - (1, TType.STRUCT, 'o1', (InvalidObjectException, InvalidObjectException.thrift_spec), None, ), # 1 - (2, TType.STRUCT, 'o2', (AlreadyExistsException, AlreadyExistsException.thrift_spec), None, ), # 2 - (3, TType.STRUCT, 'o3', (MetaException, MetaException.thrift_spec), None, ), # 3 + (0, TType.STRUCT, 'success', (Partition, Partition.thrift_spec), None, ), # 0 + (1, TType.STRUCT, 'o1', (MetaException, MetaException.thrift_spec), None, ), # 1 + (2, TType.STRUCT, 'o2', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 2 + (3, TType.STRUCT, 'o3', (InvalidObjectException, InvalidObjectException.thrift_spec), None, ), # 3 + (4, TType.STRUCT, 'o4', (InvalidInputException, InvalidInputException.thrift_spec), None, ), # 4 ) - def __init__(self, success=None, o1=None, o2=None, o3=None,): + def __init__(self, success=None, o1=None, o2=None, o3=None, o4=None,): self.success = success self.o1 = o1 self.o2 = o2 self.o3 = o3 + self.o4 = o4 def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -9912,28 +12774,35 @@ def read(self, iprot): if ftype == TType.STOP: break if fid == 0: - if ftype == TType.I32: - self.success = iprot.readI32(); + if ftype == TType.STRUCT: + self.success = Partition() + self.success.read(iprot) else: iprot.skip(ftype) elif fid == 1: if ftype == TType.STRUCT: - self.o1 = InvalidObjectException() + self.o1 = MetaException() self.o1.read(iprot) else: iprot.skip(ftype) elif fid == 2: if ftype == TType.STRUCT: - self.o2 = AlreadyExistsException() + self.o2 = NoSuchObjectException() self.o2.read(iprot) else: iprot.skip(ftype) elif fid == 3: if ftype == TType.STRUCT: - self.o3 = MetaException() + self.o3 = InvalidObjectException() self.o3.read(iprot) else: iprot.skip(ftype) + elif fid == 4: + if ftype == TType.STRUCT: + self.o4 = InvalidInputException() + self.o4.read(iprot) + else: + iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() @@ -9943,10 +12812,10 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('add_partitions_result') + oprot.writeStructBegin('exchange_partition_result') if self.success is not None: - oprot.writeFieldBegin('success', TType.I32, 0) - oprot.writeI32(self.success) + oprot.writeFieldBegin('success', TType.STRUCT, 0) + self.success.write(oprot) oprot.writeFieldEnd() if self.o1 is not None: oprot.writeFieldBegin('o1', TType.STRUCT, 1) @@ -9960,6 +12829,10 @@ def write(self, oprot): oprot.writeFieldBegin('o3', TType.STRUCT, 3) self.o3.write(oprot) oprot.writeFieldEnd() + if self.o4 is not None: + oprot.writeFieldBegin('o4', TType.STRUCT, 4) + self.o4.write(oprot) + oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -9978,12 +12851,14 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class append_partition_args: +class get_partition_with_auth_args: """ Attributes: - db_name - tbl_name - part_vals + - user_name + - group_names """ thrift_spec = ( @@ -9991,12 +12866,16 @@ class append_partition_args: (1, TType.STRING, 'db_name', None, None, ), # 1 (2, TType.STRING, 'tbl_name', None, None, ), # 2 (3, TType.LIST, 'part_vals', (TType.STRING,None), None, ), # 3 + (4, TType.STRING, 'user_name', None, None, ), # 4 + (5, TType.LIST, 'group_names', (TType.STRING,None), None, ), # 5 ) - def __init__(self, db_name=None, tbl_name=None, part_vals=None,): + def __init__(self, db_name=None, tbl_name=None, part_vals=None, user_name=None, group_names=None,): self.db_name = db_name self.tbl_name = tbl_name self.part_vals = part_vals + self.user_name = user_name + self.group_names = group_names def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -10020,10 +12899,25 @@ def read(self, iprot): elif fid == 3: if ftype == TType.LIST: self.part_vals = [] - (_etype402, _size399) = iprot.readListBegin() - for _i403 in xrange(_size399): - _elem404 = iprot.readString(); - self.part_vals.append(_elem404) + (_etype488, _size485) = iprot.readListBegin() + for _i489 in xrange(_size485): + _elem490 = iprot.readString(); + self.part_vals.append(_elem490) + iprot.readListEnd() + else: + iprot.skip(ftype) + elif fid == 4: + if ftype == TType.STRING: + self.user_name = iprot.readString(); + else: + iprot.skip(ftype) + elif fid == 5: + if ftype == TType.LIST: + self.group_names = [] + (_etype494, _size491) = iprot.readListBegin() + for _i495 in xrange(_size491): + _elem496 = iprot.readString(); + self.group_names.append(_elem496) iprot.readListEnd() else: iprot.skip(ftype) @@ -10036,7 +12930,7 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('append_partition_args') + oprot.writeStructBegin('get_partition_with_auth_args') if self.db_name is not None: oprot.writeFieldBegin('db_name', TType.STRING, 1) oprot.writeString(self.db_name) @@ -10048,8 +12942,19 @@ def write(self, oprot): if self.part_vals is not None: oprot.writeFieldBegin('part_vals', TType.LIST, 3) oprot.writeListBegin(TType.STRING, len(self.part_vals)) - for iter405 in self.part_vals: - oprot.writeString(iter405) + for iter497 in self.part_vals: + oprot.writeString(iter497) + oprot.writeListEnd() + oprot.writeFieldEnd() + if self.user_name is not None: + oprot.writeFieldBegin('user_name', TType.STRING, 4) + oprot.writeString(self.user_name) + oprot.writeFieldEnd() + if self.group_names is not None: + oprot.writeFieldBegin('group_names', TType.LIST, 5) + oprot.writeListBegin(TType.STRING, len(self.group_names)) + for iter498 in self.group_names: + oprot.writeString(iter498) oprot.writeListEnd() oprot.writeFieldEnd() oprot.writeFieldStop() @@ -10070,27 +12975,24 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class append_partition_result: +class get_partition_with_auth_result: """ Attributes: - success - o1 - o2 - - o3 """ thrift_spec = ( (0, TType.STRUCT, 'success', (Partition, Partition.thrift_spec), None, ), # 0 - (1, TType.STRUCT, 'o1', (InvalidObjectException, InvalidObjectException.thrift_spec), None, ), # 1 - (2, TType.STRUCT, 'o2', (AlreadyExistsException, AlreadyExistsException.thrift_spec), None, ), # 2 - (3, TType.STRUCT, 'o3', (MetaException, MetaException.thrift_spec), None, ), # 3 + (1, TType.STRUCT, 'o1', (MetaException, MetaException.thrift_spec), None, ), # 1 + (2, TType.STRUCT, 'o2', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 2 ) - def __init__(self, success=None, o1=None, o2=None, o3=None,): + def __init__(self, success=None, o1=None, o2=None,): self.success = success self.o1 = o1 self.o2 = o2 - self.o3 = o3 def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -10109,22 +13011,16 @@ def read(self, iprot): iprot.skip(ftype) elif fid == 1: if ftype == TType.STRUCT: - self.o1 = InvalidObjectException() + self.o1 = MetaException() self.o1.read(iprot) else: iprot.skip(ftype) elif fid == 2: if ftype == TType.STRUCT: - self.o2 = AlreadyExistsException() + self.o2 = NoSuchObjectException() self.o2.read(iprot) else: iprot.skip(ftype) - elif fid == 3: - if ftype == TType.STRUCT: - self.o3 = MetaException() - self.o3.read(iprot) - else: - iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() @@ -10134,7 +13030,7 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('append_partition_result') + oprot.writeStructBegin('get_partition_with_auth_result') if self.success is not None: oprot.writeFieldBegin('success', TType.STRUCT, 0) self.success.write(oprot) @@ -10147,10 +13043,6 @@ def write(self, oprot): oprot.writeFieldBegin('o2', TType.STRUCT, 2) self.o2.write(oprot) oprot.writeFieldEnd() - if self.o3 is not None: - oprot.writeFieldBegin('o3', TType.STRUCT, 3) - self.o3.write(oprot) - oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -10169,19 +13061,25 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class add_partitions_req_args: +class get_partition_by_name_args: """ Attributes: - - request + - db_name + - tbl_name + - part_name """ thrift_spec = ( None, # 0 - (1, TType.STRUCT, 'request', (AddPartitionsRequest, AddPartitionsRequest.thrift_spec), None, ), # 1 + (1, TType.STRING, 'db_name', None, None, ), # 1 + (2, TType.STRING, 'tbl_name', None, None, ), # 2 + (3, TType.STRING, 'part_name', None, None, ), # 3 ) - def __init__(self, request=None,): - self.request = request + def __init__(self, db_name=None, tbl_name=None, part_name=None,): + self.db_name = db_name + self.tbl_name = tbl_name + self.part_name = part_name def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -10193,9 +13091,18 @@ def read(self, iprot): if ftype == TType.STOP: break if fid == 1: - if ftype == TType.STRUCT: - self.request = AddPartitionsRequest() - self.request.read(iprot) + if ftype == TType.STRING: + self.db_name = iprot.readString(); + else: + iprot.skip(ftype) + elif fid == 2: + if ftype == TType.STRING: + self.tbl_name = iprot.readString(); + else: + iprot.skip(ftype) + elif fid == 3: + if ftype == TType.STRING: + self.part_name = iprot.readString(); else: iprot.skip(ftype) else: @@ -10207,10 +13114,18 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('add_partitions_req_args') - if self.request is not None: - oprot.writeFieldBegin('request', TType.STRUCT, 1) - self.request.write(oprot) + oprot.writeStructBegin('get_partition_by_name_args') + if self.db_name is not None: + oprot.writeFieldBegin('db_name', TType.STRING, 1) + oprot.writeString(self.db_name) + oprot.writeFieldEnd() + if self.tbl_name is not None: + oprot.writeFieldBegin('tbl_name', TType.STRING, 2) + oprot.writeString(self.tbl_name) + oprot.writeFieldEnd() + if self.part_name is not None: + oprot.writeFieldBegin('part_name', TType.STRING, 3) + oprot.writeString(self.part_name) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -10230,27 +13145,24 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class add_partitions_req_result: +class get_partition_by_name_result: """ Attributes: - success - o1 - o2 - - o3 """ thrift_spec = ( - (0, TType.STRUCT, 'success', (AddPartitionsResult, AddPartitionsResult.thrift_spec), None, ), # 0 - (1, TType.STRUCT, 'o1', (InvalidObjectException, InvalidObjectException.thrift_spec), None, ), # 1 - (2, TType.STRUCT, 'o2', (AlreadyExistsException, AlreadyExistsException.thrift_spec), None, ), # 2 - (3, TType.STRUCT, 'o3', (MetaException, MetaException.thrift_spec), None, ), # 3 + (0, TType.STRUCT, 'success', (Partition, Partition.thrift_spec), None, ), # 0 + (1, TType.STRUCT, 'o1', (MetaException, MetaException.thrift_spec), None, ), # 1 + (2, TType.STRUCT, 'o2', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 2 ) - def __init__(self, success=None, o1=None, o2=None, o3=None,): + def __init__(self, success=None, o1=None, o2=None,): self.success = success self.o1 = o1 self.o2 = o2 - self.o3 = o3 def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -10263,28 +13175,22 @@ def read(self, iprot): break if fid == 0: if ftype == TType.STRUCT: - self.success = AddPartitionsResult() + self.success = Partition() self.success.read(iprot) else: iprot.skip(ftype) elif fid == 1: if ftype == TType.STRUCT: - self.o1 = InvalidObjectException() + self.o1 = MetaException() self.o1.read(iprot) else: iprot.skip(ftype) elif fid == 2: if ftype == TType.STRUCT: - self.o2 = AlreadyExistsException() + self.o2 = NoSuchObjectException() self.o2.read(iprot) else: iprot.skip(ftype) - elif fid == 3: - if ftype == TType.STRUCT: - self.o3 = MetaException() - self.o3.read(iprot) - else: - iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() @@ -10294,7 +13200,7 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('add_partitions_req_result') + oprot.writeStructBegin('get_partition_by_name_result') if self.success is not None: oprot.writeFieldBegin('success', TType.STRUCT, 0) self.success.write(oprot) @@ -10307,10 +13213,6 @@ def write(self, oprot): oprot.writeFieldBegin('o2', TType.STRUCT, 2) self.o2.write(oprot) oprot.writeFieldEnd() - if self.o3 is not None: - oprot.writeFieldBegin('o3', TType.STRUCT, 3) - self.o3.write(oprot) - oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -10329,28 +13231,25 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class append_partition_with_environment_context_args: +class get_partitions_args: """ Attributes: - db_name - tbl_name - - part_vals - - environment_context + - max_parts """ thrift_spec = ( None, # 0 (1, TType.STRING, 'db_name', None, None, ), # 1 (2, TType.STRING, 'tbl_name', None, None, ), # 2 - (3, TType.LIST, 'part_vals', (TType.STRING,None), None, ), # 3 - (4, TType.STRUCT, 'environment_context', (EnvironmentContext, EnvironmentContext.thrift_spec), None, ), # 4 + (3, TType.I16, 'max_parts', None, -1, ), # 3 ) - def __init__(self, db_name=None, tbl_name=None, part_vals=None, environment_context=None,): + def __init__(self, db_name=None, tbl_name=None, max_parts=thrift_spec[3][4],): self.db_name = db_name self.tbl_name = tbl_name - self.part_vals = part_vals - self.environment_context = environment_context + self.max_parts = max_parts def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -10372,19 +13271,8 @@ def read(self, iprot): else: iprot.skip(ftype) elif fid == 3: - if ftype == TType.LIST: - self.part_vals = [] - (_etype409, _size406) = iprot.readListBegin() - for _i410 in xrange(_size406): - _elem411 = iprot.readString(); - self.part_vals.append(_elem411) - iprot.readListEnd() - else: - iprot.skip(ftype) - elif fid == 4: - if ftype == TType.STRUCT: - self.environment_context = EnvironmentContext() - self.environment_context.read(iprot) + if ftype == TType.I16: + self.max_parts = iprot.readI16(); else: iprot.skip(ftype) else: @@ -10396,7 +13284,7 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('append_partition_with_environment_context_args') + oprot.writeStructBegin('get_partitions_args') if self.db_name is not None: oprot.writeFieldBegin('db_name', TType.STRING, 1) oprot.writeString(self.db_name) @@ -10405,16 +13293,9 @@ def write(self, oprot): oprot.writeFieldBegin('tbl_name', TType.STRING, 2) oprot.writeString(self.tbl_name) oprot.writeFieldEnd() - if self.part_vals is not None: - oprot.writeFieldBegin('part_vals', TType.LIST, 3) - oprot.writeListBegin(TType.STRING, len(self.part_vals)) - for iter412 in self.part_vals: - oprot.writeString(iter412) - oprot.writeListEnd() - oprot.writeFieldEnd() - if self.environment_context is not None: - oprot.writeFieldBegin('environment_context', TType.STRUCT, 4) - self.environment_context.write(oprot) + if self.max_parts is not None: + oprot.writeFieldBegin('max_parts', TType.I16, 3) + oprot.writeI16(self.max_parts) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -10434,27 +13315,24 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class append_partition_with_environment_context_result: +class get_partitions_result: """ Attributes: - success - o1 - o2 - - o3 """ thrift_spec = ( - (0, TType.STRUCT, 'success', (Partition, Partition.thrift_spec), None, ), # 0 - (1, TType.STRUCT, 'o1', (InvalidObjectException, InvalidObjectException.thrift_spec), None, ), # 1 - (2, TType.STRUCT, 'o2', (AlreadyExistsException, AlreadyExistsException.thrift_spec), None, ), # 2 - (3, TType.STRUCT, 'o3', (MetaException, MetaException.thrift_spec), None, ), # 3 + (0, TType.LIST, 'success', (TType.STRUCT,(Partition, Partition.thrift_spec)), None, ), # 0 + (1, TType.STRUCT, 'o1', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 1 + (2, TType.STRUCT, 'o2', (MetaException, MetaException.thrift_spec), None, ), # 2 ) - def __init__(self, success=None, o1=None, o2=None, o3=None,): + def __init__(self, success=None, o1=None, o2=None,): self.success = success self.o1 = o1 self.o2 = o2 - self.o3 = o3 def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -10466,29 +13344,28 @@ def read(self, iprot): if ftype == TType.STOP: break if fid == 0: - if ftype == TType.STRUCT: - self.success = Partition() - self.success.read(iprot) + if ftype == TType.LIST: + self.success = [] + (_etype502, _size499) = iprot.readListBegin() + for _i503 in xrange(_size499): + _elem504 = Partition() + _elem504.read(iprot) + self.success.append(_elem504) + iprot.readListEnd() else: iprot.skip(ftype) elif fid == 1: if ftype == TType.STRUCT: - self.o1 = InvalidObjectException() + self.o1 = NoSuchObjectException() self.o1.read(iprot) else: iprot.skip(ftype) elif fid == 2: if ftype == TType.STRUCT: - self.o2 = AlreadyExistsException() + self.o2 = MetaException() self.o2.read(iprot) else: iprot.skip(ftype) - elif fid == 3: - if ftype == TType.STRUCT: - self.o3 = MetaException() - self.o3.read(iprot) - else: - iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() @@ -10498,10 +13375,13 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('append_partition_with_environment_context_result') + oprot.writeStructBegin('get_partitions_result') if self.success is not None: - oprot.writeFieldBegin('success', TType.STRUCT, 0) - self.success.write(oprot) + oprot.writeFieldBegin('success', TType.LIST, 0) + oprot.writeListBegin(TType.STRUCT, len(self.success)) + for iter505 in self.success: + iter505.write(oprot) + oprot.writeListEnd() oprot.writeFieldEnd() if self.o1 is not None: oprot.writeFieldBegin('o1', TType.STRUCT, 1) @@ -10511,10 +13391,6 @@ def write(self, oprot): oprot.writeFieldBegin('o2', TType.STRUCT, 2) self.o2.write(oprot) oprot.writeFieldEnd() - if self.o3 is not None: - oprot.writeFieldBegin('o3', TType.STRUCT, 3) - self.o3.write(oprot) - oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -10533,25 +13409,31 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class append_partition_by_name_args: +class get_partitions_with_auth_args: """ Attributes: - db_name - tbl_name - - part_name + - max_parts + - user_name + - group_names """ thrift_spec = ( None, # 0 (1, TType.STRING, 'db_name', None, None, ), # 1 (2, TType.STRING, 'tbl_name', None, None, ), # 2 - (3, TType.STRING, 'part_name', None, None, ), # 3 + (3, TType.I16, 'max_parts', None, -1, ), # 3 + (4, TType.STRING, 'user_name', None, None, ), # 4 + (5, TType.LIST, 'group_names', (TType.STRING,None), None, ), # 5 ) - def __init__(self, db_name=None, tbl_name=None, part_name=None,): + def __init__(self, db_name=None, tbl_name=None, max_parts=thrift_spec[3][4], user_name=None, group_names=None,): self.db_name = db_name self.tbl_name = tbl_name - self.part_name = part_name + self.max_parts = max_parts + self.user_name = user_name + self.group_names = group_names def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -10573,8 +13455,23 @@ def read(self, iprot): else: iprot.skip(ftype) elif fid == 3: + if ftype == TType.I16: + self.max_parts = iprot.readI16(); + else: + iprot.skip(ftype) + elif fid == 4: if ftype == TType.STRING: - self.part_name = iprot.readString(); + self.user_name = iprot.readString(); + else: + iprot.skip(ftype) + elif fid == 5: + if ftype == TType.LIST: + self.group_names = [] + (_etype509, _size506) = iprot.readListBegin() + for _i510 in xrange(_size506): + _elem511 = iprot.readString(); + self.group_names.append(_elem511) + iprot.readListEnd() else: iprot.skip(ftype) else: @@ -10586,7 +13483,7 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('append_partition_by_name_args') + oprot.writeStructBegin('get_partitions_with_auth_args') if self.db_name is not None: oprot.writeFieldBegin('db_name', TType.STRING, 1) oprot.writeString(self.db_name) @@ -10595,9 +13492,20 @@ def write(self, oprot): oprot.writeFieldBegin('tbl_name', TType.STRING, 2) oprot.writeString(self.tbl_name) oprot.writeFieldEnd() - if self.part_name is not None: - oprot.writeFieldBegin('part_name', TType.STRING, 3) - oprot.writeString(self.part_name) + if self.max_parts is not None: + oprot.writeFieldBegin('max_parts', TType.I16, 3) + oprot.writeI16(self.max_parts) + oprot.writeFieldEnd() + if self.user_name is not None: + oprot.writeFieldBegin('user_name', TType.STRING, 4) + oprot.writeString(self.user_name) + oprot.writeFieldEnd() + if self.group_names is not None: + oprot.writeFieldBegin('group_names', TType.LIST, 5) + oprot.writeListBegin(TType.STRING, len(self.group_names)) + for iter512 in self.group_names: + oprot.writeString(iter512) + oprot.writeListEnd() oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -10617,27 +13525,24 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class append_partition_by_name_result: +class get_partitions_with_auth_result: """ Attributes: - success - o1 - o2 - - o3 """ thrift_spec = ( - (0, TType.STRUCT, 'success', (Partition, Partition.thrift_spec), None, ), # 0 - (1, TType.STRUCT, 'o1', (InvalidObjectException, InvalidObjectException.thrift_spec), None, ), # 1 - (2, TType.STRUCT, 'o2', (AlreadyExistsException, AlreadyExistsException.thrift_spec), None, ), # 2 - (3, TType.STRUCT, 'o3', (MetaException, MetaException.thrift_spec), None, ), # 3 + (0, TType.LIST, 'success', (TType.STRUCT,(Partition, Partition.thrift_spec)), None, ), # 0 + (1, TType.STRUCT, 'o1', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 1 + (2, TType.STRUCT, 'o2', (MetaException, MetaException.thrift_spec), None, ), # 2 ) - def __init__(self, success=None, o1=None, o2=None, o3=None,): + def __init__(self, success=None, o1=None, o2=None,): self.success = success self.o1 = o1 self.o2 = o2 - self.o3 = o3 def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -10649,29 +13554,28 @@ def read(self, iprot): if ftype == TType.STOP: break if fid == 0: - if ftype == TType.STRUCT: - self.success = Partition() - self.success.read(iprot) + if ftype == TType.LIST: + self.success = [] + (_etype516, _size513) = iprot.readListBegin() + for _i517 in xrange(_size513): + _elem518 = Partition() + _elem518.read(iprot) + self.success.append(_elem518) + iprot.readListEnd() else: iprot.skip(ftype) elif fid == 1: if ftype == TType.STRUCT: - self.o1 = InvalidObjectException() + self.o1 = NoSuchObjectException() self.o1.read(iprot) else: iprot.skip(ftype) elif fid == 2: if ftype == TType.STRUCT: - self.o2 = AlreadyExistsException() + self.o2 = MetaException() self.o2.read(iprot) else: iprot.skip(ftype) - elif fid == 3: - if ftype == TType.STRUCT: - self.o3 = MetaException() - self.o3.read(iprot) - else: - iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() @@ -10681,10 +13585,13 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('append_partition_by_name_result') + oprot.writeStructBegin('get_partitions_with_auth_result') if self.success is not None: - oprot.writeFieldBegin('success', TType.STRUCT, 0) - self.success.write(oprot) + oprot.writeFieldBegin('success', TType.LIST, 0) + oprot.writeListBegin(TType.STRUCT, len(self.success)) + for iter519 in self.success: + iter519.write(oprot) + oprot.writeListEnd() oprot.writeFieldEnd() if self.o1 is not None: oprot.writeFieldBegin('o1', TType.STRUCT, 1) @@ -10694,10 +13601,6 @@ def write(self, oprot): oprot.writeFieldBegin('o2', TType.STRUCT, 2) self.o2.write(oprot) oprot.writeFieldEnd() - if self.o3 is not None: - oprot.writeFieldBegin('o3', TType.STRUCT, 3) - self.o3.write(oprot) - oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -10716,28 +13619,25 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class append_partition_by_name_with_environment_context_args: +class get_partition_names_args: """ Attributes: - db_name - tbl_name - - part_name - - environment_context + - max_parts """ thrift_spec = ( None, # 0 (1, TType.STRING, 'db_name', None, None, ), # 1 (2, TType.STRING, 'tbl_name', None, None, ), # 2 - (3, TType.STRING, 'part_name', None, None, ), # 3 - (4, TType.STRUCT, 'environment_context', (EnvironmentContext, EnvironmentContext.thrift_spec), None, ), # 4 + (3, TType.I16, 'max_parts', None, -1, ), # 3 ) - def __init__(self, db_name=None, tbl_name=None, part_name=None, environment_context=None,): + def __init__(self, db_name=None, tbl_name=None, max_parts=thrift_spec[3][4],): self.db_name = db_name self.tbl_name = tbl_name - self.part_name = part_name - self.environment_context = environment_context + self.max_parts = max_parts def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -10759,14 +13659,8 @@ def read(self, iprot): else: iprot.skip(ftype) elif fid == 3: - if ftype == TType.STRING: - self.part_name = iprot.readString(); - else: - iprot.skip(ftype) - elif fid == 4: - if ftype == TType.STRUCT: - self.environment_context = EnvironmentContext() - self.environment_context.read(iprot) + if ftype == TType.I16: + self.max_parts = iprot.readI16(); else: iprot.skip(ftype) else: @@ -10778,7 +13672,7 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('append_partition_by_name_with_environment_context_args') + oprot.writeStructBegin('get_partition_names_args') if self.db_name is not None: oprot.writeFieldBegin('db_name', TType.STRING, 1) oprot.writeString(self.db_name) @@ -10787,13 +13681,9 @@ def write(self, oprot): oprot.writeFieldBegin('tbl_name', TType.STRING, 2) oprot.writeString(self.tbl_name) oprot.writeFieldEnd() - if self.part_name is not None: - oprot.writeFieldBegin('part_name', TType.STRING, 3) - oprot.writeString(self.part_name) - oprot.writeFieldEnd() - if self.environment_context is not None: - oprot.writeFieldBegin('environment_context', TType.STRUCT, 4) - self.environment_context.write(oprot) + if self.max_parts is not None: + oprot.writeFieldBegin('max_parts', TType.I16, 3) + oprot.writeI16(self.max_parts) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -10813,27 +13703,21 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class append_partition_by_name_with_environment_context_result: +class get_partition_names_result: """ Attributes: - success - - o1 - o2 - - o3 """ thrift_spec = ( - (0, TType.STRUCT, 'success', (Partition, Partition.thrift_spec), None, ), # 0 - (1, TType.STRUCT, 'o1', (InvalidObjectException, InvalidObjectException.thrift_spec), None, ), # 1 - (2, TType.STRUCT, 'o2', (AlreadyExistsException, AlreadyExistsException.thrift_spec), None, ), # 2 - (3, TType.STRUCT, 'o3', (MetaException, MetaException.thrift_spec), None, ), # 3 + (0, TType.LIST, 'success', (TType.STRING,None), None, ), # 0 + (1, TType.STRUCT, 'o2', (MetaException, MetaException.thrift_spec), None, ), # 1 ) - def __init__(self, success=None, o1=None, o2=None, o3=None,): + def __init__(self, success=None, o2=None,): self.success = success - self.o1 = o1 self.o2 = o2 - self.o3 = o3 def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -10845,29 +13729,21 @@ def read(self, iprot): if ftype == TType.STOP: break if fid == 0: - if ftype == TType.STRUCT: - self.success = Partition() - self.success.read(iprot) + if ftype == TType.LIST: + self.success = [] + (_etype523, _size520) = iprot.readListBegin() + for _i524 in xrange(_size520): + _elem525 = iprot.readString(); + self.success.append(_elem525) + iprot.readListEnd() else: iprot.skip(ftype) elif fid == 1: if ftype == TType.STRUCT: - self.o1 = InvalidObjectException() - self.o1.read(iprot) - else: - iprot.skip(ftype) - elif fid == 2: - if ftype == TType.STRUCT: - self.o2 = AlreadyExistsException() + self.o2 = MetaException() self.o2.read(iprot) else: iprot.skip(ftype) - elif fid == 3: - if ftype == TType.STRUCT: - self.o3 = MetaException() - self.o3.read(iprot) - else: - iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() @@ -10877,23 +13753,18 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('append_partition_by_name_with_environment_context_result') + oprot.writeStructBegin('get_partition_names_result') if self.success is not None: - oprot.writeFieldBegin('success', TType.STRUCT, 0) - self.success.write(oprot) - oprot.writeFieldEnd() - if self.o1 is not None: - oprot.writeFieldBegin('o1', TType.STRUCT, 1) - self.o1.write(oprot) + oprot.writeFieldBegin('success', TType.LIST, 0) + oprot.writeListBegin(TType.STRING, len(self.success)) + for iter526 in self.success: + oprot.writeString(iter526) + oprot.writeListEnd() oprot.writeFieldEnd() if self.o2 is not None: - oprot.writeFieldBegin('o2', TType.STRUCT, 2) + oprot.writeFieldBegin('o2', TType.STRUCT, 1) self.o2.write(oprot) oprot.writeFieldEnd() - if self.o3 is not None: - oprot.writeFieldBegin('o3', TType.STRUCT, 3) - self.o3.write(oprot) - oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -10912,13 +13783,13 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class drop_partition_args: +class get_partitions_ps_args: """ Attributes: - db_name - tbl_name - part_vals - - deleteData + - max_parts """ thrift_spec = ( @@ -10926,14 +13797,14 @@ class drop_partition_args: (1, TType.STRING, 'db_name', None, None, ), # 1 (2, TType.STRING, 'tbl_name', None, None, ), # 2 (3, TType.LIST, 'part_vals', (TType.STRING,None), None, ), # 3 - (4, TType.BOOL, 'deleteData', None, None, ), # 4 + (4, TType.I16, 'max_parts', None, -1, ), # 4 ) - def __init__(self, db_name=None, tbl_name=None, part_vals=None, deleteData=None,): + def __init__(self, db_name=None, tbl_name=None, part_vals=None, max_parts=thrift_spec[4][4],): self.db_name = db_name self.tbl_name = tbl_name self.part_vals = part_vals - self.deleteData = deleteData + self.max_parts = max_parts def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -10957,16 +13828,16 @@ def read(self, iprot): elif fid == 3: if ftype == TType.LIST: self.part_vals = [] - (_etype416, _size413) = iprot.readListBegin() - for _i417 in xrange(_size413): - _elem418 = iprot.readString(); - self.part_vals.append(_elem418) + (_etype530, _size527) = iprot.readListBegin() + for _i531 in xrange(_size527): + _elem532 = iprot.readString(); + self.part_vals.append(_elem532) iprot.readListEnd() else: iprot.skip(ftype) elif fid == 4: - if ftype == TType.BOOL: - self.deleteData = iprot.readBool(); + if ftype == TType.I16: + self.max_parts = iprot.readI16(); else: iprot.skip(ftype) else: @@ -10978,7 +13849,7 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('drop_partition_args') + oprot.writeStructBegin('get_partitions_ps_args') if self.db_name is not None: oprot.writeFieldBegin('db_name', TType.STRING, 1) oprot.writeString(self.db_name) @@ -10990,13 +13861,13 @@ def write(self, oprot): if self.part_vals is not None: oprot.writeFieldBegin('part_vals', TType.LIST, 3) oprot.writeListBegin(TType.STRING, len(self.part_vals)) - for iter419 in self.part_vals: - oprot.writeString(iter419) + for iter533 in self.part_vals: + oprot.writeString(iter533) oprot.writeListEnd() oprot.writeFieldEnd() - if self.deleteData is not None: - oprot.writeFieldBegin('deleteData', TType.BOOL, 4) - oprot.writeBool(self.deleteData) + if self.max_parts is not None: + oprot.writeFieldBegin('max_parts', TType.I16, 4) + oprot.writeI16(self.max_parts) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -11016,7 +13887,7 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class drop_partition_result: +class get_partitions_ps_result: """ Attributes: - success @@ -11025,9 +13896,9 @@ class drop_partition_result: """ thrift_spec = ( - (0, TType.BOOL, 'success', None, None, ), # 0 - (1, TType.STRUCT, 'o1', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 1 - (2, TType.STRUCT, 'o2', (MetaException, MetaException.thrift_spec), None, ), # 2 + (0, TType.LIST, 'success', (TType.STRUCT,(Partition, Partition.thrift_spec)), None, ), # 0 + (1, TType.STRUCT, 'o1', (MetaException, MetaException.thrift_spec), None, ), # 1 + (2, TType.STRUCT, 'o2', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 2 ) def __init__(self, success=None, o1=None, o2=None,): @@ -11045,19 +13916,25 @@ def read(self, iprot): if ftype == TType.STOP: break if fid == 0: - if ftype == TType.BOOL: - self.success = iprot.readBool(); + if ftype == TType.LIST: + self.success = [] + (_etype537, _size534) = iprot.readListBegin() + for _i538 in xrange(_size534): + _elem539 = Partition() + _elem539.read(iprot) + self.success.append(_elem539) + iprot.readListEnd() else: iprot.skip(ftype) elif fid == 1: if ftype == TType.STRUCT: - self.o1 = NoSuchObjectException() + self.o1 = MetaException() self.o1.read(iprot) else: iprot.skip(ftype) elif fid == 2: if ftype == TType.STRUCT: - self.o2 = MetaException() + self.o2 = NoSuchObjectException() self.o2.read(iprot) else: iprot.skip(ftype) @@ -11070,10 +13947,13 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('drop_partition_result') + oprot.writeStructBegin('get_partitions_ps_result') if self.success is not None: - oprot.writeFieldBegin('success', TType.BOOL, 0) - oprot.writeBool(self.success) + oprot.writeFieldBegin('success', TType.LIST, 0) + oprot.writeListBegin(TType.STRUCT, len(self.success)) + for iter540 in self.success: + iter540.write(oprot) + oprot.writeListEnd() oprot.writeFieldEnd() if self.o1 is not None: oprot.writeFieldBegin('o1', TType.STRUCT, 1) @@ -11101,14 +13981,15 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class drop_partition_with_environment_context_args: +class get_partitions_ps_with_auth_args: """ Attributes: - db_name - tbl_name - part_vals - - deleteData - - environment_context + - max_parts + - user_name + - group_names """ thrift_spec = ( @@ -11116,16 +13997,18 @@ class drop_partition_with_environment_context_args: (1, TType.STRING, 'db_name', None, None, ), # 1 (2, TType.STRING, 'tbl_name', None, None, ), # 2 (3, TType.LIST, 'part_vals', (TType.STRING,None), None, ), # 3 - (4, TType.BOOL, 'deleteData', None, None, ), # 4 - (5, TType.STRUCT, 'environment_context', (EnvironmentContext, EnvironmentContext.thrift_spec), None, ), # 5 + (4, TType.I16, 'max_parts', None, -1, ), # 4 + (5, TType.STRING, 'user_name', None, None, ), # 5 + (6, TType.LIST, 'group_names', (TType.STRING,None), None, ), # 6 ) - def __init__(self, db_name=None, tbl_name=None, part_vals=None, deleteData=None, environment_context=None,): + def __init__(self, db_name=None, tbl_name=None, part_vals=None, max_parts=thrift_spec[4][4], user_name=None, group_names=None,): self.db_name = db_name self.tbl_name = tbl_name self.part_vals = part_vals - self.deleteData = deleteData - self.environment_context = environment_context + self.max_parts = max_parts + self.user_name = user_name + self.group_names = group_names def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -11149,22 +14032,31 @@ def read(self, iprot): elif fid == 3: if ftype == TType.LIST: self.part_vals = [] - (_etype423, _size420) = iprot.readListBegin() - for _i424 in xrange(_size420): - _elem425 = iprot.readString(); - self.part_vals.append(_elem425) + (_etype544, _size541) = iprot.readListBegin() + for _i545 in xrange(_size541): + _elem546 = iprot.readString(); + self.part_vals.append(_elem546) iprot.readListEnd() else: iprot.skip(ftype) elif fid == 4: - if ftype == TType.BOOL: - self.deleteData = iprot.readBool(); + if ftype == TType.I16: + self.max_parts = iprot.readI16(); else: iprot.skip(ftype) elif fid == 5: - if ftype == TType.STRUCT: - self.environment_context = EnvironmentContext() - self.environment_context.read(iprot) + if ftype == TType.STRING: + self.user_name = iprot.readString(); + else: + iprot.skip(ftype) + elif fid == 6: + if ftype == TType.LIST: + self.group_names = [] + (_etype550, _size547) = iprot.readListBegin() + for _i551 in xrange(_size547): + _elem552 = iprot.readString(); + self.group_names.append(_elem552) + iprot.readListEnd() else: iprot.skip(ftype) else: @@ -11176,7 +14068,7 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('drop_partition_with_environment_context_args') + oprot.writeStructBegin('get_partitions_ps_with_auth_args') if self.db_name is not None: oprot.writeFieldBegin('db_name', TType.STRING, 1) oprot.writeString(self.db_name) @@ -11188,17 +14080,24 @@ def write(self, oprot): if self.part_vals is not None: oprot.writeFieldBegin('part_vals', TType.LIST, 3) oprot.writeListBegin(TType.STRING, len(self.part_vals)) - for iter426 in self.part_vals: - oprot.writeString(iter426) + for iter553 in self.part_vals: + oprot.writeString(iter553) oprot.writeListEnd() oprot.writeFieldEnd() - if self.deleteData is not None: - oprot.writeFieldBegin('deleteData', TType.BOOL, 4) - oprot.writeBool(self.deleteData) + if self.max_parts is not None: + oprot.writeFieldBegin('max_parts', TType.I16, 4) + oprot.writeI16(self.max_parts) oprot.writeFieldEnd() - if self.environment_context is not None: - oprot.writeFieldBegin('environment_context', TType.STRUCT, 5) - self.environment_context.write(oprot) + if self.user_name is not None: + oprot.writeFieldBegin('user_name', TType.STRING, 5) + oprot.writeString(self.user_name) + oprot.writeFieldEnd() + if self.group_names is not None: + oprot.writeFieldBegin('group_names', TType.LIST, 6) + oprot.writeListBegin(TType.STRING, len(self.group_names)) + for iter554 in self.group_names: + oprot.writeString(iter554) + oprot.writeListEnd() oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -11218,7 +14117,7 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class drop_partition_with_environment_context_result: +class get_partitions_ps_with_auth_result: """ Attributes: - success @@ -11227,7 +14126,7 @@ class drop_partition_with_environment_context_result: """ thrift_spec = ( - (0, TType.BOOL, 'success', None, None, ), # 0 + (0, TType.LIST, 'success', (TType.STRUCT,(Partition, Partition.thrift_spec)), None, ), # 0 (1, TType.STRUCT, 'o1', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 1 (2, TType.STRUCT, 'o2', (MetaException, MetaException.thrift_spec), None, ), # 2 ) @@ -11247,8 +14146,14 @@ def read(self, iprot): if ftype == TType.STOP: break if fid == 0: - if ftype == TType.BOOL: - self.success = iprot.readBool(); + if ftype == TType.LIST: + self.success = [] + (_etype558, _size555) = iprot.readListBegin() + for _i559 in xrange(_size555): + _elem560 = Partition() + _elem560.read(iprot) + self.success.append(_elem560) + iprot.readListEnd() else: iprot.skip(ftype) elif fid == 1: @@ -11272,10 +14177,13 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('drop_partition_with_environment_context_result') + oprot.writeStructBegin('get_partitions_ps_with_auth_result') if self.success is not None: - oprot.writeFieldBegin('success', TType.BOOL, 0) - oprot.writeBool(self.success) + oprot.writeFieldBegin('success', TType.LIST, 0) + oprot.writeListBegin(TType.STRUCT, len(self.success)) + for iter561 in self.success: + iter561.write(oprot) + oprot.writeListEnd() oprot.writeFieldEnd() if self.o1 is not None: oprot.writeFieldBegin('o1', TType.STRUCT, 1) @@ -11303,28 +14211,28 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class drop_partition_by_name_args: +class get_partition_names_ps_args: """ Attributes: - db_name - tbl_name - - part_name - - deleteData + - part_vals + - max_parts """ thrift_spec = ( None, # 0 (1, TType.STRING, 'db_name', None, None, ), # 1 (2, TType.STRING, 'tbl_name', None, None, ), # 2 - (3, TType.STRING, 'part_name', None, None, ), # 3 - (4, TType.BOOL, 'deleteData', None, None, ), # 4 + (3, TType.LIST, 'part_vals', (TType.STRING,None), None, ), # 3 + (4, TType.I16, 'max_parts', None, -1, ), # 4 ) - def __init__(self, db_name=None, tbl_name=None, part_name=None, deleteData=None,): + def __init__(self, db_name=None, tbl_name=None, part_vals=None, max_parts=thrift_spec[4][4],): self.db_name = db_name self.tbl_name = tbl_name - self.part_name = part_name - self.deleteData = deleteData + self.part_vals = part_vals + self.max_parts = max_parts def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -11346,13 +14254,18 @@ def read(self, iprot): else: iprot.skip(ftype) elif fid == 3: - if ftype == TType.STRING: - self.part_name = iprot.readString(); + if ftype == TType.LIST: + self.part_vals = [] + (_etype565, _size562) = iprot.readListBegin() + for _i566 in xrange(_size562): + _elem567 = iprot.readString(); + self.part_vals.append(_elem567) + iprot.readListEnd() else: iprot.skip(ftype) elif fid == 4: - if ftype == TType.BOOL: - self.deleteData = iprot.readBool(); + if ftype == TType.I16: + self.max_parts = iprot.readI16(); else: iprot.skip(ftype) else: @@ -11364,7 +14277,7 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('drop_partition_by_name_args') + oprot.writeStructBegin('get_partition_names_ps_args') if self.db_name is not None: oprot.writeFieldBegin('db_name', TType.STRING, 1) oprot.writeString(self.db_name) @@ -11373,13 +14286,16 @@ def write(self, oprot): oprot.writeFieldBegin('tbl_name', TType.STRING, 2) oprot.writeString(self.tbl_name) oprot.writeFieldEnd() - if self.part_name is not None: - oprot.writeFieldBegin('part_name', TType.STRING, 3) - oprot.writeString(self.part_name) + if self.part_vals is not None: + oprot.writeFieldBegin('part_vals', TType.LIST, 3) + oprot.writeListBegin(TType.STRING, len(self.part_vals)) + for iter568 in self.part_vals: + oprot.writeString(iter568) + oprot.writeListEnd() oprot.writeFieldEnd() - if self.deleteData is not None: - oprot.writeFieldBegin('deleteData', TType.BOOL, 4) - oprot.writeBool(self.deleteData) + if self.max_parts is not None: + oprot.writeFieldBegin('max_parts', TType.I16, 4) + oprot.writeI16(self.max_parts) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -11399,7 +14315,7 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class drop_partition_by_name_result: +class get_partition_names_ps_result: """ Attributes: - success @@ -11408,9 +14324,9 @@ class drop_partition_by_name_result: """ thrift_spec = ( - (0, TType.BOOL, 'success', None, None, ), # 0 - (1, TType.STRUCT, 'o1', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 1 - (2, TType.STRUCT, 'o2', (MetaException, MetaException.thrift_spec), None, ), # 2 + (0, TType.LIST, 'success', (TType.STRING,None), None, ), # 0 + (1, TType.STRUCT, 'o1', (MetaException, MetaException.thrift_spec), None, ), # 1 + (2, TType.STRUCT, 'o2', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 2 ) def __init__(self, success=None, o1=None, o2=None,): @@ -11428,19 +14344,24 @@ def read(self, iprot): if ftype == TType.STOP: break if fid == 0: - if ftype == TType.BOOL: - self.success = iprot.readBool(); + if ftype == TType.LIST: + self.success = [] + (_etype572, _size569) = iprot.readListBegin() + for _i573 in xrange(_size569): + _elem574 = iprot.readString(); + self.success.append(_elem574) + iprot.readListEnd() else: iprot.skip(ftype) elif fid == 1: if ftype == TType.STRUCT: - self.o1 = NoSuchObjectException() + self.o1 = MetaException() self.o1.read(iprot) else: iprot.skip(ftype) elif fid == 2: if ftype == TType.STRUCT: - self.o2 = MetaException() + self.o2 = NoSuchObjectException() self.o2.read(iprot) else: iprot.skip(ftype) @@ -11453,10 +14374,13 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('drop_partition_by_name_result') + oprot.writeStructBegin('get_partition_names_ps_result') if self.success is not None: - oprot.writeFieldBegin('success', TType.BOOL, 0) - oprot.writeBool(self.success) + oprot.writeFieldBegin('success', TType.LIST, 0) + oprot.writeListBegin(TType.STRING, len(self.success)) + for iter575 in self.success: + oprot.writeString(iter575) + oprot.writeListEnd() oprot.writeFieldEnd() if self.o1 is not None: oprot.writeFieldBegin('o1', TType.STRUCT, 1) @@ -11484,31 +14408,28 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class drop_partition_by_name_with_environment_context_args: +class get_partitions_by_filter_args: """ Attributes: - db_name - tbl_name - - part_name - - deleteData - - environment_context + - filter + - max_parts """ thrift_spec = ( None, # 0 (1, TType.STRING, 'db_name', None, None, ), # 1 (2, TType.STRING, 'tbl_name', None, None, ), # 2 - (3, TType.STRING, 'part_name', None, None, ), # 3 - (4, TType.BOOL, 'deleteData', None, None, ), # 4 - (5, TType.STRUCT, 'environment_context', (EnvironmentContext, EnvironmentContext.thrift_spec), None, ), # 5 + (3, TType.STRING, 'filter', None, None, ), # 3 + (4, TType.I16, 'max_parts', None, -1, ), # 4 ) - def __init__(self, db_name=None, tbl_name=None, part_name=None, deleteData=None, environment_context=None,): + def __init__(self, db_name=None, tbl_name=None, filter=None, max_parts=thrift_spec[4][4],): self.db_name = db_name self.tbl_name = tbl_name - self.part_name = part_name - self.deleteData = deleteData - self.environment_context = environment_context + self.filter = filter + self.max_parts = max_parts def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -11531,18 +14452,12 @@ def read(self, iprot): iprot.skip(ftype) elif fid == 3: if ftype == TType.STRING: - self.part_name = iprot.readString(); + self.filter = iprot.readString(); else: iprot.skip(ftype) elif fid == 4: - if ftype == TType.BOOL: - self.deleteData = iprot.readBool(); - else: - iprot.skip(ftype) - elif fid == 5: - if ftype == TType.STRUCT: - self.environment_context = EnvironmentContext() - self.environment_context.read(iprot) + if ftype == TType.I16: + self.max_parts = iprot.readI16(); else: iprot.skip(ftype) else: @@ -11554,7 +14469,7 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('drop_partition_by_name_with_environment_context_args') + oprot.writeStructBegin('get_partitions_by_filter_args') if self.db_name is not None: oprot.writeFieldBegin('db_name', TType.STRING, 1) oprot.writeString(self.db_name) @@ -11563,17 +14478,13 @@ def write(self, oprot): oprot.writeFieldBegin('tbl_name', TType.STRING, 2) oprot.writeString(self.tbl_name) oprot.writeFieldEnd() - if self.part_name is not None: - oprot.writeFieldBegin('part_name', TType.STRING, 3) - oprot.writeString(self.part_name) - oprot.writeFieldEnd() - if self.deleteData is not None: - oprot.writeFieldBegin('deleteData', TType.BOOL, 4) - oprot.writeBool(self.deleteData) + if self.filter is not None: + oprot.writeFieldBegin('filter', TType.STRING, 3) + oprot.writeString(self.filter) oprot.writeFieldEnd() - if self.environment_context is not None: - oprot.writeFieldBegin('environment_context', TType.STRUCT, 5) - self.environment_context.write(oprot) + if self.max_parts is not None: + oprot.writeFieldBegin('max_parts', TType.I16, 4) + oprot.writeI16(self.max_parts) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -11593,7 +14504,7 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class drop_partition_by_name_with_environment_context_result: +class get_partitions_by_filter_result: """ Attributes: - success @@ -11602,9 +14513,9 @@ class drop_partition_by_name_with_environment_context_result: """ thrift_spec = ( - (0, TType.BOOL, 'success', None, None, ), # 0 - (1, TType.STRUCT, 'o1', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 1 - (2, TType.STRUCT, 'o2', (MetaException, MetaException.thrift_spec), None, ), # 2 + (0, TType.LIST, 'success', (TType.STRUCT,(Partition, Partition.thrift_spec)), None, ), # 0 + (1, TType.STRUCT, 'o1', (MetaException, MetaException.thrift_spec), None, ), # 1 + (2, TType.STRUCT, 'o2', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 2 ) def __init__(self, success=None, o1=None, o2=None,): @@ -11622,19 +14533,25 @@ def read(self, iprot): if ftype == TType.STOP: break if fid == 0: - if ftype == TType.BOOL: - self.success = iprot.readBool(); + if ftype == TType.LIST: + self.success = [] + (_etype579, _size576) = iprot.readListBegin() + for _i580 in xrange(_size576): + _elem581 = Partition() + _elem581.read(iprot) + self.success.append(_elem581) + iprot.readListEnd() else: iprot.skip(ftype) elif fid == 1: if ftype == TType.STRUCT: - self.o1 = NoSuchObjectException() + self.o1 = MetaException() self.o1.read(iprot) else: iprot.skip(ftype) elif fid == 2: if ftype == TType.STRUCT: - self.o2 = MetaException() + self.o2 = NoSuchObjectException() self.o2.read(iprot) else: iprot.skip(ftype) @@ -11647,10 +14564,13 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('drop_partition_by_name_with_environment_context_result') + oprot.writeStructBegin('get_partitions_by_filter_result') if self.success is not None: - oprot.writeFieldBegin('success', TType.BOOL, 0) - oprot.writeBool(self.success) + oprot.writeFieldBegin('success', TType.LIST, 0) + oprot.writeListBegin(TType.STRUCT, len(self.success)) + for iter582 in self.success: + iter582.write(oprot) + oprot.writeListEnd() oprot.writeFieldEnd() if self.o1 is not None: oprot.writeFieldBegin('o1', TType.STRUCT, 1) @@ -11678,7 +14598,7 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class drop_partitions_req_args: +class get_partitions_by_expr_args: """ Attributes: - req @@ -11686,7 +14606,7 @@ class drop_partitions_req_args: thrift_spec = ( None, # 0 - (1, TType.STRUCT, 'req', (DropPartitionsRequest, DropPartitionsRequest.thrift_spec), None, ), # 1 + (1, TType.STRUCT, 'req', (PartitionsByExprRequest, PartitionsByExprRequest.thrift_spec), None, ), # 1 ) def __init__(self, req=None,): @@ -11703,7 +14623,7 @@ def read(self, iprot): break if fid == 1: if ftype == TType.STRUCT: - self.req = DropPartitionsRequest() + self.req = PartitionsByExprRequest() self.req.read(iprot) else: iprot.skip(ftype) @@ -11716,7 +14636,7 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('drop_partitions_req_args') + oprot.writeStructBegin('get_partitions_by_expr_args') if self.req is not None: oprot.writeFieldBegin('req', TType.STRUCT, 1) self.req.write(oprot) @@ -11739,7 +14659,7 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class drop_partitions_req_result: +class get_partitions_by_expr_result: """ Attributes: - success @@ -11748,9 +14668,9 @@ class drop_partitions_req_result: """ thrift_spec = ( - (0, TType.STRUCT, 'success', (DropPartitionsResult, DropPartitionsResult.thrift_spec), None, ), # 0 - (1, TType.STRUCT, 'o1', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 1 - (2, TType.STRUCT, 'o2', (MetaException, MetaException.thrift_spec), None, ), # 2 + (0, TType.STRUCT, 'success', (PartitionsByExprResult, PartitionsByExprResult.thrift_spec), None, ), # 0 + (1, TType.STRUCT, 'o1', (MetaException, MetaException.thrift_spec), None, ), # 1 + (2, TType.STRUCT, 'o2', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 2 ) def __init__(self, success=None, o1=None, o2=None,): @@ -11769,19 +14689,19 @@ def read(self, iprot): break if fid == 0: if ftype == TType.STRUCT: - self.success = DropPartitionsResult() + self.success = PartitionsByExprResult() self.success.read(iprot) else: iprot.skip(ftype) elif fid == 1: if ftype == TType.STRUCT: - self.o1 = NoSuchObjectException() + self.o1 = MetaException() self.o1.read(iprot) else: iprot.skip(ftype) elif fid == 2: if ftype == TType.STRUCT: - self.o2 = MetaException() + self.o2 = NoSuchObjectException() self.o2.read(iprot) else: iprot.skip(ftype) @@ -11794,7 +14714,7 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('drop_partitions_req_result') + oprot.writeStructBegin('get_partitions_by_expr_result') if self.success is not None: oprot.writeFieldBegin('success', TType.STRUCT, 0) self.success.write(oprot) @@ -11825,25 +14745,25 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class get_partition_args: +class get_partitions_by_names_args: """ Attributes: - db_name - tbl_name - - part_vals + - names """ thrift_spec = ( None, # 0 (1, TType.STRING, 'db_name', None, None, ), # 1 (2, TType.STRING, 'tbl_name', None, None, ), # 2 - (3, TType.LIST, 'part_vals', (TType.STRING,None), None, ), # 3 + (3, TType.LIST, 'names', (TType.STRING,None), None, ), # 3 ) - def __init__(self, db_name=None, tbl_name=None, part_vals=None,): + def __init__(self, db_name=None, tbl_name=None, names=None,): self.db_name = db_name self.tbl_name = tbl_name - self.part_vals = part_vals + self.names = names def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -11866,11 +14786,11 @@ def read(self, iprot): iprot.skip(ftype) elif fid == 3: if ftype == TType.LIST: - self.part_vals = [] - (_etype430, _size427) = iprot.readListBegin() - for _i431 in xrange(_size427): - _elem432 = iprot.readString(); - self.part_vals.append(_elem432) + self.names = [] + (_etype586, _size583) = iprot.readListBegin() + for _i587 in xrange(_size583): + _elem588 = iprot.readString(); + self.names.append(_elem588) iprot.readListEnd() else: iprot.skip(ftype) @@ -11883,7 +14803,7 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('get_partition_args') + oprot.writeStructBegin('get_partitions_by_names_args') if self.db_name is not None: oprot.writeFieldBegin('db_name', TType.STRING, 1) oprot.writeString(self.db_name) @@ -11892,11 +14812,11 @@ def write(self, oprot): oprot.writeFieldBegin('tbl_name', TType.STRING, 2) oprot.writeString(self.tbl_name) oprot.writeFieldEnd() - if self.part_vals is not None: - oprot.writeFieldBegin('part_vals', TType.LIST, 3) - oprot.writeListBegin(TType.STRING, len(self.part_vals)) - for iter433 in self.part_vals: - oprot.writeString(iter433) + if self.names is not None: + oprot.writeFieldBegin('names', TType.LIST, 3) + oprot.writeListBegin(TType.STRING, len(self.names)) + for iter589 in self.names: + oprot.writeString(iter589) oprot.writeListEnd() oprot.writeFieldEnd() oprot.writeFieldStop() @@ -11917,7 +14837,7 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class get_partition_result: +class get_partitions_by_names_result: """ Attributes: - success @@ -11926,7 +14846,7 @@ class get_partition_result: """ thrift_spec = ( - (0, TType.STRUCT, 'success', (Partition, Partition.thrift_spec), None, ), # 0 + (0, TType.LIST, 'success', (TType.STRUCT,(Partition, Partition.thrift_spec)), None, ), # 0 (1, TType.STRUCT, 'o1', (MetaException, MetaException.thrift_spec), None, ), # 1 (2, TType.STRUCT, 'o2', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 2 ) @@ -11946,9 +14866,14 @@ def read(self, iprot): if ftype == TType.STOP: break if fid == 0: - if ftype == TType.STRUCT: - self.success = Partition() - self.success.read(iprot) + if ftype == TType.LIST: + self.success = [] + (_etype593, _size590) = iprot.readListBegin() + for _i594 in xrange(_size590): + _elem595 = Partition() + _elem595.read(iprot) + self.success.append(_elem595) + iprot.readListEnd() else: iprot.skip(ftype) elif fid == 1: @@ -11972,10 +14897,13 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('get_partition_result') + oprot.writeStructBegin('get_partitions_by_names_result') if self.success is not None: - oprot.writeFieldBegin('success', TType.STRUCT, 0) - self.success.write(oprot) + oprot.writeFieldBegin('success', TType.LIST, 0) + oprot.writeListBegin(TType.STRUCT, len(self.success)) + for iter596 in self.success: + iter596.write(oprot) + oprot.writeListEnd() oprot.writeFieldEnd() if self.o1 is not None: oprot.writeFieldBegin('o1', TType.STRUCT, 1) @@ -12003,31 +14931,25 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class exchange_partition_args: +class alter_partition_args: """ Attributes: - - partitionSpecs - - source_db - - source_table_name - - dest_db - - dest_table_name + - db_name + - tbl_name + - new_part """ thrift_spec = ( None, # 0 - (1, TType.MAP, 'partitionSpecs', (TType.STRING,None,TType.STRING,None), None, ), # 1 - (2, TType.STRING, 'source_db', None, None, ), # 2 - (3, TType.STRING, 'source_table_name', None, None, ), # 3 - (4, TType.STRING, 'dest_db', None, None, ), # 4 - (5, TType.STRING, 'dest_table_name', None, None, ), # 5 - ) - - def __init__(self, partitionSpecs=None, source_db=None, source_table_name=None, dest_db=None, dest_table_name=None,): - self.partitionSpecs = partitionSpecs - self.source_db = source_db - self.source_table_name = source_table_name - self.dest_db = dest_db - self.dest_table_name = dest_table_name + (1, TType.STRING, 'db_name', None, None, ), # 1 + (2, TType.STRING, 'tbl_name', None, None, ), # 2 + (3, TType.STRUCT, 'new_part', (Partition, Partition.thrift_spec), None, ), # 3 + ) + + def __init__(self, db_name=None, tbl_name=None, new_part=None,): + self.db_name = db_name + self.tbl_name = tbl_name + self.new_part = new_part def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -12039,34 +14961,19 @@ def read(self, iprot): if ftype == TType.STOP: break if fid == 1: - if ftype == TType.MAP: - self.partitionSpecs = {} - (_ktype435, _vtype436, _size434 ) = iprot.readMapBegin() - for _i438 in xrange(_size434): - _key439 = iprot.readString(); - _val440 = iprot.readString(); - self.partitionSpecs[_key439] = _val440 - iprot.readMapEnd() + if ftype == TType.STRING: + self.db_name = iprot.readString(); else: iprot.skip(ftype) elif fid == 2: if ftype == TType.STRING: - self.source_db = iprot.readString(); + self.tbl_name = iprot.readString(); else: iprot.skip(ftype) elif fid == 3: - if ftype == TType.STRING: - self.source_table_name = iprot.readString(); - else: - iprot.skip(ftype) - elif fid == 4: - if ftype == TType.STRING: - self.dest_db = iprot.readString(); - else: - iprot.skip(ftype) - elif fid == 5: - if ftype == TType.STRING: - self.dest_table_name = iprot.readString(); + if ftype == TType.STRUCT: + self.new_part = Partition() + self.new_part.read(iprot) else: iprot.skip(ftype) else: @@ -12078,30 +14985,18 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('exchange_partition_args') - if self.partitionSpecs is not None: - oprot.writeFieldBegin('partitionSpecs', TType.MAP, 1) - oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.partitionSpecs)) - for kiter441,viter442 in self.partitionSpecs.items(): - oprot.writeString(kiter441) - oprot.writeString(viter442) - oprot.writeMapEnd() - oprot.writeFieldEnd() - if self.source_db is not None: - oprot.writeFieldBegin('source_db', TType.STRING, 2) - oprot.writeString(self.source_db) - oprot.writeFieldEnd() - if self.source_table_name is not None: - oprot.writeFieldBegin('source_table_name', TType.STRING, 3) - oprot.writeString(self.source_table_name) + oprot.writeStructBegin('alter_partition_args') + if self.db_name is not None: + oprot.writeFieldBegin('db_name', TType.STRING, 1) + oprot.writeString(self.db_name) oprot.writeFieldEnd() - if self.dest_db is not None: - oprot.writeFieldBegin('dest_db', TType.STRING, 4) - oprot.writeString(self.dest_db) + if self.tbl_name is not None: + oprot.writeFieldBegin('tbl_name', TType.STRING, 2) + oprot.writeString(self.tbl_name) oprot.writeFieldEnd() - if self.dest_table_name is not None: - oprot.writeFieldBegin('dest_table_name', TType.STRING, 5) - oprot.writeString(self.dest_table_name) + if self.new_part is not None: + oprot.writeFieldBegin('new_part', TType.STRUCT, 3) + self.new_part.write(oprot) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -12121,30 +15016,22 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class exchange_partition_result: +class alter_partition_result: """ Attributes: - - success - o1 - o2 - - o3 - - o4 """ thrift_spec = ( - (0, TType.STRUCT, 'success', (Partition, Partition.thrift_spec), None, ), # 0 - (1, TType.STRUCT, 'o1', (MetaException, MetaException.thrift_spec), None, ), # 1 - (2, TType.STRUCT, 'o2', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 2 - (3, TType.STRUCT, 'o3', (InvalidObjectException, InvalidObjectException.thrift_spec), None, ), # 3 - (4, TType.STRUCT, 'o4', (InvalidInputException, InvalidInputException.thrift_spec), None, ), # 4 + None, # 0 + (1, TType.STRUCT, 'o1', (InvalidOperationException, InvalidOperationException.thrift_spec), None, ), # 1 + (2, TType.STRUCT, 'o2', (MetaException, MetaException.thrift_spec), None, ), # 2 ) - def __init__(self, success=None, o1=None, o2=None, o3=None, o4=None,): - self.success = success + def __init__(self, o1=None, o2=None,): self.o1 = o1 self.o2 = o2 - self.o3 = o3 - self.o4 = o4 def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -12155,36 +15042,18 @@ def read(self, iprot): (fname, ftype, fid) = iprot.readFieldBegin() if ftype == TType.STOP: break - if fid == 0: - if ftype == TType.STRUCT: - self.success = Partition() - self.success.read(iprot) - else: - iprot.skip(ftype) - elif fid == 1: + if fid == 1: if ftype == TType.STRUCT: - self.o1 = MetaException() + self.o1 = InvalidOperationException() self.o1.read(iprot) else: iprot.skip(ftype) elif fid == 2: if ftype == TType.STRUCT: - self.o2 = NoSuchObjectException() + self.o2 = MetaException() self.o2.read(iprot) else: iprot.skip(ftype) - elif fid == 3: - if ftype == TType.STRUCT: - self.o3 = InvalidObjectException() - self.o3.read(iprot) - else: - iprot.skip(ftype) - elif fid == 4: - if ftype == TType.STRUCT: - self.o4 = InvalidInputException() - self.o4.read(iprot) - else: - iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() @@ -12194,11 +15063,7 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('exchange_partition_result') - if self.success is not None: - oprot.writeFieldBegin('success', TType.STRUCT, 0) - self.success.write(oprot) - oprot.writeFieldEnd() + oprot.writeStructBegin('alter_partition_result') if self.o1 is not None: oprot.writeFieldBegin('o1', TType.STRUCT, 1) self.o1.write(oprot) @@ -12207,14 +15072,6 @@ def write(self, oprot): oprot.writeFieldBegin('o2', TType.STRUCT, 2) self.o2.write(oprot) oprot.writeFieldEnd() - if self.o3 is not None: - oprot.writeFieldBegin('o3', TType.STRUCT, 3) - self.o3.write(oprot) - oprot.writeFieldEnd() - if self.o4 is not None: - oprot.writeFieldBegin('o4', TType.STRUCT, 4) - self.o4.write(oprot) - oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -12233,31 +15090,25 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class get_partition_with_auth_args: +class alter_partitions_args: """ Attributes: - db_name - tbl_name - - part_vals - - user_name - - group_names + - new_parts """ thrift_spec = ( None, # 0 (1, TType.STRING, 'db_name', None, None, ), # 1 (2, TType.STRING, 'tbl_name', None, None, ), # 2 - (3, TType.LIST, 'part_vals', (TType.STRING,None), None, ), # 3 - (4, TType.STRING, 'user_name', None, None, ), # 4 - (5, TType.LIST, 'group_names', (TType.STRING,None), None, ), # 5 + (3, TType.LIST, 'new_parts', (TType.STRUCT,(Partition, Partition.thrift_spec)), None, ), # 3 ) - def __init__(self, db_name=None, tbl_name=None, part_vals=None, user_name=None, group_names=None,): + def __init__(self, db_name=None, tbl_name=None, new_parts=None,): self.db_name = db_name self.tbl_name = tbl_name - self.part_vals = part_vals - self.user_name = user_name - self.group_names = group_names + self.new_parts = new_parts def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -12280,26 +15131,12 @@ def read(self, iprot): iprot.skip(ftype) elif fid == 3: if ftype == TType.LIST: - self.part_vals = [] - (_etype446, _size443) = iprot.readListBegin() - for _i447 in xrange(_size443): - _elem448 = iprot.readString(); - self.part_vals.append(_elem448) - iprot.readListEnd() - else: - iprot.skip(ftype) - elif fid == 4: - if ftype == TType.STRING: - self.user_name = iprot.readString(); - else: - iprot.skip(ftype) - elif fid == 5: - if ftype == TType.LIST: - self.group_names = [] - (_etype452, _size449) = iprot.readListBegin() - for _i453 in xrange(_size449): - _elem454 = iprot.readString(); - self.group_names.append(_elem454) + self.new_parts = [] + (_etype600, _size597) = iprot.readListBegin() + for _i601 in xrange(_size597): + _elem602 = Partition() + _elem602.read(iprot) + self.new_parts.append(_elem602) iprot.readListEnd() else: iprot.skip(ftype) @@ -12312,7 +15149,7 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('get_partition_with_auth_args') + oprot.writeStructBegin('alter_partitions_args') if self.db_name is not None: oprot.writeFieldBegin('db_name', TType.STRING, 1) oprot.writeString(self.db_name) @@ -12321,22 +15158,11 @@ def write(self, oprot): oprot.writeFieldBegin('tbl_name', TType.STRING, 2) oprot.writeString(self.tbl_name) oprot.writeFieldEnd() - if self.part_vals is not None: - oprot.writeFieldBegin('part_vals', TType.LIST, 3) - oprot.writeListBegin(TType.STRING, len(self.part_vals)) - for iter455 in self.part_vals: - oprot.writeString(iter455) - oprot.writeListEnd() - oprot.writeFieldEnd() - if self.user_name is not None: - oprot.writeFieldBegin('user_name', TType.STRING, 4) - oprot.writeString(self.user_name) - oprot.writeFieldEnd() - if self.group_names is not None: - oprot.writeFieldBegin('group_names', TType.LIST, 5) - oprot.writeListBegin(TType.STRING, len(self.group_names)) - for iter456 in self.group_names: - oprot.writeString(iter456) + if self.new_parts is not None: + oprot.writeFieldBegin('new_parts', TType.LIST, 3) + oprot.writeListBegin(TType.STRUCT, len(self.new_parts)) + for iter603 in self.new_parts: + iter603.write(oprot) oprot.writeListEnd() oprot.writeFieldEnd() oprot.writeFieldStop() @@ -12357,22 +15183,20 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class get_partition_with_auth_result: +class alter_partitions_result: """ Attributes: - - success - o1 - o2 """ thrift_spec = ( - (0, TType.STRUCT, 'success', (Partition, Partition.thrift_spec), None, ), # 0 - (1, TType.STRUCT, 'o1', (MetaException, MetaException.thrift_spec), None, ), # 1 - (2, TType.STRUCT, 'o2', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 2 + None, # 0 + (1, TType.STRUCT, 'o1', (InvalidOperationException, InvalidOperationException.thrift_spec), None, ), # 1 + (2, TType.STRUCT, 'o2', (MetaException, MetaException.thrift_spec), None, ), # 2 ) - def __init__(self, success=None, o1=None, o2=None,): - self.success = success + def __init__(self, o1=None, o2=None,): self.o1 = o1 self.o2 = o2 @@ -12385,21 +15209,15 @@ def read(self, iprot): (fname, ftype, fid) = iprot.readFieldBegin() if ftype == TType.STOP: break - if fid == 0: - if ftype == TType.STRUCT: - self.success = Partition() - self.success.read(iprot) - else: - iprot.skip(ftype) - elif fid == 1: + if fid == 1: if ftype == TType.STRUCT: - self.o1 = MetaException() + self.o1 = InvalidOperationException() self.o1.read(iprot) else: iprot.skip(ftype) elif fid == 2: if ftype == TType.STRUCT: - self.o2 = NoSuchObjectException() + self.o2 = MetaException() self.o2.read(iprot) else: iprot.skip(ftype) @@ -12412,11 +15230,7 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('get_partition_with_auth_result') - if self.success is not None: - oprot.writeFieldBegin('success', TType.STRUCT, 0) - self.success.write(oprot) - oprot.writeFieldEnd() + oprot.writeStructBegin('alter_partitions_result') if self.o1 is not None: oprot.writeFieldBegin('o1', TType.STRUCT, 1) self.o1.write(oprot) @@ -12443,25 +15257,28 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class get_partition_by_name_args: +class alter_partition_with_environment_context_args: """ Attributes: - db_name - tbl_name - - part_name + - new_part + - environment_context """ thrift_spec = ( None, # 0 (1, TType.STRING, 'db_name', None, None, ), # 1 (2, TType.STRING, 'tbl_name', None, None, ), # 2 - (3, TType.STRING, 'part_name', None, None, ), # 3 + (3, TType.STRUCT, 'new_part', (Partition, Partition.thrift_spec), None, ), # 3 + (4, TType.STRUCT, 'environment_context', (EnvironmentContext, EnvironmentContext.thrift_spec), None, ), # 4 ) - def __init__(self, db_name=None, tbl_name=None, part_name=None,): + def __init__(self, db_name=None, tbl_name=None, new_part=None, environment_context=None,): self.db_name = db_name self.tbl_name = tbl_name - self.part_name = part_name + self.new_part = new_part + self.environment_context = environment_context def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -12483,8 +15300,15 @@ def read(self, iprot): else: iprot.skip(ftype) elif fid == 3: - if ftype == TType.STRING: - self.part_name = iprot.readString(); + if ftype == TType.STRUCT: + self.new_part = Partition() + self.new_part.read(iprot) + else: + iprot.skip(ftype) + elif fid == 4: + if ftype == TType.STRUCT: + self.environment_context = EnvironmentContext() + self.environment_context.read(iprot) else: iprot.skip(ftype) else: @@ -12496,7 +15320,7 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('get_partition_by_name_args') + oprot.writeStructBegin('alter_partition_with_environment_context_args') if self.db_name is not None: oprot.writeFieldBegin('db_name', TType.STRING, 1) oprot.writeString(self.db_name) @@ -12505,9 +15329,13 @@ def write(self, oprot): oprot.writeFieldBegin('tbl_name', TType.STRING, 2) oprot.writeString(self.tbl_name) oprot.writeFieldEnd() - if self.part_name is not None: - oprot.writeFieldBegin('part_name', TType.STRING, 3) - oprot.writeString(self.part_name) + if self.new_part is not None: + oprot.writeFieldBegin('new_part', TType.STRUCT, 3) + self.new_part.write(oprot) + oprot.writeFieldEnd() + if self.environment_context is not None: + oprot.writeFieldBegin('environment_context', TType.STRUCT, 4) + self.environment_context.write(oprot) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -12527,22 +15355,20 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class get_partition_by_name_result: +class alter_partition_with_environment_context_result: """ Attributes: - - success - o1 - o2 """ thrift_spec = ( - (0, TType.STRUCT, 'success', (Partition, Partition.thrift_spec), None, ), # 0 - (1, TType.STRUCT, 'o1', (MetaException, MetaException.thrift_spec), None, ), # 1 - (2, TType.STRUCT, 'o2', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 2 + None, # 0 + (1, TType.STRUCT, 'o1', (InvalidOperationException, InvalidOperationException.thrift_spec), None, ), # 1 + (2, TType.STRUCT, 'o2', (MetaException, MetaException.thrift_spec), None, ), # 2 ) - def __init__(self, success=None, o1=None, o2=None,): - self.success = success + def __init__(self, o1=None, o2=None,): self.o1 = o1 self.o2 = o2 @@ -12555,21 +15381,15 @@ def read(self, iprot): (fname, ftype, fid) = iprot.readFieldBegin() if ftype == TType.STOP: break - if fid == 0: - if ftype == TType.STRUCT: - self.success = Partition() - self.success.read(iprot) - else: - iprot.skip(ftype) - elif fid == 1: + if fid == 1: if ftype == TType.STRUCT: - self.o1 = MetaException() + self.o1 = InvalidOperationException() self.o1.read(iprot) else: iprot.skip(ftype) elif fid == 2: if ftype == TType.STRUCT: - self.o2 = NoSuchObjectException() + self.o2 = MetaException() self.o2.read(iprot) else: iprot.skip(ftype) @@ -12582,11 +15402,7 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('get_partition_by_name_result') - if self.success is not None: - oprot.writeFieldBegin('success', TType.STRUCT, 0) - self.success.write(oprot) - oprot.writeFieldEnd() + oprot.writeStructBegin('alter_partition_with_environment_context_result') if self.o1 is not None: oprot.writeFieldBegin('o1', TType.STRUCT, 1) self.o1.write(oprot) @@ -12613,25 +15429,28 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class get_partitions_args: +class rename_partition_args: """ Attributes: - db_name - tbl_name - - max_parts + - part_vals + - new_part """ thrift_spec = ( None, # 0 (1, TType.STRING, 'db_name', None, None, ), # 1 (2, TType.STRING, 'tbl_name', None, None, ), # 2 - (3, TType.I16, 'max_parts', None, -1, ), # 3 + (3, TType.LIST, 'part_vals', (TType.STRING,None), None, ), # 3 + (4, TType.STRUCT, 'new_part', (Partition, Partition.thrift_spec), None, ), # 4 ) - def __init__(self, db_name=None, tbl_name=None, max_parts=thrift_spec[3][4],): + def __init__(self, db_name=None, tbl_name=None, part_vals=None, new_part=None,): self.db_name = db_name self.tbl_name = tbl_name - self.max_parts = max_parts + self.part_vals = part_vals + self.new_part = new_part def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -12653,8 +15472,19 @@ def read(self, iprot): else: iprot.skip(ftype) elif fid == 3: - if ftype == TType.I16: - self.max_parts = iprot.readI16(); + if ftype == TType.LIST: + self.part_vals = [] + (_etype607, _size604) = iprot.readListBegin() + for _i608 in xrange(_size604): + _elem609 = iprot.readString(); + self.part_vals.append(_elem609) + iprot.readListEnd() + else: + iprot.skip(ftype) + elif fid == 4: + if ftype == TType.STRUCT: + self.new_part = Partition() + self.new_part.read(iprot) else: iprot.skip(ftype) else: @@ -12666,7 +15496,7 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('get_partitions_args') + oprot.writeStructBegin('rename_partition_args') if self.db_name is not None: oprot.writeFieldBegin('db_name', TType.STRING, 1) oprot.writeString(self.db_name) @@ -12675,9 +15505,16 @@ def write(self, oprot): oprot.writeFieldBegin('tbl_name', TType.STRING, 2) oprot.writeString(self.tbl_name) oprot.writeFieldEnd() - if self.max_parts is not None: - oprot.writeFieldBegin('max_parts', TType.I16, 3) - oprot.writeI16(self.max_parts) + if self.part_vals is not None: + oprot.writeFieldBegin('part_vals', TType.LIST, 3) + oprot.writeListBegin(TType.STRING, len(self.part_vals)) + for iter610 in self.part_vals: + oprot.writeString(iter610) + oprot.writeListEnd() + oprot.writeFieldEnd() + if self.new_part is not None: + oprot.writeFieldBegin('new_part', TType.STRUCT, 4) + self.new_part.write(oprot) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -12697,22 +15534,20 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class get_partitions_result: +class rename_partition_result: """ Attributes: - - success - o1 - o2 """ thrift_spec = ( - (0, TType.LIST, 'success', (TType.STRUCT,(Partition, Partition.thrift_spec)), None, ), # 0 - (1, TType.STRUCT, 'o1', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 1 + None, # 0 + (1, TType.STRUCT, 'o1', (InvalidOperationException, InvalidOperationException.thrift_spec), None, ), # 1 (2, TType.STRUCT, 'o2', (MetaException, MetaException.thrift_spec), None, ), # 2 ) - def __init__(self, success=None, o1=None, o2=None,): - self.success = success + def __init__(self, o1=None, o2=None,): self.o1 = o1 self.o2 = o2 @@ -12725,20 +15560,9 @@ def read(self, iprot): (fname, ftype, fid) = iprot.readFieldBegin() if ftype == TType.STOP: break - if fid == 0: - if ftype == TType.LIST: - self.success = [] - (_etype460, _size457) = iprot.readListBegin() - for _i461 in xrange(_size457): - _elem462 = Partition() - _elem462.read(iprot) - self.success.append(_elem462) - iprot.readListEnd() - else: - iprot.skip(ftype) - elif fid == 1: + if fid == 1: if ftype == TType.STRUCT: - self.o1 = NoSuchObjectException() + self.o1 = InvalidOperationException() self.o1.read(iprot) else: iprot.skip(ftype) @@ -12757,14 +15581,7 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('get_partitions_result') - if self.success is not None: - oprot.writeFieldBegin('success', TType.LIST, 0) - oprot.writeListBegin(TType.STRUCT, len(self.success)) - for iter463 in self.success: - iter463.write(oprot) - oprot.writeListEnd() - oprot.writeFieldEnd() + oprot.writeStructBegin('rename_partition_result') if self.o1 is not None: oprot.writeFieldBegin('o1', TType.STRUCT, 1) self.o1.write(oprot) @@ -12791,31 +15608,22 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class get_partitions_with_auth_args: +class partition_name_has_valid_characters_args: """ Attributes: - - db_name - - tbl_name - - max_parts - - user_name - - group_names + - part_vals + - throw_exception """ thrift_spec = ( None, # 0 - (1, TType.STRING, 'db_name', None, None, ), # 1 - (2, TType.STRING, 'tbl_name', None, None, ), # 2 - (3, TType.I16, 'max_parts', None, -1, ), # 3 - (4, TType.STRING, 'user_name', None, None, ), # 4 - (5, TType.LIST, 'group_names', (TType.STRING,None), None, ), # 5 + (1, TType.LIST, 'part_vals', (TType.STRING,None), None, ), # 1 + (2, TType.BOOL, 'throw_exception', None, None, ), # 2 ) - def __init__(self, db_name=None, tbl_name=None, max_parts=thrift_spec[3][4], user_name=None, group_names=None,): - self.db_name = db_name - self.tbl_name = tbl_name - self.max_parts = max_parts - self.user_name = user_name - self.group_names = group_names + def __init__(self, part_vals=None, throw_exception=None,): + self.part_vals = part_vals + self.throw_exception = throw_exception def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -12827,33 +15635,18 @@ def read(self, iprot): if ftype == TType.STOP: break if fid == 1: - if ftype == TType.STRING: - self.db_name = iprot.readString(); + if ftype == TType.LIST: + self.part_vals = [] + (_etype614, _size611) = iprot.readListBegin() + for _i615 in xrange(_size611): + _elem616 = iprot.readString(); + self.part_vals.append(_elem616) + iprot.readListEnd() else: iprot.skip(ftype) elif fid == 2: - if ftype == TType.STRING: - self.tbl_name = iprot.readString(); - else: - iprot.skip(ftype) - elif fid == 3: - if ftype == TType.I16: - self.max_parts = iprot.readI16(); - else: - iprot.skip(ftype) - elif fid == 4: - if ftype == TType.STRING: - self.user_name = iprot.readString(); - else: - iprot.skip(ftype) - elif fid == 5: - if ftype == TType.LIST: - self.group_names = [] - (_etype467, _size464) = iprot.readListBegin() - for _i468 in xrange(_size464): - _elem469 = iprot.readString(); - self.group_names.append(_elem469) - iprot.readListEnd() + if ftype == TType.BOOL: + self.throw_exception = iprot.readBool(); else: iprot.skip(ftype) else: @@ -12865,30 +15658,18 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('get_partitions_with_auth_args') - if self.db_name is not None: - oprot.writeFieldBegin('db_name', TType.STRING, 1) - oprot.writeString(self.db_name) - oprot.writeFieldEnd() - if self.tbl_name is not None: - oprot.writeFieldBegin('tbl_name', TType.STRING, 2) - oprot.writeString(self.tbl_name) - oprot.writeFieldEnd() - if self.max_parts is not None: - oprot.writeFieldBegin('max_parts', TType.I16, 3) - oprot.writeI16(self.max_parts) - oprot.writeFieldEnd() - if self.user_name is not None: - oprot.writeFieldBegin('user_name', TType.STRING, 4) - oprot.writeString(self.user_name) - oprot.writeFieldEnd() - if self.group_names is not None: - oprot.writeFieldBegin('group_names', TType.LIST, 5) - oprot.writeListBegin(TType.STRING, len(self.group_names)) - for iter470 in self.group_names: - oprot.writeString(iter470) + oprot.writeStructBegin('partition_name_has_valid_characters_args') + if self.part_vals is not None: + oprot.writeFieldBegin('part_vals', TType.LIST, 1) + oprot.writeListBegin(TType.STRING, len(self.part_vals)) + for iter617 in self.part_vals: + oprot.writeString(iter617) oprot.writeListEnd() oprot.writeFieldEnd() + if self.throw_exception is not None: + oprot.writeFieldBegin('throw_exception', TType.BOOL, 2) + oprot.writeBool(self.throw_exception) + oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -12907,24 +15688,21 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class get_partitions_with_auth_result: +class partition_name_has_valid_characters_result: """ Attributes: - success - o1 - - o2 """ thrift_spec = ( - (0, TType.LIST, 'success', (TType.STRUCT,(Partition, Partition.thrift_spec)), None, ), # 0 - (1, TType.STRUCT, 'o1', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 1 - (2, TType.STRUCT, 'o2', (MetaException, MetaException.thrift_spec), None, ), # 2 + (0, TType.BOOL, 'success', None, None, ), # 0 + (1, TType.STRUCT, 'o1', (MetaException, MetaException.thrift_spec), None, ), # 1 ) - def __init__(self, success=None, o1=None, o2=None,): + def __init__(self, success=None, o1=None,): self.success = success self.o1 = o1 - self.o2 = o2 def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -12936,28 +15714,16 @@ def read(self, iprot): if ftype == TType.STOP: break if fid == 0: - if ftype == TType.LIST: - self.success = [] - (_etype474, _size471) = iprot.readListBegin() - for _i475 in xrange(_size471): - _elem476 = Partition() - _elem476.read(iprot) - self.success.append(_elem476) - iprot.readListEnd() + if ftype == TType.BOOL: + self.success = iprot.readBool(); else: iprot.skip(ftype) elif fid == 1: if ftype == TType.STRUCT: - self.o1 = NoSuchObjectException() + self.o1 = MetaException() self.o1.read(iprot) else: iprot.skip(ftype) - elif fid == 2: - if ftype == TType.STRUCT: - self.o2 = MetaException() - self.o2.read(iprot) - else: - iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() @@ -12967,22 +15733,15 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('get_partitions_with_auth_result') + oprot.writeStructBegin('partition_name_has_valid_characters_result') if self.success is not None: - oprot.writeFieldBegin('success', TType.LIST, 0) - oprot.writeListBegin(TType.STRUCT, len(self.success)) - for iter477 in self.success: - iter477.write(oprot) - oprot.writeListEnd() + oprot.writeFieldBegin('success', TType.BOOL, 0) + oprot.writeBool(self.success) oprot.writeFieldEnd() if self.o1 is not None: oprot.writeFieldBegin('o1', TType.STRUCT, 1) self.o1.write(oprot) oprot.writeFieldEnd() - if self.o2 is not None: - oprot.writeFieldBegin('o2', TType.STRUCT, 2) - self.o2.write(oprot) - oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -13001,25 +15760,22 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class get_partition_names_args: +class get_config_value_args: """ Attributes: - - db_name - - tbl_name - - max_parts + - name + - defaultValue """ thrift_spec = ( None, # 0 - (1, TType.STRING, 'db_name', None, None, ), # 1 - (2, TType.STRING, 'tbl_name', None, None, ), # 2 - (3, TType.I16, 'max_parts', None, -1, ), # 3 + (1, TType.STRING, 'name', None, None, ), # 1 + (2, TType.STRING, 'defaultValue', None, None, ), # 2 ) - def __init__(self, db_name=None, tbl_name=None, max_parts=thrift_spec[3][4],): - self.db_name = db_name - self.tbl_name = tbl_name - self.max_parts = max_parts + def __init__(self, name=None, defaultValue=None,): + self.name = name + self.defaultValue = defaultValue def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -13032,17 +15788,12 @@ def read(self, iprot): break if fid == 1: if ftype == TType.STRING: - self.db_name = iprot.readString(); + self.name = iprot.readString(); else: iprot.skip(ftype) elif fid == 2: if ftype == TType.STRING: - self.tbl_name = iprot.readString(); - else: - iprot.skip(ftype) - elif fid == 3: - if ftype == TType.I16: - self.max_parts = iprot.readI16(); + self.defaultValue = iprot.readString(); else: iprot.skip(ftype) else: @@ -13054,18 +15805,14 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('get_partition_names_args') - if self.db_name is not None: - oprot.writeFieldBegin('db_name', TType.STRING, 1) - oprot.writeString(self.db_name) - oprot.writeFieldEnd() - if self.tbl_name is not None: - oprot.writeFieldBegin('tbl_name', TType.STRING, 2) - oprot.writeString(self.tbl_name) + oprot.writeStructBegin('get_config_value_args') + if self.name is not None: + oprot.writeFieldBegin('name', TType.STRING, 1) + oprot.writeString(self.name) oprot.writeFieldEnd() - if self.max_parts is not None: - oprot.writeFieldBegin('max_parts', TType.I16, 3) - oprot.writeI16(self.max_parts) + if self.defaultValue is not None: + oprot.writeFieldBegin('defaultValue', TType.STRING, 2) + oprot.writeString(self.defaultValue) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -13085,21 +15832,21 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class get_partition_names_result: +class get_config_value_result: """ Attributes: - success - - o2 + - o1 """ thrift_spec = ( - (0, TType.LIST, 'success', (TType.STRING,None), None, ), # 0 - (1, TType.STRUCT, 'o2', (MetaException, MetaException.thrift_spec), None, ), # 1 + (0, TType.STRING, 'success', None, None, ), # 0 + (1, TType.STRUCT, 'o1', (ConfigValSecurityException, ConfigValSecurityException.thrift_spec), None, ), # 1 ) - def __init__(self, success=None, o2=None,): + def __init__(self, success=None, o1=None,): self.success = success - self.o2 = o2 + self.o1 = o1 def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -13111,19 +15858,14 @@ def read(self, iprot): if ftype == TType.STOP: break if fid == 0: - if ftype == TType.LIST: - self.success = [] - (_etype481, _size478) = iprot.readListBegin() - for _i482 in xrange(_size478): - _elem483 = iprot.readString(); - self.success.append(_elem483) - iprot.readListEnd() + if ftype == TType.STRING: + self.success = iprot.readString(); else: iprot.skip(ftype) elif fid == 1: if ftype == TType.STRUCT: - self.o2 = MetaException() - self.o2.read(iprot) + self.o1 = ConfigValSecurityException() + self.o1.read(iprot) else: iprot.skip(ftype) else: @@ -13135,17 +15877,14 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('get_partition_names_result') + oprot.writeStructBegin('get_config_value_result') if self.success is not None: - oprot.writeFieldBegin('success', TType.LIST, 0) - oprot.writeListBegin(TType.STRING, len(self.success)) - for iter484 in self.success: - oprot.writeString(iter484) - oprot.writeListEnd() + oprot.writeFieldBegin('success', TType.STRING, 0) + oprot.writeString(self.success) oprot.writeFieldEnd() - if self.o2 is not None: - oprot.writeFieldBegin('o2', TType.STRUCT, 1) - self.o2.write(oprot) + if self.o1 is not None: + oprot.writeFieldBegin('o1', TType.STRUCT, 1) + self.o1.write(oprot) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -13165,28 +15904,19 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class get_partitions_ps_args: +class partition_name_to_vals_args: """ Attributes: - - db_name - - tbl_name - - part_vals - - max_parts + - part_name """ thrift_spec = ( None, # 0 - (1, TType.STRING, 'db_name', None, None, ), # 1 - (2, TType.STRING, 'tbl_name', None, None, ), # 2 - (3, TType.LIST, 'part_vals', (TType.STRING,None), None, ), # 3 - (4, TType.I16, 'max_parts', None, -1, ), # 4 + (1, TType.STRING, 'part_name', None, None, ), # 1 ) - def __init__(self, db_name=None, tbl_name=None, part_vals=None, max_parts=thrift_spec[4][4],): - self.db_name = db_name - self.tbl_name = tbl_name - self.part_vals = part_vals - self.max_parts = max_parts + def __init__(self, part_name=None,): + self.part_name = part_name def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -13199,27 +15929,7 @@ def read(self, iprot): break if fid == 1: if ftype == TType.STRING: - self.db_name = iprot.readString(); - else: - iprot.skip(ftype) - elif fid == 2: - if ftype == TType.STRING: - self.tbl_name = iprot.readString(); - else: - iprot.skip(ftype) - elif fid == 3: - if ftype == TType.LIST: - self.part_vals = [] - (_etype488, _size485) = iprot.readListBegin() - for _i489 in xrange(_size485): - _elem490 = iprot.readString(); - self.part_vals.append(_elem490) - iprot.readListEnd() - else: - iprot.skip(ftype) - elif fid == 4: - if ftype == TType.I16: - self.max_parts = iprot.readI16(); + self.part_name = iprot.readString(); else: iprot.skip(ftype) else: @@ -13231,25 +15941,10 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('get_partitions_ps_args') - if self.db_name is not None: - oprot.writeFieldBegin('db_name', TType.STRING, 1) - oprot.writeString(self.db_name) - oprot.writeFieldEnd() - if self.tbl_name is not None: - oprot.writeFieldBegin('tbl_name', TType.STRING, 2) - oprot.writeString(self.tbl_name) - oprot.writeFieldEnd() - if self.part_vals is not None: - oprot.writeFieldBegin('part_vals', TType.LIST, 3) - oprot.writeListBegin(TType.STRING, len(self.part_vals)) - for iter491 in self.part_vals: - oprot.writeString(iter491) - oprot.writeListEnd() - oprot.writeFieldEnd() - if self.max_parts is not None: - oprot.writeFieldBegin('max_parts', TType.I16, 4) - oprot.writeI16(self.max_parts) + oprot.writeStructBegin('partition_name_to_vals_args') + if self.part_name is not None: + oprot.writeFieldBegin('part_name', TType.STRING, 1) + oprot.writeString(self.part_name) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -13269,24 +15964,21 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class get_partitions_ps_result: +class partition_name_to_vals_result: """ Attributes: - success - o1 - - o2 """ thrift_spec = ( - (0, TType.LIST, 'success', (TType.STRUCT,(Partition, Partition.thrift_spec)), None, ), # 0 + (0, TType.LIST, 'success', (TType.STRING,None), None, ), # 0 (1, TType.STRUCT, 'o1', (MetaException, MetaException.thrift_spec), None, ), # 1 - (2, TType.STRUCT, 'o2', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 2 ) - def __init__(self, success=None, o1=None, o2=None,): + def __init__(self, success=None, o1=None,): self.success = success self.o1 = o1 - self.o2 = o2 def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -13300,11 +15992,10 @@ def read(self, iprot): if fid == 0: if ftype == TType.LIST: self.success = [] - (_etype495, _size492) = iprot.readListBegin() - for _i496 in xrange(_size492): - _elem497 = Partition() - _elem497.read(iprot) - self.success.append(_elem497) + (_etype621, _size618) = iprot.readListBegin() + for _i622 in xrange(_size618): + _elem623 = iprot.readString(); + self.success.append(_elem623) iprot.readListEnd() else: iprot.skip(ftype) @@ -13314,12 +16005,6 @@ def read(self, iprot): self.o1.read(iprot) else: iprot.skip(ftype) - elif fid == 2: - if ftype == TType.STRUCT: - self.o2 = NoSuchObjectException() - self.o2.read(iprot) - else: - iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() @@ -13329,22 +16014,18 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('get_partitions_ps_result') + oprot.writeStructBegin('partition_name_to_vals_result') if self.success is not None: oprot.writeFieldBegin('success', TType.LIST, 0) - oprot.writeListBegin(TType.STRUCT, len(self.success)) - for iter498 in self.success: - iter498.write(oprot) + oprot.writeListBegin(TType.STRING, len(self.success)) + for iter624 in self.success: + oprot.writeString(iter624) oprot.writeListEnd() oprot.writeFieldEnd() if self.o1 is not None: oprot.writeFieldBegin('o1', TType.STRUCT, 1) self.o1.write(oprot) oprot.writeFieldEnd() - if self.o2 is not None: - oprot.writeFieldBegin('o2', TType.STRUCT, 2) - self.o2.write(oprot) - oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -13363,34 +16044,19 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class get_partitions_ps_with_auth_args: +class partition_name_to_spec_args: """ Attributes: - - db_name - - tbl_name - - part_vals - - max_parts - - user_name - - group_names + - part_name """ thrift_spec = ( None, # 0 - (1, TType.STRING, 'db_name', None, None, ), # 1 - (2, TType.STRING, 'tbl_name', None, None, ), # 2 - (3, TType.LIST, 'part_vals', (TType.STRING,None), None, ), # 3 - (4, TType.I16, 'max_parts', None, -1, ), # 4 - (5, TType.STRING, 'user_name', None, None, ), # 5 - (6, TType.LIST, 'group_names', (TType.STRING,None), None, ), # 6 + (1, TType.STRING, 'part_name', None, None, ), # 1 ) - def __init__(self, db_name=None, tbl_name=None, part_vals=None, max_parts=thrift_spec[4][4], user_name=None, group_names=None,): - self.db_name = db_name - self.tbl_name = tbl_name - self.part_vals = part_vals - self.max_parts = max_parts - self.user_name = user_name - self.group_names = group_names + def __init__(self, part_name=None,): + self.part_name = part_name def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -13403,42 +16069,7 @@ def read(self, iprot): break if fid == 1: if ftype == TType.STRING: - self.db_name = iprot.readString(); - else: - iprot.skip(ftype) - elif fid == 2: - if ftype == TType.STRING: - self.tbl_name = iprot.readString(); - else: - iprot.skip(ftype) - elif fid == 3: - if ftype == TType.LIST: - self.part_vals = [] - (_etype502, _size499) = iprot.readListBegin() - for _i503 in xrange(_size499): - _elem504 = iprot.readString(); - self.part_vals.append(_elem504) - iprot.readListEnd() - else: - iprot.skip(ftype) - elif fid == 4: - if ftype == TType.I16: - self.max_parts = iprot.readI16(); - else: - iprot.skip(ftype) - elif fid == 5: - if ftype == TType.STRING: - self.user_name = iprot.readString(); - else: - iprot.skip(ftype) - elif fid == 6: - if ftype == TType.LIST: - self.group_names = [] - (_etype508, _size505) = iprot.readListBegin() - for _i509 in xrange(_size505): - _elem510 = iprot.readString(); - self.group_names.append(_elem510) - iprot.readListEnd() + self.part_name = iprot.readString(); else: iprot.skip(ftype) else: @@ -13450,36 +16081,10 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('get_partitions_ps_with_auth_args') - if self.db_name is not None: - oprot.writeFieldBegin('db_name', TType.STRING, 1) - oprot.writeString(self.db_name) - oprot.writeFieldEnd() - if self.tbl_name is not None: - oprot.writeFieldBegin('tbl_name', TType.STRING, 2) - oprot.writeString(self.tbl_name) - oprot.writeFieldEnd() - if self.part_vals is not None: - oprot.writeFieldBegin('part_vals', TType.LIST, 3) - oprot.writeListBegin(TType.STRING, len(self.part_vals)) - for iter511 in self.part_vals: - oprot.writeString(iter511) - oprot.writeListEnd() - oprot.writeFieldEnd() - if self.max_parts is not None: - oprot.writeFieldBegin('max_parts', TType.I16, 4) - oprot.writeI16(self.max_parts) - oprot.writeFieldEnd() - if self.user_name is not None: - oprot.writeFieldBegin('user_name', TType.STRING, 5) - oprot.writeString(self.user_name) - oprot.writeFieldEnd() - if self.group_names is not None: - oprot.writeFieldBegin('group_names', TType.LIST, 6) - oprot.writeListBegin(TType.STRING, len(self.group_names)) - for iter512 in self.group_names: - oprot.writeString(iter512) - oprot.writeListEnd() + oprot.writeStructBegin('partition_name_to_spec_args') + if self.part_name is not None: + oprot.writeFieldBegin('part_name', TType.STRING, 1) + oprot.writeString(self.part_name) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -13499,24 +16104,21 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class get_partitions_ps_with_auth_result: +class partition_name_to_spec_result: """ Attributes: - success - o1 - - o2 """ thrift_spec = ( - (0, TType.LIST, 'success', (TType.STRUCT,(Partition, Partition.thrift_spec)), None, ), # 0 - (1, TType.STRUCT, 'o1', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 1 - (2, TType.STRUCT, 'o2', (MetaException, MetaException.thrift_spec), None, ), # 2 + (0, TType.MAP, 'success', (TType.STRING,None,TType.STRING,None), None, ), # 0 + (1, TType.STRUCT, 'o1', (MetaException, MetaException.thrift_spec), None, ), # 1 ) - def __init__(self, success=None, o1=None, o2=None,): + def __init__(self, success=None, o1=None,): self.success = success self.o1 = o1 - self.o2 = o2 def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -13528,28 +16130,22 @@ def read(self, iprot): if ftype == TType.STOP: break if fid == 0: - if ftype == TType.LIST: - self.success = [] - (_etype516, _size513) = iprot.readListBegin() - for _i517 in xrange(_size513): - _elem518 = Partition() - _elem518.read(iprot) - self.success.append(_elem518) - iprot.readListEnd() + if ftype == TType.MAP: + self.success = {} + (_ktype626, _vtype627, _size625 ) = iprot.readMapBegin() + for _i629 in xrange(_size625): + _key630 = iprot.readString(); + _val631 = iprot.readString(); + self.success[_key630] = _val631 + iprot.readMapEnd() else: iprot.skip(ftype) elif fid == 1: if ftype == TType.STRUCT: - self.o1 = NoSuchObjectException() + self.o1 = MetaException() self.o1.read(iprot) else: iprot.skip(ftype) - elif fid == 2: - if ftype == TType.STRUCT: - self.o2 = MetaException() - self.o2.read(iprot) - else: - iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() @@ -13559,22 +16155,19 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('get_partitions_ps_with_auth_result') + oprot.writeStructBegin('partition_name_to_spec_result') if self.success is not None: - oprot.writeFieldBegin('success', TType.LIST, 0) - oprot.writeListBegin(TType.STRUCT, len(self.success)) - for iter519 in self.success: - iter519.write(oprot) - oprot.writeListEnd() + oprot.writeFieldBegin('success', TType.MAP, 0) + oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.success)) + for kiter632,viter633 in self.success.items(): + oprot.writeString(kiter632) + oprot.writeString(viter633) + oprot.writeMapEnd() oprot.writeFieldEnd() if self.o1 is not None: oprot.writeFieldBegin('o1', TType.STRUCT, 1) self.o1.write(oprot) oprot.writeFieldEnd() - if self.o2 is not None: - oprot.writeFieldBegin('o2', TType.STRUCT, 2) - self.o2.write(oprot) - oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -13593,28 +16186,28 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class get_partition_names_ps_args: +class markPartitionForEvent_args: """ Attributes: - db_name - tbl_name - part_vals - - max_parts + - eventType """ thrift_spec = ( None, # 0 (1, TType.STRING, 'db_name', None, None, ), # 1 (2, TType.STRING, 'tbl_name', None, None, ), # 2 - (3, TType.LIST, 'part_vals', (TType.STRING,None), None, ), # 3 - (4, TType.I16, 'max_parts', None, -1, ), # 4 + (3, TType.MAP, 'part_vals', (TType.STRING,None,TType.STRING,None), None, ), # 3 + (4, TType.I32, 'eventType', None, None, ), # 4 ) - def __init__(self, db_name=None, tbl_name=None, part_vals=None, max_parts=thrift_spec[4][4],): + def __init__(self, db_name=None, tbl_name=None, part_vals=None, eventType=None,): self.db_name = db_name self.tbl_name = tbl_name self.part_vals = part_vals - self.max_parts = max_parts + self.eventType = eventType def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -13636,18 +16229,19 @@ def read(self, iprot): else: iprot.skip(ftype) elif fid == 3: - if ftype == TType.LIST: - self.part_vals = [] - (_etype523, _size520) = iprot.readListBegin() - for _i524 in xrange(_size520): - _elem525 = iprot.readString(); - self.part_vals.append(_elem525) - iprot.readListEnd() + if ftype == TType.MAP: + self.part_vals = {} + (_ktype635, _vtype636, _size634 ) = iprot.readMapBegin() + for _i638 in xrange(_size634): + _key639 = iprot.readString(); + _val640 = iprot.readString(); + self.part_vals[_key639] = _val640 + iprot.readMapEnd() else: iprot.skip(ftype) elif fid == 4: - if ftype == TType.I16: - self.max_parts = iprot.readI16(); + if ftype == TType.I32: + self.eventType = iprot.readI32(); else: iprot.skip(ftype) else: @@ -13659,7 +16253,7 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('get_partition_names_ps_args') + oprot.writeStructBegin('markPartitionForEvent_args') if self.db_name is not None: oprot.writeFieldBegin('db_name', TType.STRING, 1) oprot.writeString(self.db_name) @@ -13669,15 +16263,16 @@ def write(self, oprot): oprot.writeString(self.tbl_name) oprot.writeFieldEnd() if self.part_vals is not None: - oprot.writeFieldBegin('part_vals', TType.LIST, 3) - oprot.writeListBegin(TType.STRING, len(self.part_vals)) - for iter526 in self.part_vals: - oprot.writeString(iter526) - oprot.writeListEnd() + oprot.writeFieldBegin('part_vals', TType.MAP, 3) + oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.part_vals)) + for kiter641,viter642 in self.part_vals.items(): + oprot.writeString(kiter641) + oprot.writeString(viter642) + oprot.writeMapEnd() oprot.writeFieldEnd() - if self.max_parts is not None: - oprot.writeFieldBegin('max_parts', TType.I16, 4) - oprot.writeI16(self.max_parts) + if self.eventType is not None: + oprot.writeFieldBegin('eventType', TType.I32, 4) + oprot.writeI32(self.eventType) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -13697,24 +16292,34 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class get_partition_names_ps_result: +class markPartitionForEvent_result: """ Attributes: - - success - o1 - o2 + - o3 + - o4 + - o5 + - o6 """ thrift_spec = ( - (0, TType.LIST, 'success', (TType.STRING,None), None, ), # 0 + None, # 0 (1, TType.STRUCT, 'o1', (MetaException, MetaException.thrift_spec), None, ), # 1 (2, TType.STRUCT, 'o2', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 2 + (3, TType.STRUCT, 'o3', (UnknownDBException, UnknownDBException.thrift_spec), None, ), # 3 + (4, TType.STRUCT, 'o4', (UnknownTableException, UnknownTableException.thrift_spec), None, ), # 4 + (5, TType.STRUCT, 'o5', (UnknownPartitionException, UnknownPartitionException.thrift_spec), None, ), # 5 + (6, TType.STRUCT, 'o6', (InvalidPartitionException, InvalidPartitionException.thrift_spec), None, ), # 6 ) - def __init__(self, success=None, o1=None, o2=None,): - self.success = success + def __init__(self, o1=None, o2=None, o3=None, o4=None, o5=None, o6=None,): self.o1 = o1 self.o2 = o2 + self.o3 = o3 + self.o4 = o4 + self.o5 = o5 + self.o6 = o6 def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -13725,17 +16330,7 @@ def read(self, iprot): (fname, ftype, fid) = iprot.readFieldBegin() if ftype == TType.STOP: break - if fid == 0: - if ftype == TType.LIST: - self.success = [] - (_etype530, _size527) = iprot.readListBegin() - for _i531 in xrange(_size527): - _elem532 = iprot.readString(); - self.success.append(_elem532) - iprot.readListEnd() - else: - iprot.skip(ftype) - elif fid == 1: + if fid == 1: if ftype == TType.STRUCT: self.o1 = MetaException() self.o1.read(iprot) @@ -13747,6 +16342,30 @@ def read(self, iprot): self.o2.read(iprot) else: iprot.skip(ftype) + elif fid == 3: + if ftype == TType.STRUCT: + self.o3 = UnknownDBException() + self.o3.read(iprot) + else: + iprot.skip(ftype) + elif fid == 4: + if ftype == TType.STRUCT: + self.o4 = UnknownTableException() + self.o4.read(iprot) + else: + iprot.skip(ftype) + elif fid == 5: + if ftype == TType.STRUCT: + self.o5 = UnknownPartitionException() + self.o5.read(iprot) + else: + iprot.skip(ftype) + elif fid == 6: + if ftype == TType.STRUCT: + self.o6 = InvalidPartitionException() + self.o6.read(iprot) + else: + iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() @@ -13756,14 +16375,7 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('get_partition_names_ps_result') - if self.success is not None: - oprot.writeFieldBegin('success', TType.LIST, 0) - oprot.writeListBegin(TType.STRING, len(self.success)) - for iter533 in self.success: - oprot.writeString(iter533) - oprot.writeListEnd() - oprot.writeFieldEnd() + oprot.writeStructBegin('markPartitionForEvent_result') if self.o1 is not None: oprot.writeFieldBegin('o1', TType.STRUCT, 1) self.o1.write(oprot) @@ -13772,6 +16384,22 @@ def write(self, oprot): oprot.writeFieldBegin('o2', TType.STRUCT, 2) self.o2.write(oprot) oprot.writeFieldEnd() + if self.o3 is not None: + oprot.writeFieldBegin('o3', TType.STRUCT, 3) + self.o3.write(oprot) + oprot.writeFieldEnd() + if self.o4 is not None: + oprot.writeFieldBegin('o4', TType.STRUCT, 4) + self.o4.write(oprot) + oprot.writeFieldEnd() + if self.o5 is not None: + oprot.writeFieldBegin('o5', TType.STRUCT, 5) + self.o5.write(oprot) + oprot.writeFieldEnd() + if self.o6 is not None: + oprot.writeFieldBegin('o6', TType.STRUCT, 6) + self.o6.write(oprot) + oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -13790,28 +16418,28 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class get_partitions_by_filter_args: +class isPartitionMarkedForEvent_args: """ Attributes: - db_name - tbl_name - - filter - - max_parts + - part_vals + - eventType """ thrift_spec = ( None, # 0 (1, TType.STRING, 'db_name', None, None, ), # 1 (2, TType.STRING, 'tbl_name', None, None, ), # 2 - (3, TType.STRING, 'filter', None, None, ), # 3 - (4, TType.I16, 'max_parts', None, -1, ), # 4 + (3, TType.MAP, 'part_vals', (TType.STRING,None,TType.STRING,None), None, ), # 3 + (4, TType.I32, 'eventType', None, None, ), # 4 ) - def __init__(self, db_name=None, tbl_name=None, filter=None, max_parts=thrift_spec[4][4],): + def __init__(self, db_name=None, tbl_name=None, part_vals=None, eventType=None,): self.db_name = db_name self.tbl_name = tbl_name - self.filter = filter - self.max_parts = max_parts + self.part_vals = part_vals + self.eventType = eventType def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -13833,13 +16461,19 @@ def read(self, iprot): else: iprot.skip(ftype) elif fid == 3: - if ftype == TType.STRING: - self.filter = iprot.readString(); + if ftype == TType.MAP: + self.part_vals = {} + (_ktype644, _vtype645, _size643 ) = iprot.readMapBegin() + for _i647 in xrange(_size643): + _key648 = iprot.readString(); + _val649 = iprot.readString(); + self.part_vals[_key648] = _val649 + iprot.readMapEnd() else: iprot.skip(ftype) elif fid == 4: - if ftype == TType.I16: - self.max_parts = iprot.readI16(); + if ftype == TType.I32: + self.eventType = iprot.readI32(); else: iprot.skip(ftype) else: @@ -13851,7 +16485,7 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('get_partitions_by_filter_args') + oprot.writeStructBegin('isPartitionMarkedForEvent_args') if self.db_name is not None: oprot.writeFieldBegin('db_name', TType.STRING, 1) oprot.writeString(self.db_name) @@ -13860,13 +16494,17 @@ def write(self, oprot): oprot.writeFieldBegin('tbl_name', TType.STRING, 2) oprot.writeString(self.tbl_name) oprot.writeFieldEnd() - if self.filter is not None: - oprot.writeFieldBegin('filter', TType.STRING, 3) - oprot.writeString(self.filter) + if self.part_vals is not None: + oprot.writeFieldBegin('part_vals', TType.MAP, 3) + oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.part_vals)) + for kiter650,viter651 in self.part_vals.items(): + oprot.writeString(kiter650) + oprot.writeString(viter651) + oprot.writeMapEnd() oprot.writeFieldEnd() - if self.max_parts is not None: - oprot.writeFieldBegin('max_parts', TType.I16, 4) - oprot.writeI16(self.max_parts) + if self.eventType is not None: + oprot.writeFieldBegin('eventType', TType.I32, 4) + oprot.writeI32(self.eventType) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -13886,24 +16524,36 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class get_partitions_by_filter_result: +class isPartitionMarkedForEvent_result: """ Attributes: - success - o1 - o2 + - o3 + - o4 + - o5 + - o6 """ thrift_spec = ( - (0, TType.LIST, 'success', (TType.STRUCT,(Partition, Partition.thrift_spec)), None, ), # 0 + (0, TType.BOOL, 'success', None, None, ), # 0 (1, TType.STRUCT, 'o1', (MetaException, MetaException.thrift_spec), None, ), # 1 (2, TType.STRUCT, 'o2', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 2 + (3, TType.STRUCT, 'o3', (UnknownDBException, UnknownDBException.thrift_spec), None, ), # 3 + (4, TType.STRUCT, 'o4', (UnknownTableException, UnknownTableException.thrift_spec), None, ), # 4 + (5, TType.STRUCT, 'o5', (UnknownPartitionException, UnknownPartitionException.thrift_spec), None, ), # 5 + (6, TType.STRUCT, 'o6', (InvalidPartitionException, InvalidPartitionException.thrift_spec), None, ), # 6 ) - def __init__(self, success=None, o1=None, o2=None,): + def __init__(self, success=None, o1=None, o2=None, o3=None, o4=None, o5=None, o6=None,): self.success = success self.o1 = o1 self.o2 = o2 + self.o3 = o3 + self.o4 = o4 + self.o5 = o5 + self.o6 = o6 def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -13915,14 +16565,8 @@ def read(self, iprot): if ftype == TType.STOP: break if fid == 0: - if ftype == TType.LIST: - self.success = [] - (_etype537, _size534) = iprot.readListBegin() - for _i538 in xrange(_size534): - _elem539 = Partition() - _elem539.read(iprot) - self.success.append(_elem539) - iprot.readListEnd() + if ftype == TType.BOOL: + self.success = iprot.readBool(); else: iprot.skip(ftype) elif fid == 1: @@ -13937,6 +16581,30 @@ def read(self, iprot): self.o2.read(iprot) else: iprot.skip(ftype) + elif fid == 3: + if ftype == TType.STRUCT: + self.o3 = UnknownDBException() + self.o3.read(iprot) + else: + iprot.skip(ftype) + elif fid == 4: + if ftype == TType.STRUCT: + self.o4 = UnknownTableException() + self.o4.read(iprot) + else: + iprot.skip(ftype) + elif fid == 5: + if ftype == TType.STRUCT: + self.o5 = UnknownPartitionException() + self.o5.read(iprot) + else: + iprot.skip(ftype) + elif fid == 6: + if ftype == TType.STRUCT: + self.o6 = InvalidPartitionException() + self.o6.read(iprot) + else: + iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() @@ -13946,13 +16614,10 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('get_partitions_by_filter_result') + oprot.writeStructBegin('isPartitionMarkedForEvent_result') if self.success is not None: - oprot.writeFieldBegin('success', TType.LIST, 0) - oprot.writeListBegin(TType.STRUCT, len(self.success)) - for iter540 in self.success: - iter540.write(oprot) - oprot.writeListEnd() + oprot.writeFieldBegin('success', TType.BOOL, 0) + oprot.writeBool(self.success) oprot.writeFieldEnd() if self.o1 is not None: oprot.writeFieldBegin('o1', TType.STRUCT, 1) @@ -13962,6 +16627,22 @@ def write(self, oprot): oprot.writeFieldBegin('o2', TType.STRUCT, 2) self.o2.write(oprot) oprot.writeFieldEnd() + if self.o3 is not None: + oprot.writeFieldBegin('o3', TType.STRUCT, 3) + self.o3.write(oprot) + oprot.writeFieldEnd() + if self.o4 is not None: + oprot.writeFieldBegin('o4', TType.STRUCT, 4) + self.o4.write(oprot) + oprot.writeFieldEnd() + if self.o5 is not None: + oprot.writeFieldBegin('o5', TType.STRUCT, 5) + self.o5.write(oprot) + oprot.writeFieldEnd() + if self.o6 is not None: + oprot.writeFieldBegin('o6', TType.STRUCT, 6) + self.o6.write(oprot) + oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -13980,19 +16661,22 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class get_partitions_by_expr_args: +class add_index_args: """ Attributes: - - req + - new_index + - index_table """ thrift_spec = ( None, # 0 - (1, TType.STRUCT, 'req', (PartitionsByExprRequest, PartitionsByExprRequest.thrift_spec), None, ), # 1 + (1, TType.STRUCT, 'new_index', (Index, Index.thrift_spec), None, ), # 1 + (2, TType.STRUCT, 'index_table', (Table, Table.thrift_spec), None, ), # 2 ) - def __init__(self, req=None,): - self.req = req + def __init__(self, new_index=None, index_table=None,): + self.new_index = new_index + self.index_table = index_table def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -14005,8 +16689,14 @@ def read(self, iprot): break if fid == 1: if ftype == TType.STRUCT: - self.req = PartitionsByExprRequest() - self.req.read(iprot) + self.new_index = Index() + self.new_index.read(iprot) + else: + iprot.skip(ftype) + elif fid == 2: + if ftype == TType.STRUCT: + self.index_table = Table() + self.index_table.read(iprot) else: iprot.skip(ftype) else: @@ -14018,10 +16708,14 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('get_partitions_by_expr_args') - if self.req is not None: - oprot.writeFieldBegin('req', TType.STRUCT, 1) - self.req.write(oprot) + oprot.writeStructBegin('add_index_args') + if self.new_index is not None: + oprot.writeFieldBegin('new_index', TType.STRUCT, 1) + self.new_index.write(oprot) + oprot.writeFieldEnd() + if self.index_table is not None: + oprot.writeFieldBegin('index_table', TType.STRUCT, 2) + self.index_table.write(oprot) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -14041,24 +16735,27 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class get_partitions_by_expr_result: +class add_index_result: """ Attributes: - success - o1 - o2 + - o3 """ thrift_spec = ( - (0, TType.STRUCT, 'success', (PartitionsByExprResult, PartitionsByExprResult.thrift_spec), None, ), # 0 - (1, TType.STRUCT, 'o1', (MetaException, MetaException.thrift_spec), None, ), # 1 - (2, TType.STRUCT, 'o2', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 2 + (0, TType.STRUCT, 'success', (Index, Index.thrift_spec), None, ), # 0 + (1, TType.STRUCT, 'o1', (InvalidObjectException, InvalidObjectException.thrift_spec), None, ), # 1 + (2, TType.STRUCT, 'o2', (AlreadyExistsException, AlreadyExistsException.thrift_spec), None, ), # 2 + (3, TType.STRUCT, 'o3', (MetaException, MetaException.thrift_spec), None, ), # 3 ) - def __init__(self, success=None, o1=None, o2=None,): + def __init__(self, success=None, o1=None, o2=None, o3=None,): self.success = success self.o1 = o1 self.o2 = o2 + self.o3 = o3 def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -14071,20 +16768,26 @@ def read(self, iprot): break if fid == 0: if ftype == TType.STRUCT: - self.success = PartitionsByExprResult() + self.success = Index() self.success.read(iprot) else: iprot.skip(ftype) elif fid == 1: if ftype == TType.STRUCT: - self.o1 = MetaException() + self.o1 = InvalidObjectException() self.o1.read(iprot) else: iprot.skip(ftype) - elif fid == 2: + elif fid == 2: + if ftype == TType.STRUCT: + self.o2 = AlreadyExistsException() + self.o2.read(iprot) + else: + iprot.skip(ftype) + elif fid == 3: if ftype == TType.STRUCT: - self.o2 = NoSuchObjectException() - self.o2.read(iprot) + self.o3 = MetaException() + self.o3.read(iprot) else: iprot.skip(ftype) else: @@ -14096,7 +16799,7 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('get_partitions_by_expr_result') + oprot.writeStructBegin('add_index_result') if self.success is not None: oprot.writeFieldBegin('success', TType.STRUCT, 0) self.success.write(oprot) @@ -14109,6 +16812,10 @@ def write(self, oprot): oprot.writeFieldBegin('o2', TType.STRUCT, 2) self.o2.write(oprot) oprot.writeFieldEnd() + if self.o3 is not None: + oprot.writeFieldBegin('o3', TType.STRUCT, 3) + self.o3.write(oprot) + oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -14127,25 +16834,28 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class get_partitions_by_names_args: +class alter_index_args: """ Attributes: - - db_name - - tbl_name - - names + - dbname + - base_tbl_name + - idx_name + - new_idx """ thrift_spec = ( None, # 0 - (1, TType.STRING, 'db_name', None, None, ), # 1 - (2, TType.STRING, 'tbl_name', None, None, ), # 2 - (3, TType.LIST, 'names', (TType.STRING,None), None, ), # 3 + (1, TType.STRING, 'dbname', None, None, ), # 1 + (2, TType.STRING, 'base_tbl_name', None, None, ), # 2 + (3, TType.STRING, 'idx_name', None, None, ), # 3 + (4, TType.STRUCT, 'new_idx', (Index, Index.thrift_spec), None, ), # 4 ) - def __init__(self, db_name=None, tbl_name=None, names=None,): - self.db_name = db_name - self.tbl_name = tbl_name - self.names = names + def __init__(self, dbname=None, base_tbl_name=None, idx_name=None, new_idx=None,): + self.dbname = dbname + self.base_tbl_name = base_tbl_name + self.idx_name = idx_name + self.new_idx = new_idx def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -14158,22 +16868,23 @@ def read(self, iprot): break if fid == 1: if ftype == TType.STRING: - self.db_name = iprot.readString(); + self.dbname = iprot.readString(); else: iprot.skip(ftype) elif fid == 2: if ftype == TType.STRING: - self.tbl_name = iprot.readString(); + self.base_tbl_name = iprot.readString(); else: iprot.skip(ftype) elif fid == 3: - if ftype == TType.LIST: - self.names = [] - (_etype544, _size541) = iprot.readListBegin() - for _i545 in xrange(_size541): - _elem546 = iprot.readString(); - self.names.append(_elem546) - iprot.readListEnd() + if ftype == TType.STRING: + self.idx_name = iprot.readString(); + else: + iprot.skip(ftype) + elif fid == 4: + if ftype == TType.STRUCT: + self.new_idx = Index() + self.new_idx.read(iprot) else: iprot.skip(ftype) else: @@ -14185,21 +16896,22 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('get_partitions_by_names_args') - if self.db_name is not None: - oprot.writeFieldBegin('db_name', TType.STRING, 1) - oprot.writeString(self.db_name) + oprot.writeStructBegin('alter_index_args') + if self.dbname is not None: + oprot.writeFieldBegin('dbname', TType.STRING, 1) + oprot.writeString(self.dbname) oprot.writeFieldEnd() - if self.tbl_name is not None: - oprot.writeFieldBegin('tbl_name', TType.STRING, 2) - oprot.writeString(self.tbl_name) + if self.base_tbl_name is not None: + oprot.writeFieldBegin('base_tbl_name', TType.STRING, 2) + oprot.writeString(self.base_tbl_name) oprot.writeFieldEnd() - if self.names is not None: - oprot.writeFieldBegin('names', TType.LIST, 3) - oprot.writeListBegin(TType.STRING, len(self.names)) - for iter547 in self.names: - oprot.writeString(iter547) - oprot.writeListEnd() + if self.idx_name is not None: + oprot.writeFieldBegin('idx_name', TType.STRING, 3) + oprot.writeString(self.idx_name) + oprot.writeFieldEnd() + if self.new_idx is not None: + oprot.writeFieldBegin('new_idx', TType.STRUCT, 4) + self.new_idx.write(oprot) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -14219,22 +16931,20 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class get_partitions_by_names_result: +class alter_index_result: """ Attributes: - - success - o1 - o2 """ thrift_spec = ( - (0, TType.LIST, 'success', (TType.STRUCT,(Partition, Partition.thrift_spec)), None, ), # 0 - (1, TType.STRUCT, 'o1', (MetaException, MetaException.thrift_spec), None, ), # 1 - (2, TType.STRUCT, 'o2', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 2 + None, # 0 + (1, TType.STRUCT, 'o1', (InvalidOperationException, InvalidOperationException.thrift_spec), None, ), # 1 + (2, TType.STRUCT, 'o2', (MetaException, MetaException.thrift_spec), None, ), # 2 ) - def __init__(self, success=None, o1=None, o2=None,): - self.success = success + def __init__(self, o1=None, o2=None,): self.o1 = o1 self.o2 = o2 @@ -14247,26 +16957,15 @@ def read(self, iprot): (fname, ftype, fid) = iprot.readFieldBegin() if ftype == TType.STOP: break - if fid == 0: - if ftype == TType.LIST: - self.success = [] - (_etype551, _size548) = iprot.readListBegin() - for _i552 in xrange(_size548): - _elem553 = Partition() - _elem553.read(iprot) - self.success.append(_elem553) - iprot.readListEnd() - else: - iprot.skip(ftype) - elif fid == 1: + if fid == 1: if ftype == TType.STRUCT: - self.o1 = MetaException() + self.o1 = InvalidOperationException() self.o1.read(iprot) else: iprot.skip(ftype) elif fid == 2: if ftype == TType.STRUCT: - self.o2 = NoSuchObjectException() + self.o2 = MetaException() self.o2.read(iprot) else: iprot.skip(ftype) @@ -14279,14 +16978,7 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('get_partitions_by_names_result') - if self.success is not None: - oprot.writeFieldBegin('success', TType.LIST, 0) - oprot.writeListBegin(TType.STRUCT, len(self.success)) - for iter554 in self.success: - iter554.write(oprot) - oprot.writeListEnd() - oprot.writeFieldEnd() + oprot.writeStructBegin('alter_index_result') if self.o1 is not None: oprot.writeFieldBegin('o1', TType.STRUCT, 1) self.o1.write(oprot) @@ -14313,25 +17005,28 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class alter_partition_args: +class drop_index_by_name_args: """ Attributes: - db_name - tbl_name - - new_part + - index_name + - deleteData """ thrift_spec = ( None, # 0 (1, TType.STRING, 'db_name', None, None, ), # 1 (2, TType.STRING, 'tbl_name', None, None, ), # 2 - (3, TType.STRUCT, 'new_part', (Partition, Partition.thrift_spec), None, ), # 3 + (3, TType.STRING, 'index_name', None, None, ), # 3 + (4, TType.BOOL, 'deleteData', None, None, ), # 4 ) - def __init__(self, db_name=None, tbl_name=None, new_part=None,): + def __init__(self, db_name=None, tbl_name=None, index_name=None, deleteData=None,): self.db_name = db_name self.tbl_name = tbl_name - self.new_part = new_part + self.index_name = index_name + self.deleteData = deleteData def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -14353,9 +17048,13 @@ def read(self, iprot): else: iprot.skip(ftype) elif fid == 3: - if ftype == TType.STRUCT: - self.new_part = Partition() - self.new_part.read(iprot) + if ftype == TType.STRING: + self.index_name = iprot.readString(); + else: + iprot.skip(ftype) + elif fid == 4: + if ftype == TType.BOOL: + self.deleteData = iprot.readBool(); else: iprot.skip(ftype) else: @@ -14367,7 +17066,7 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('alter_partition_args') + oprot.writeStructBegin('drop_index_by_name_args') if self.db_name is not None: oprot.writeFieldBegin('db_name', TType.STRING, 1) oprot.writeString(self.db_name) @@ -14376,9 +17075,13 @@ def write(self, oprot): oprot.writeFieldBegin('tbl_name', TType.STRING, 2) oprot.writeString(self.tbl_name) oprot.writeFieldEnd() - if self.new_part is not None: - oprot.writeFieldBegin('new_part', TType.STRUCT, 3) - self.new_part.write(oprot) + if self.index_name is not None: + oprot.writeFieldBegin('index_name', TType.STRING, 3) + oprot.writeString(self.index_name) + oprot.writeFieldEnd() + if self.deleteData is not None: + oprot.writeFieldBegin('deleteData', TType.BOOL, 4) + oprot.writeBool(self.deleteData) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -14398,20 +17101,22 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class alter_partition_result: +class drop_index_by_name_result: """ Attributes: + - success - o1 - o2 """ thrift_spec = ( - None, # 0 - (1, TType.STRUCT, 'o1', (InvalidOperationException, InvalidOperationException.thrift_spec), None, ), # 1 + (0, TType.BOOL, 'success', None, None, ), # 0 + (1, TType.STRUCT, 'o1', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 1 (2, TType.STRUCT, 'o2', (MetaException, MetaException.thrift_spec), None, ), # 2 ) - def __init__(self, o1=None, o2=None,): + def __init__(self, success=None, o1=None, o2=None,): + self.success = success self.o1 = o1 self.o2 = o2 @@ -14424,9 +17129,14 @@ def read(self, iprot): (fname, ftype, fid) = iprot.readFieldBegin() if ftype == TType.STOP: break - if fid == 1: + if fid == 0: + if ftype == TType.BOOL: + self.success = iprot.readBool(); + else: + iprot.skip(ftype) + elif fid == 1: if ftype == TType.STRUCT: - self.o1 = InvalidOperationException() + self.o1 = NoSuchObjectException() self.o1.read(iprot) else: iprot.skip(ftype) @@ -14445,7 +17155,11 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('alter_partition_result') + oprot.writeStructBegin('drop_index_by_name_result') + if self.success is not None: + oprot.writeFieldBegin('success', TType.BOOL, 0) + oprot.writeBool(self.success) + oprot.writeFieldEnd() if self.o1 is not None: oprot.writeFieldBegin('o1', TType.STRUCT, 1) self.o1.write(oprot) @@ -14472,25 +17186,25 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class alter_partitions_args: +class get_index_by_name_args: """ Attributes: - db_name - tbl_name - - new_parts + - index_name """ thrift_spec = ( None, # 0 (1, TType.STRING, 'db_name', None, None, ), # 1 (2, TType.STRING, 'tbl_name', None, None, ), # 2 - (3, TType.LIST, 'new_parts', (TType.STRUCT,(Partition, Partition.thrift_spec)), None, ), # 3 + (3, TType.STRING, 'index_name', None, None, ), # 3 ) - def __init__(self, db_name=None, tbl_name=None, new_parts=None,): + def __init__(self, db_name=None, tbl_name=None, index_name=None,): self.db_name = db_name self.tbl_name = tbl_name - self.new_parts = new_parts + self.index_name = index_name def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -14512,14 +17226,8 @@ def read(self, iprot): else: iprot.skip(ftype) elif fid == 3: - if ftype == TType.LIST: - self.new_parts = [] - (_etype558, _size555) = iprot.readListBegin() - for _i559 in xrange(_size555): - _elem560 = Partition() - _elem560.read(iprot) - self.new_parts.append(_elem560) - iprot.readListEnd() + if ftype == TType.STRING: + self.index_name = iprot.readString(); else: iprot.skip(ftype) else: @@ -14531,7 +17239,7 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('alter_partitions_args') + oprot.writeStructBegin('get_index_by_name_args') if self.db_name is not None: oprot.writeFieldBegin('db_name', TType.STRING, 1) oprot.writeString(self.db_name) @@ -14540,12 +17248,9 @@ def write(self, oprot): oprot.writeFieldBegin('tbl_name', TType.STRING, 2) oprot.writeString(self.tbl_name) oprot.writeFieldEnd() - if self.new_parts is not None: - oprot.writeFieldBegin('new_parts', TType.LIST, 3) - oprot.writeListBegin(TType.STRUCT, len(self.new_parts)) - for iter561 in self.new_parts: - iter561.write(oprot) - oprot.writeListEnd() + if self.index_name is not None: + oprot.writeFieldBegin('index_name', TType.STRING, 3) + oprot.writeString(self.index_name) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -14565,20 +17270,22 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class alter_partitions_result: +class get_index_by_name_result: """ Attributes: + - success - o1 - o2 """ thrift_spec = ( - None, # 0 - (1, TType.STRUCT, 'o1', (InvalidOperationException, InvalidOperationException.thrift_spec), None, ), # 1 - (2, TType.STRUCT, 'o2', (MetaException, MetaException.thrift_spec), None, ), # 2 + (0, TType.STRUCT, 'success', (Index, Index.thrift_spec), None, ), # 0 + (1, TType.STRUCT, 'o1', (MetaException, MetaException.thrift_spec), None, ), # 1 + (2, TType.STRUCT, 'o2', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 2 ) - def __init__(self, o1=None, o2=None,): + def __init__(self, success=None, o1=None, o2=None,): + self.success = success self.o1 = o1 self.o2 = o2 @@ -14591,15 +17298,21 @@ def read(self, iprot): (fname, ftype, fid) = iprot.readFieldBegin() if ftype == TType.STOP: break - if fid == 1: + if fid == 0: if ftype == TType.STRUCT: - self.o1 = InvalidOperationException() + self.success = Index() + self.success.read(iprot) + else: + iprot.skip(ftype) + elif fid == 1: + if ftype == TType.STRUCT: + self.o1 = MetaException() self.o1.read(iprot) else: iprot.skip(ftype) elif fid == 2: if ftype == TType.STRUCT: - self.o2 = MetaException() + self.o2 = NoSuchObjectException() self.o2.read(iprot) else: iprot.skip(ftype) @@ -14612,7 +17325,11 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('alter_partitions_result') + oprot.writeStructBegin('get_index_by_name_result') + if self.success is not None: + oprot.writeFieldBegin('success', TType.STRUCT, 0) + self.success.write(oprot) + oprot.writeFieldEnd() if self.o1 is not None: oprot.writeFieldBegin('o1', TType.STRUCT, 1) self.o1.write(oprot) @@ -14639,28 +17356,25 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class alter_partition_with_environment_context_args: +class get_indexes_args: """ Attributes: - db_name - tbl_name - - new_part - - environment_context + - max_indexes """ thrift_spec = ( None, # 0 (1, TType.STRING, 'db_name', None, None, ), # 1 (2, TType.STRING, 'tbl_name', None, None, ), # 2 - (3, TType.STRUCT, 'new_part', (Partition, Partition.thrift_spec), None, ), # 3 - (4, TType.STRUCT, 'environment_context', (EnvironmentContext, EnvironmentContext.thrift_spec), None, ), # 4 + (3, TType.I16, 'max_indexes', None, -1, ), # 3 ) - def __init__(self, db_name=None, tbl_name=None, new_part=None, environment_context=None,): + def __init__(self, db_name=None, tbl_name=None, max_indexes=thrift_spec[3][4],): self.db_name = db_name self.tbl_name = tbl_name - self.new_part = new_part - self.environment_context = environment_context + self.max_indexes = max_indexes def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -14678,19 +17392,12 @@ def read(self, iprot): iprot.skip(ftype) elif fid == 2: if ftype == TType.STRING: - self.tbl_name = iprot.readString(); - else: - iprot.skip(ftype) - elif fid == 3: - if ftype == TType.STRUCT: - self.new_part = Partition() - self.new_part.read(iprot) - else: - iprot.skip(ftype) - elif fid == 4: - if ftype == TType.STRUCT: - self.environment_context = EnvironmentContext() - self.environment_context.read(iprot) + self.tbl_name = iprot.readString(); + else: + iprot.skip(ftype) + elif fid == 3: + if ftype == TType.I16: + self.max_indexes = iprot.readI16(); else: iprot.skip(ftype) else: @@ -14702,7 +17409,7 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('alter_partition_with_environment_context_args') + oprot.writeStructBegin('get_indexes_args') if self.db_name is not None: oprot.writeFieldBegin('db_name', TType.STRING, 1) oprot.writeString(self.db_name) @@ -14711,13 +17418,9 @@ def write(self, oprot): oprot.writeFieldBegin('tbl_name', TType.STRING, 2) oprot.writeString(self.tbl_name) oprot.writeFieldEnd() - if self.new_part is not None: - oprot.writeFieldBegin('new_part', TType.STRUCT, 3) - self.new_part.write(oprot) - oprot.writeFieldEnd() - if self.environment_context is not None: - oprot.writeFieldBegin('environment_context', TType.STRUCT, 4) - self.environment_context.write(oprot) + if self.max_indexes is not None: + oprot.writeFieldBegin('max_indexes', TType.I16, 3) + oprot.writeI16(self.max_indexes) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -14737,20 +17440,22 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class alter_partition_with_environment_context_result: +class get_indexes_result: """ Attributes: + - success - o1 - o2 """ thrift_spec = ( - None, # 0 - (1, TType.STRUCT, 'o1', (InvalidOperationException, InvalidOperationException.thrift_spec), None, ), # 1 + (0, TType.LIST, 'success', (TType.STRUCT,(Index, Index.thrift_spec)), None, ), # 0 + (1, TType.STRUCT, 'o1', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 1 (2, TType.STRUCT, 'o2', (MetaException, MetaException.thrift_spec), None, ), # 2 ) - def __init__(self, o1=None, o2=None,): + def __init__(self, success=None, o1=None, o2=None,): + self.success = success self.o1 = o1 self.o2 = o2 @@ -14763,9 +17468,20 @@ def read(self, iprot): (fname, ftype, fid) = iprot.readFieldBegin() if ftype == TType.STOP: break - if fid == 1: + if fid == 0: + if ftype == TType.LIST: + self.success = [] + (_etype655, _size652) = iprot.readListBegin() + for _i656 in xrange(_size652): + _elem657 = Index() + _elem657.read(iprot) + self.success.append(_elem657) + iprot.readListEnd() + else: + iprot.skip(ftype) + elif fid == 1: if ftype == TType.STRUCT: - self.o1 = InvalidOperationException() + self.o1 = NoSuchObjectException() self.o1.read(iprot) else: iprot.skip(ftype) @@ -14784,7 +17500,14 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('alter_partition_with_environment_context_result') + oprot.writeStructBegin('get_indexes_result') + if self.success is not None: + oprot.writeFieldBegin('success', TType.LIST, 0) + oprot.writeListBegin(TType.STRUCT, len(self.success)) + for iter658 in self.success: + iter658.write(oprot) + oprot.writeListEnd() + oprot.writeFieldEnd() if self.o1 is not None: oprot.writeFieldBegin('o1', TType.STRUCT, 1) self.o1.write(oprot) @@ -14811,28 +17534,25 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class rename_partition_args: +class get_index_names_args: """ Attributes: - db_name - tbl_name - - part_vals - - new_part + - max_indexes """ thrift_spec = ( None, # 0 (1, TType.STRING, 'db_name', None, None, ), # 1 (2, TType.STRING, 'tbl_name', None, None, ), # 2 - (3, TType.LIST, 'part_vals', (TType.STRING,None), None, ), # 3 - (4, TType.STRUCT, 'new_part', (Partition, Partition.thrift_spec), None, ), # 4 + (3, TType.I16, 'max_indexes', None, -1, ), # 3 ) - def __init__(self, db_name=None, tbl_name=None, part_vals=None, new_part=None,): + def __init__(self, db_name=None, tbl_name=None, max_indexes=thrift_spec[3][4],): self.db_name = db_name self.tbl_name = tbl_name - self.part_vals = part_vals - self.new_part = new_part + self.max_indexes = max_indexes def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -14854,19 +17574,8 @@ def read(self, iprot): else: iprot.skip(ftype) elif fid == 3: - if ftype == TType.LIST: - self.part_vals = [] - (_etype565, _size562) = iprot.readListBegin() - for _i566 in xrange(_size562): - _elem567 = iprot.readString(); - self.part_vals.append(_elem567) - iprot.readListEnd() - else: - iprot.skip(ftype) - elif fid == 4: - if ftype == TType.STRUCT: - self.new_part = Partition() - self.new_part.read(iprot) + if ftype == TType.I16: + self.max_indexes = iprot.readI16(); else: iprot.skip(ftype) else: @@ -14878,7 +17587,7 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('rename_partition_args') + oprot.writeStructBegin('get_index_names_args') if self.db_name is not None: oprot.writeFieldBegin('db_name', TType.STRING, 1) oprot.writeString(self.db_name) @@ -14887,16 +17596,9 @@ def write(self, oprot): oprot.writeFieldBegin('tbl_name', TType.STRING, 2) oprot.writeString(self.tbl_name) oprot.writeFieldEnd() - if self.part_vals is not None: - oprot.writeFieldBegin('part_vals', TType.LIST, 3) - oprot.writeListBegin(TType.STRING, len(self.part_vals)) - for iter568 in self.part_vals: - oprot.writeString(iter568) - oprot.writeListEnd() - oprot.writeFieldEnd() - if self.new_part is not None: - oprot.writeFieldBegin('new_part', TType.STRUCT, 4) - self.new_part.write(oprot) + if self.max_indexes is not None: + oprot.writeFieldBegin('max_indexes', TType.I16, 3) + oprot.writeI16(self.max_indexes) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -14916,21 +17618,20 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class rename_partition_result: +class get_index_names_result: """ Attributes: - - o1 + - success - o2 """ thrift_spec = ( - None, # 0 - (1, TType.STRUCT, 'o1', (InvalidOperationException, InvalidOperationException.thrift_spec), None, ), # 1 - (2, TType.STRUCT, 'o2', (MetaException, MetaException.thrift_spec), None, ), # 2 + (0, TType.LIST, 'success', (TType.STRING,None), None, ), # 0 + (1, TType.STRUCT, 'o2', (MetaException, MetaException.thrift_spec), None, ), # 1 ) - def __init__(self, o1=None, o2=None,): - self.o1 = o1 + def __init__(self, success=None, o2=None,): + self.success = success self.o2 = o2 def read(self, iprot): @@ -14942,13 +17643,17 @@ def read(self, iprot): (fname, ftype, fid) = iprot.readFieldBegin() if ftype == TType.STOP: break - if fid == 1: - if ftype == TType.STRUCT: - self.o1 = InvalidOperationException() - self.o1.read(iprot) + if fid == 0: + if ftype == TType.LIST: + self.success = [] + (_etype662, _size659) = iprot.readListBegin() + for _i663 in xrange(_size659): + _elem664 = iprot.readString(); + self.success.append(_elem664) + iprot.readListEnd() else: iprot.skip(ftype) - elif fid == 2: + elif fid == 1: if ftype == TType.STRUCT: self.o2 = MetaException() self.o2.read(iprot) @@ -14963,13 +17668,16 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('rename_partition_result') - if self.o1 is not None: - oprot.writeFieldBegin('o1', TType.STRUCT, 1) - self.o1.write(oprot) + oprot.writeStructBegin('get_index_names_result') + if self.success is not None: + oprot.writeFieldBegin('success', TType.LIST, 0) + oprot.writeListBegin(TType.STRING, len(self.success)) + for iter665 in self.success: + oprot.writeString(iter665) + oprot.writeListEnd() oprot.writeFieldEnd() if self.o2 is not None: - oprot.writeFieldBegin('o2', TType.STRUCT, 2) + oprot.writeFieldBegin('o2', TType.STRUCT, 1) self.o2.write(oprot) oprot.writeFieldEnd() oprot.writeFieldStop() @@ -14990,22 +17698,19 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class partition_name_has_valid_characters_args: +class update_table_column_statistics_args: """ Attributes: - - part_vals - - throw_exception + - stats_obj """ thrift_spec = ( None, # 0 - (1, TType.LIST, 'part_vals', (TType.STRING,None), None, ), # 1 - (2, TType.BOOL, 'throw_exception', None, None, ), # 2 + (1, TType.STRUCT, 'stats_obj', (ColumnStatistics, ColumnStatistics.thrift_spec), None, ), # 1 ) - def __init__(self, part_vals=None, throw_exception=None,): - self.part_vals = part_vals - self.throw_exception = throw_exception + def __init__(self, stats_obj=None,): + self.stats_obj = stats_obj def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -15017,18 +17722,9 @@ def read(self, iprot): if ftype == TType.STOP: break if fid == 1: - if ftype == TType.LIST: - self.part_vals = [] - (_etype572, _size569) = iprot.readListBegin() - for _i573 in xrange(_size569): - _elem574 = iprot.readString(); - self.part_vals.append(_elem574) - iprot.readListEnd() - else: - iprot.skip(ftype) - elif fid == 2: - if ftype == TType.BOOL: - self.throw_exception = iprot.readBool(); + if ftype == TType.STRUCT: + self.stats_obj = ColumnStatistics() + self.stats_obj.read(iprot) else: iprot.skip(ftype) else: @@ -15040,17 +17736,10 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('partition_name_has_valid_characters_args') - if self.part_vals is not None: - oprot.writeFieldBegin('part_vals', TType.LIST, 1) - oprot.writeListBegin(TType.STRING, len(self.part_vals)) - for iter575 in self.part_vals: - oprot.writeString(iter575) - oprot.writeListEnd() - oprot.writeFieldEnd() - if self.throw_exception is not None: - oprot.writeFieldBegin('throw_exception', TType.BOOL, 2) - oprot.writeBool(self.throw_exception) + oprot.writeStructBegin('update_table_column_statistics_args') + if self.stats_obj is not None: + oprot.writeFieldBegin('stats_obj', TType.STRUCT, 1) + self.stats_obj.write(oprot) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -15070,21 +17759,30 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class partition_name_has_valid_characters_result: +class update_table_column_statistics_result: """ Attributes: - success - o1 + - o2 + - o3 + - o4 """ thrift_spec = ( (0, TType.BOOL, 'success', None, None, ), # 0 - (1, TType.STRUCT, 'o1', (MetaException, MetaException.thrift_spec), None, ), # 1 + (1, TType.STRUCT, 'o1', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 1 + (2, TType.STRUCT, 'o2', (InvalidObjectException, InvalidObjectException.thrift_spec), None, ), # 2 + (3, TType.STRUCT, 'o3', (MetaException, MetaException.thrift_spec), None, ), # 3 + (4, TType.STRUCT, 'o4', (InvalidInputException, InvalidInputException.thrift_spec), None, ), # 4 ) - def __init__(self, success=None, o1=None,): + def __init__(self, success=None, o1=None, o2=None, o3=None, o4=None,): self.success = success self.o1 = o1 + self.o2 = o2 + self.o3 = o3 + self.o4 = o4 def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -15102,10 +17800,28 @@ def read(self, iprot): iprot.skip(ftype) elif fid == 1: if ftype == TType.STRUCT: - self.o1 = MetaException() + self.o1 = NoSuchObjectException() self.o1.read(iprot) else: iprot.skip(ftype) + elif fid == 2: + if ftype == TType.STRUCT: + self.o2 = InvalidObjectException() + self.o2.read(iprot) + else: + iprot.skip(ftype) + elif fid == 3: + if ftype == TType.STRUCT: + self.o3 = MetaException() + self.o3.read(iprot) + else: + iprot.skip(ftype) + elif fid == 4: + if ftype == TType.STRUCT: + self.o4 = InvalidInputException() + self.o4.read(iprot) + else: + iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() @@ -15115,7 +17831,7 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('partition_name_has_valid_characters_result') + oprot.writeStructBegin('update_table_column_statistics_result') if self.success is not None: oprot.writeFieldBegin('success', TType.BOOL, 0) oprot.writeBool(self.success) @@ -15124,6 +17840,18 @@ def write(self, oprot): oprot.writeFieldBegin('o1', TType.STRUCT, 1) self.o1.write(oprot) oprot.writeFieldEnd() + if self.o2 is not None: + oprot.writeFieldBegin('o2', TType.STRUCT, 2) + self.o2.write(oprot) + oprot.writeFieldEnd() + if self.o3 is not None: + oprot.writeFieldBegin('o3', TType.STRUCT, 3) + self.o3.write(oprot) + oprot.writeFieldEnd() + if self.o4 is not None: + oprot.writeFieldBegin('o4', TType.STRUCT, 4) + self.o4.write(oprot) + oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -15142,22 +17870,19 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class get_config_value_args: +class update_partition_column_statistics_args: """ Attributes: - - name - - defaultValue + - stats_obj """ thrift_spec = ( None, # 0 - (1, TType.STRING, 'name', None, None, ), # 1 - (2, TType.STRING, 'defaultValue', None, None, ), # 2 + (1, TType.STRUCT, 'stats_obj', (ColumnStatistics, ColumnStatistics.thrift_spec), None, ), # 1 ) - def __init__(self, name=None, defaultValue=None,): - self.name = name - self.defaultValue = defaultValue + def __init__(self, stats_obj=None,): + self.stats_obj = stats_obj def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -15169,13 +17894,9 @@ def read(self, iprot): if ftype == TType.STOP: break if fid == 1: - if ftype == TType.STRING: - self.name = iprot.readString(); - else: - iprot.skip(ftype) - elif fid == 2: - if ftype == TType.STRING: - self.defaultValue = iprot.readString(); + if ftype == TType.STRUCT: + self.stats_obj = ColumnStatistics() + self.stats_obj.read(iprot) else: iprot.skip(ftype) else: @@ -15187,14 +17908,10 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('get_config_value_args') - if self.name is not None: - oprot.writeFieldBegin('name', TType.STRING, 1) - oprot.writeString(self.name) - oprot.writeFieldEnd() - if self.defaultValue is not None: - oprot.writeFieldBegin('defaultValue', TType.STRING, 2) - oprot.writeString(self.defaultValue) + oprot.writeStructBegin('update_partition_column_statistics_args') + if self.stats_obj is not None: + oprot.writeFieldBegin('stats_obj', TType.STRUCT, 1) + self.stats_obj.write(oprot) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -15214,21 +17931,30 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class get_config_value_result: +class update_partition_column_statistics_result: """ Attributes: - success - o1 + - o2 + - o3 + - o4 """ thrift_spec = ( - (0, TType.STRING, 'success', None, None, ), # 0 - (1, TType.STRUCT, 'o1', (ConfigValSecurityException, ConfigValSecurityException.thrift_spec), None, ), # 1 + (0, TType.BOOL, 'success', None, None, ), # 0 + (1, TType.STRUCT, 'o1', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 1 + (2, TType.STRUCT, 'o2', (InvalidObjectException, InvalidObjectException.thrift_spec), None, ), # 2 + (3, TType.STRUCT, 'o3', (MetaException, MetaException.thrift_spec), None, ), # 3 + (4, TType.STRUCT, 'o4', (InvalidInputException, InvalidInputException.thrift_spec), None, ), # 4 ) - def __init__(self, success=None, o1=None,): + def __init__(self, success=None, o1=None, o2=None, o3=None, o4=None,): self.success = success self.o1 = o1 + self.o2 = o2 + self.o3 = o3 + self.o4 = o4 def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -15240,16 +17966,34 @@ def read(self, iprot): if ftype == TType.STOP: break if fid == 0: - if ftype == TType.STRING: - self.success = iprot.readString(); + if ftype == TType.BOOL: + self.success = iprot.readBool(); else: iprot.skip(ftype) elif fid == 1: if ftype == TType.STRUCT: - self.o1 = ConfigValSecurityException() + self.o1 = NoSuchObjectException() self.o1.read(iprot) else: iprot.skip(ftype) + elif fid == 2: + if ftype == TType.STRUCT: + self.o2 = InvalidObjectException() + self.o2.read(iprot) + else: + iprot.skip(ftype) + elif fid == 3: + if ftype == TType.STRUCT: + self.o3 = MetaException() + self.o3.read(iprot) + else: + iprot.skip(ftype) + elif fid == 4: + if ftype == TType.STRUCT: + self.o4 = InvalidInputException() + self.o4.read(iprot) + else: + iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() @@ -15259,15 +18003,27 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('get_config_value_result') + oprot.writeStructBegin('update_partition_column_statistics_result') if self.success is not None: - oprot.writeFieldBegin('success', TType.STRING, 0) - oprot.writeString(self.success) + oprot.writeFieldBegin('success', TType.BOOL, 0) + oprot.writeBool(self.success) oprot.writeFieldEnd() if self.o1 is not None: oprot.writeFieldBegin('o1', TType.STRUCT, 1) self.o1.write(oprot) oprot.writeFieldEnd() + if self.o2 is not None: + oprot.writeFieldBegin('o2', TType.STRUCT, 2) + self.o2.write(oprot) + oprot.writeFieldEnd() + if self.o3 is not None: + oprot.writeFieldBegin('o3', TType.STRUCT, 3) + self.o3.write(oprot) + oprot.writeFieldEnd() + if self.o4 is not None: + oprot.writeFieldBegin('o4', TType.STRUCT, 4) + self.o4.write(oprot) + oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -15286,19 +18042,25 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class partition_name_to_vals_args: +class get_table_column_statistics_args: """ Attributes: - - part_name + - db_name + - tbl_name + - col_name """ thrift_spec = ( None, # 0 - (1, TType.STRING, 'part_name', None, None, ), # 1 + (1, TType.STRING, 'db_name', None, None, ), # 1 + (2, TType.STRING, 'tbl_name', None, None, ), # 2 + (3, TType.STRING, 'col_name', None, None, ), # 3 ) - def __init__(self, part_name=None,): - self.part_name = part_name + def __init__(self, db_name=None, tbl_name=None, col_name=None,): + self.db_name = db_name + self.tbl_name = tbl_name + self.col_name = col_name def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -15311,7 +18073,17 @@ def read(self, iprot): break if fid == 1: if ftype == TType.STRING: - self.part_name = iprot.readString(); + self.db_name = iprot.readString(); + else: + iprot.skip(ftype) + elif fid == 2: + if ftype == TType.STRING: + self.tbl_name = iprot.readString(); + else: + iprot.skip(ftype) + elif fid == 3: + if ftype == TType.STRING: + self.col_name = iprot.readString(); else: iprot.skip(ftype) else: @@ -15323,10 +18095,18 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('partition_name_to_vals_args') - if self.part_name is not None: - oprot.writeFieldBegin('part_name', TType.STRING, 1) - oprot.writeString(self.part_name) + oprot.writeStructBegin('get_table_column_statistics_args') + if self.db_name is not None: + oprot.writeFieldBegin('db_name', TType.STRING, 1) + oprot.writeString(self.db_name) + oprot.writeFieldEnd() + if self.tbl_name is not None: + oprot.writeFieldBegin('tbl_name', TType.STRING, 2) + oprot.writeString(self.tbl_name) + oprot.writeFieldEnd() + if self.col_name is not None: + oprot.writeFieldBegin('col_name', TType.STRING, 3) + oprot.writeString(self.col_name) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -15346,21 +18126,30 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class partition_name_to_vals_result: +class get_table_column_statistics_result: """ Attributes: - success - o1 + - o2 + - o3 + - o4 """ thrift_spec = ( - (0, TType.LIST, 'success', (TType.STRING,None), None, ), # 0 - (1, TType.STRUCT, 'o1', (MetaException, MetaException.thrift_spec), None, ), # 1 + (0, TType.STRUCT, 'success', (ColumnStatistics, ColumnStatistics.thrift_spec), None, ), # 0 + (1, TType.STRUCT, 'o1', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 1 + (2, TType.STRUCT, 'o2', (MetaException, MetaException.thrift_spec), None, ), # 2 + (3, TType.STRUCT, 'o3', (InvalidInputException, InvalidInputException.thrift_spec), None, ), # 3 + (4, TType.STRUCT, 'o4', (InvalidObjectException, InvalidObjectException.thrift_spec), None, ), # 4 ) - def __init__(self, success=None, o1=None,): + def __init__(self, success=None, o1=None, o2=None, o3=None, o4=None,): self.success = success self.o1 = o1 + self.o2 = o2 + self.o3 = o3 + self.o4 = o4 def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -15372,21 +18161,35 @@ def read(self, iprot): if ftype == TType.STOP: break if fid == 0: - if ftype == TType.LIST: - self.success = [] - (_etype579, _size576) = iprot.readListBegin() - for _i580 in xrange(_size576): - _elem581 = iprot.readString(); - self.success.append(_elem581) - iprot.readListEnd() + if ftype == TType.STRUCT: + self.success = ColumnStatistics() + self.success.read(iprot) else: iprot.skip(ftype) elif fid == 1: if ftype == TType.STRUCT: - self.o1 = MetaException() + self.o1 = NoSuchObjectException() self.o1.read(iprot) else: iprot.skip(ftype) + elif fid == 2: + if ftype == TType.STRUCT: + self.o2 = MetaException() + self.o2.read(iprot) + else: + iprot.skip(ftype) + elif fid == 3: + if ftype == TType.STRUCT: + self.o3 = InvalidInputException() + self.o3.read(iprot) + else: + iprot.skip(ftype) + elif fid == 4: + if ftype == TType.STRUCT: + self.o4 = InvalidObjectException() + self.o4.read(iprot) + else: + iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() @@ -15396,18 +18199,27 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('partition_name_to_vals_result') + oprot.writeStructBegin('get_table_column_statistics_result') if self.success is not None: - oprot.writeFieldBegin('success', TType.LIST, 0) - oprot.writeListBegin(TType.STRING, len(self.success)) - for iter582 in self.success: - oprot.writeString(iter582) - oprot.writeListEnd() + oprot.writeFieldBegin('success', TType.STRUCT, 0) + self.success.write(oprot) oprot.writeFieldEnd() if self.o1 is not None: oprot.writeFieldBegin('o1', TType.STRUCT, 1) self.o1.write(oprot) oprot.writeFieldEnd() + if self.o2 is not None: + oprot.writeFieldBegin('o2', TType.STRUCT, 2) + self.o2.write(oprot) + oprot.writeFieldEnd() + if self.o3 is not None: + oprot.writeFieldBegin('o3', TType.STRUCT, 3) + self.o3.write(oprot) + oprot.writeFieldEnd() + if self.o4 is not None: + oprot.writeFieldBegin('o4', TType.STRUCT, 4) + self.o4.write(oprot) + oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -15426,19 +18238,28 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class partition_name_to_spec_args: +class get_partition_column_statistics_args: """ Attributes: + - db_name + - tbl_name - part_name + - col_name """ thrift_spec = ( None, # 0 - (1, TType.STRING, 'part_name', None, None, ), # 1 + (1, TType.STRING, 'db_name', None, None, ), # 1 + (2, TType.STRING, 'tbl_name', None, None, ), # 2 + (3, TType.STRING, 'part_name', None, None, ), # 3 + (4, TType.STRING, 'col_name', None, None, ), # 4 ) - def __init__(self, part_name=None,): + def __init__(self, db_name=None, tbl_name=None, part_name=None, col_name=None,): + self.db_name = db_name + self.tbl_name = tbl_name self.part_name = part_name + self.col_name = col_name def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -15451,9 +18272,24 @@ def read(self, iprot): break if fid == 1: if ftype == TType.STRING: + self.db_name = iprot.readString(); + else: + iprot.skip(ftype) + elif fid == 2: + if ftype == TType.STRING: + self.tbl_name = iprot.readString(); + else: + iprot.skip(ftype) + elif fid == 3: + if ftype == TType.STRING: self.part_name = iprot.readString(); else: iprot.skip(ftype) + elif fid == 4: + if ftype == TType.STRING: + self.col_name = iprot.readString(); + else: + iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() @@ -15463,11 +18299,23 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('partition_name_to_spec_args') + oprot.writeStructBegin('get_partition_column_statistics_args') + if self.db_name is not None: + oprot.writeFieldBegin('db_name', TType.STRING, 1) + oprot.writeString(self.db_name) + oprot.writeFieldEnd() + if self.tbl_name is not None: + oprot.writeFieldBegin('tbl_name', TType.STRING, 2) + oprot.writeString(self.tbl_name) + oprot.writeFieldEnd() if self.part_name is not None: - oprot.writeFieldBegin('part_name', TType.STRING, 1) + oprot.writeFieldBegin('part_name', TType.STRING, 3) oprot.writeString(self.part_name) oprot.writeFieldEnd() + if self.col_name is not None: + oprot.writeFieldBegin('col_name', TType.STRING, 4) + oprot.writeString(self.col_name) + oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -15486,21 +18334,30 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class partition_name_to_spec_result: +class get_partition_column_statistics_result: """ Attributes: - success - o1 + - o2 + - o3 + - o4 """ thrift_spec = ( - (0, TType.MAP, 'success', (TType.STRING,None,TType.STRING,None), None, ), # 0 - (1, TType.STRUCT, 'o1', (MetaException, MetaException.thrift_spec), None, ), # 1 + (0, TType.STRUCT, 'success', (ColumnStatistics, ColumnStatistics.thrift_spec), None, ), # 0 + (1, TType.STRUCT, 'o1', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 1 + (2, TType.STRUCT, 'o2', (MetaException, MetaException.thrift_spec), None, ), # 2 + (3, TType.STRUCT, 'o3', (InvalidInputException, InvalidInputException.thrift_spec), None, ), # 3 + (4, TType.STRUCT, 'o4', (InvalidObjectException, InvalidObjectException.thrift_spec), None, ), # 4 ) - def __init__(self, success=None, o1=None,): + def __init__(self, success=None, o1=None, o2=None, o3=None, o4=None,): self.success = success self.o1 = o1 + self.o2 = o2 + self.o3 = o3 + self.o4 = o4 def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -15512,20 +18369,33 @@ def read(self, iprot): if ftype == TType.STOP: break if fid == 0: - if ftype == TType.MAP: - self.success = {} - (_ktype584, _vtype585, _size583 ) = iprot.readMapBegin() - for _i587 in xrange(_size583): - _key588 = iprot.readString(); - _val589 = iprot.readString(); - self.success[_key588] = _val589 - iprot.readMapEnd() + if ftype == TType.STRUCT: + self.success = ColumnStatistics() + self.success.read(iprot) + else: + iprot.skip(ftype) + elif fid == 1: + if ftype == TType.STRUCT: + self.o1 = NoSuchObjectException() + self.o1.read(iprot) + else: + iprot.skip(ftype) + elif fid == 2: + if ftype == TType.STRUCT: + self.o2 = MetaException() + self.o2.read(iprot) + else: + iprot.skip(ftype) + elif fid == 3: + if ftype == TType.STRUCT: + self.o3 = InvalidInputException() + self.o3.read(iprot) else: iprot.skip(ftype) - elif fid == 1: + elif fid == 4: if ftype == TType.STRUCT: - self.o1 = MetaException() - self.o1.read(iprot) + self.o4 = InvalidObjectException() + self.o4.read(iprot) else: iprot.skip(ftype) else: @@ -15537,19 +18407,27 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('partition_name_to_spec_result') + oprot.writeStructBegin('get_partition_column_statistics_result') if self.success is not None: - oprot.writeFieldBegin('success', TType.MAP, 0) - oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.success)) - for kiter590,viter591 in self.success.items(): - oprot.writeString(kiter590) - oprot.writeString(viter591) - oprot.writeMapEnd() + oprot.writeFieldBegin('success', TType.STRUCT, 0) + self.success.write(oprot) oprot.writeFieldEnd() if self.o1 is not None: oprot.writeFieldBegin('o1', TType.STRUCT, 1) self.o1.write(oprot) oprot.writeFieldEnd() + if self.o2 is not None: + oprot.writeFieldBegin('o2', TType.STRUCT, 2) + self.o2.write(oprot) + oprot.writeFieldEnd() + if self.o3 is not None: + oprot.writeFieldBegin('o3', TType.STRUCT, 3) + self.o3.write(oprot) + oprot.writeFieldEnd() + if self.o4 is not None: + oprot.writeFieldBegin('o4', TType.STRUCT, 4) + self.o4.write(oprot) + oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -15568,28 +18446,19 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class markPartitionForEvent_args: +class get_table_statistics_req_args: """ Attributes: - - db_name - - tbl_name - - part_vals - - eventType + - request """ thrift_spec = ( None, # 0 - (1, TType.STRING, 'db_name', None, None, ), # 1 - (2, TType.STRING, 'tbl_name', None, None, ), # 2 - (3, TType.MAP, 'part_vals', (TType.STRING,None,TType.STRING,None), None, ), # 3 - (4, TType.I32, 'eventType', None, None, ), # 4 + (1, TType.STRUCT, 'request', (TableStatsRequest, TableStatsRequest.thrift_spec), None, ), # 1 ) - def __init__(self, db_name=None, tbl_name=None, part_vals=None, eventType=None,): - self.db_name = db_name - self.tbl_name = tbl_name - self.part_vals = part_vals - self.eventType = eventType + def __init__(self, request=None,): + self.request = request def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -15601,29 +18470,9 @@ def read(self, iprot): if ftype == TType.STOP: break if fid == 1: - if ftype == TType.STRING: - self.db_name = iprot.readString(); - else: - iprot.skip(ftype) - elif fid == 2: - if ftype == TType.STRING: - self.tbl_name = iprot.readString(); - else: - iprot.skip(ftype) - elif fid == 3: - if ftype == TType.MAP: - self.part_vals = {} - (_ktype593, _vtype594, _size592 ) = iprot.readMapBegin() - for _i596 in xrange(_size592): - _key597 = iprot.readString(); - _val598 = iprot.readString(); - self.part_vals[_key597] = _val598 - iprot.readMapEnd() - else: - iprot.skip(ftype) - elif fid == 4: - if ftype == TType.I32: - self.eventType = iprot.readI32(); + if ftype == TType.STRUCT: + self.request = TableStatsRequest() + self.request.read(iprot) else: iprot.skip(ftype) else: @@ -15635,26 +18484,10 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('markPartitionForEvent_args') - if self.db_name is not None: - oprot.writeFieldBegin('db_name', TType.STRING, 1) - oprot.writeString(self.db_name) - oprot.writeFieldEnd() - if self.tbl_name is not None: - oprot.writeFieldBegin('tbl_name', TType.STRING, 2) - oprot.writeString(self.tbl_name) - oprot.writeFieldEnd() - if self.part_vals is not None: - oprot.writeFieldBegin('part_vals', TType.MAP, 3) - oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.part_vals)) - for kiter599,viter600 in self.part_vals.items(): - oprot.writeString(kiter599) - oprot.writeString(viter600) - oprot.writeMapEnd() - oprot.writeFieldEnd() - if self.eventType is not None: - oprot.writeFieldBegin('eventType', TType.I32, 4) - oprot.writeI32(self.eventType) + oprot.writeStructBegin('get_table_statistics_req_args') + if self.request is not None: + oprot.writeFieldBegin('request', TType.STRUCT, 1) + self.request.write(oprot) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -15674,34 +18507,24 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class markPartitionForEvent_result: +class get_table_statistics_req_result: """ Attributes: + - success - o1 - o2 - - o3 - - o4 - - o5 - - o6 """ thrift_spec = ( - None, # 0 - (1, TType.STRUCT, 'o1', (MetaException, MetaException.thrift_spec), None, ), # 1 - (2, TType.STRUCT, 'o2', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 2 - (3, TType.STRUCT, 'o3', (UnknownDBException, UnknownDBException.thrift_spec), None, ), # 3 - (4, TType.STRUCT, 'o4', (UnknownTableException, UnknownTableException.thrift_spec), None, ), # 4 - (5, TType.STRUCT, 'o5', (UnknownPartitionException, UnknownPartitionException.thrift_spec), None, ), # 5 - (6, TType.STRUCT, 'o6', (InvalidPartitionException, InvalidPartitionException.thrift_spec), None, ), # 6 + (0, TType.STRUCT, 'success', (TableStatsResult, TableStatsResult.thrift_spec), None, ), # 0 + (1, TType.STRUCT, 'o1', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 1 + (2, TType.STRUCT, 'o2', (MetaException, MetaException.thrift_spec), None, ), # 2 ) - def __init__(self, o1=None, o2=None, o3=None, o4=None, o5=None, o6=None,): + def __init__(self, success=None, o1=None, o2=None,): + self.success = success self.o1 = o1 self.o2 = o2 - self.o3 = o3 - self.o4 = o4 - self.o5 = o5 - self.o6 = o6 def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -15712,40 +18535,22 @@ def read(self, iprot): (fname, ftype, fid) = iprot.readFieldBegin() if ftype == TType.STOP: break - if fid == 1: - if ftype == TType.STRUCT: - self.o1 = MetaException() - self.o1.read(iprot) - else: - iprot.skip(ftype) - elif fid == 2: - if ftype == TType.STRUCT: - self.o2 = NoSuchObjectException() - self.o2.read(iprot) - else: - iprot.skip(ftype) - elif fid == 3: - if ftype == TType.STRUCT: - self.o3 = UnknownDBException() - self.o3.read(iprot) - else: - iprot.skip(ftype) - elif fid == 4: + if fid == 0: if ftype == TType.STRUCT: - self.o4 = UnknownTableException() - self.o4.read(iprot) + self.success = TableStatsResult() + self.success.read(iprot) else: iprot.skip(ftype) - elif fid == 5: + elif fid == 1: if ftype == TType.STRUCT: - self.o5 = UnknownPartitionException() - self.o5.read(iprot) + self.o1 = NoSuchObjectException() + self.o1.read(iprot) else: iprot.skip(ftype) - elif fid == 6: + elif fid == 2: if ftype == TType.STRUCT: - self.o6 = InvalidPartitionException() - self.o6.read(iprot) + self.o2 = MetaException() + self.o2.read(iprot) else: iprot.skip(ftype) else: @@ -15757,7 +18562,11 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('markPartitionForEvent_result') + oprot.writeStructBegin('get_table_statistics_req_result') + if self.success is not None: + oprot.writeFieldBegin('success', TType.STRUCT, 0) + self.success.write(oprot) + oprot.writeFieldEnd() if self.o1 is not None: oprot.writeFieldBegin('o1', TType.STRUCT, 1) self.o1.write(oprot) @@ -15766,22 +18575,6 @@ def write(self, oprot): oprot.writeFieldBegin('o2', TType.STRUCT, 2) self.o2.write(oprot) oprot.writeFieldEnd() - if self.o3 is not None: - oprot.writeFieldBegin('o3', TType.STRUCT, 3) - self.o3.write(oprot) - oprot.writeFieldEnd() - if self.o4 is not None: - oprot.writeFieldBegin('o4', TType.STRUCT, 4) - self.o4.write(oprot) - oprot.writeFieldEnd() - if self.o5 is not None: - oprot.writeFieldBegin('o5', TType.STRUCT, 5) - self.o5.write(oprot) - oprot.writeFieldEnd() - if self.o6 is not None: - oprot.writeFieldBegin('o6', TType.STRUCT, 6) - self.o6.write(oprot) - oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -15800,28 +18593,19 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class isPartitionMarkedForEvent_args: +class get_partitions_statistics_req_args: """ Attributes: - - db_name - - tbl_name - - part_vals - - eventType + - request """ thrift_spec = ( None, # 0 - (1, TType.STRING, 'db_name', None, None, ), # 1 - (2, TType.STRING, 'tbl_name', None, None, ), # 2 - (3, TType.MAP, 'part_vals', (TType.STRING,None,TType.STRING,None), None, ), # 3 - (4, TType.I32, 'eventType', None, None, ), # 4 + (1, TType.STRUCT, 'request', (PartitionsStatsRequest, PartitionsStatsRequest.thrift_spec), None, ), # 1 ) - def __init__(self, db_name=None, tbl_name=None, part_vals=None, eventType=None,): - self.db_name = db_name - self.tbl_name = tbl_name - self.part_vals = part_vals - self.eventType = eventType + def __init__(self, request=None,): + self.request = request def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -15833,29 +18617,9 @@ def read(self, iprot): if ftype == TType.STOP: break if fid == 1: - if ftype == TType.STRING: - self.db_name = iprot.readString(); - else: - iprot.skip(ftype) - elif fid == 2: - if ftype == TType.STRING: - self.tbl_name = iprot.readString(); - else: - iprot.skip(ftype) - elif fid == 3: - if ftype == TType.MAP: - self.part_vals = {} - (_ktype602, _vtype603, _size601 ) = iprot.readMapBegin() - for _i605 in xrange(_size601): - _key606 = iprot.readString(); - _val607 = iprot.readString(); - self.part_vals[_key606] = _val607 - iprot.readMapEnd() - else: - iprot.skip(ftype) - elif fid == 4: - if ftype == TType.I32: - self.eventType = iprot.readI32(); + if ftype == TType.STRUCT: + self.request = PartitionsStatsRequest() + self.request.read(iprot) else: iprot.skip(ftype) else: @@ -15867,26 +18631,10 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('isPartitionMarkedForEvent_args') - if self.db_name is not None: - oprot.writeFieldBegin('db_name', TType.STRING, 1) - oprot.writeString(self.db_name) - oprot.writeFieldEnd() - if self.tbl_name is not None: - oprot.writeFieldBegin('tbl_name', TType.STRING, 2) - oprot.writeString(self.tbl_name) - oprot.writeFieldEnd() - if self.part_vals is not None: - oprot.writeFieldBegin('part_vals', TType.MAP, 3) - oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.part_vals)) - for kiter608,viter609 in self.part_vals.items(): - oprot.writeString(kiter608) - oprot.writeString(viter609) - oprot.writeMapEnd() - oprot.writeFieldEnd() - if self.eventType is not None: - oprot.writeFieldBegin('eventType', TType.I32, 4) - oprot.writeI32(self.eventType) + oprot.writeStructBegin('get_partitions_statistics_req_args') + if self.request is not None: + oprot.writeFieldBegin('request', TType.STRUCT, 1) + self.request.write(oprot) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -15906,36 +18654,24 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class isPartitionMarkedForEvent_result: +class get_partitions_statistics_req_result: """ Attributes: - success - o1 - o2 - - o3 - - o4 - - o5 - - o6 """ thrift_spec = ( - (0, TType.BOOL, 'success', None, None, ), # 0 - (1, TType.STRUCT, 'o1', (MetaException, MetaException.thrift_spec), None, ), # 1 - (2, TType.STRUCT, 'o2', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 2 - (3, TType.STRUCT, 'o3', (UnknownDBException, UnknownDBException.thrift_spec), None, ), # 3 - (4, TType.STRUCT, 'o4', (UnknownTableException, UnknownTableException.thrift_spec), None, ), # 4 - (5, TType.STRUCT, 'o5', (UnknownPartitionException, UnknownPartitionException.thrift_spec), None, ), # 5 - (6, TType.STRUCT, 'o6', (InvalidPartitionException, InvalidPartitionException.thrift_spec), None, ), # 6 + (0, TType.STRUCT, 'success', (PartitionsStatsResult, PartitionsStatsResult.thrift_spec), None, ), # 0 + (1, TType.STRUCT, 'o1', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 1 + (2, TType.STRUCT, 'o2', (MetaException, MetaException.thrift_spec), None, ), # 2 ) - def __init__(self, success=None, o1=None, o2=None, o3=None, o4=None, o5=None, o6=None,): + def __init__(self, success=None, o1=None, o2=None,): self.success = success self.o1 = o1 self.o2 = o2 - self.o3 = o3 - self.o4 = o4 - self.o5 = o5 - self.o6 = o6 def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -15946,47 +18682,24 @@ def read(self, iprot): (fname, ftype, fid) = iprot.readFieldBegin() if ftype == TType.STOP: break - if fid == 0: - if ftype == TType.BOOL: - self.success = iprot.readBool(); + if fid == 0: + if ftype == TType.STRUCT: + self.success = PartitionsStatsResult() + self.success.read(iprot) else: iprot.skip(ftype) elif fid == 1: if ftype == TType.STRUCT: - self.o1 = MetaException() + self.o1 = NoSuchObjectException() self.o1.read(iprot) else: iprot.skip(ftype) elif fid == 2: if ftype == TType.STRUCT: - self.o2 = NoSuchObjectException() + self.o2 = MetaException() self.o2.read(iprot) else: iprot.skip(ftype) - elif fid == 3: - if ftype == TType.STRUCT: - self.o3 = UnknownDBException() - self.o3.read(iprot) - else: - iprot.skip(ftype) - elif fid == 4: - if ftype == TType.STRUCT: - self.o4 = UnknownTableException() - self.o4.read(iprot) - else: - iprot.skip(ftype) - elif fid == 5: - if ftype == TType.STRUCT: - self.o5 = UnknownPartitionException() - self.o5.read(iprot) - else: - iprot.skip(ftype) - elif fid == 6: - if ftype == TType.STRUCT: - self.o6 = InvalidPartitionException() - self.o6.read(iprot) - else: - iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() @@ -15996,10 +18709,10 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('isPartitionMarkedForEvent_result') + oprot.writeStructBegin('get_partitions_statistics_req_result') if self.success is not None: - oprot.writeFieldBegin('success', TType.BOOL, 0) - oprot.writeBool(self.success) + oprot.writeFieldBegin('success', TType.STRUCT, 0) + self.success.write(oprot) oprot.writeFieldEnd() if self.o1 is not None: oprot.writeFieldBegin('o1', TType.STRUCT, 1) @@ -16009,22 +18722,6 @@ def write(self, oprot): oprot.writeFieldBegin('o2', TType.STRUCT, 2) self.o2.write(oprot) oprot.writeFieldEnd() - if self.o3 is not None: - oprot.writeFieldBegin('o3', TType.STRUCT, 3) - self.o3.write(oprot) - oprot.writeFieldEnd() - if self.o4 is not None: - oprot.writeFieldBegin('o4', TType.STRUCT, 4) - self.o4.write(oprot) - oprot.writeFieldEnd() - if self.o5 is not None: - oprot.writeFieldBegin('o5', TType.STRUCT, 5) - self.o5.write(oprot) - oprot.writeFieldEnd() - if self.o6 is not None: - oprot.writeFieldBegin('o6', TType.STRUCT, 6) - self.o6.write(oprot) - oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -16043,22 +18740,28 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class add_index_args: +class delete_partition_column_statistics_args: """ Attributes: - - new_index - - index_table + - db_name + - tbl_name + - part_name + - col_name """ thrift_spec = ( None, # 0 - (1, TType.STRUCT, 'new_index', (Index, Index.thrift_spec), None, ), # 1 - (2, TType.STRUCT, 'index_table', (Table, Table.thrift_spec), None, ), # 2 + (1, TType.STRING, 'db_name', None, None, ), # 1 + (2, TType.STRING, 'tbl_name', None, None, ), # 2 + (3, TType.STRING, 'part_name', None, None, ), # 3 + (4, TType.STRING, 'col_name', None, None, ), # 4 ) - def __init__(self, new_index=None, index_table=None,): - self.new_index = new_index - self.index_table = index_table + def __init__(self, db_name=None, tbl_name=None, part_name=None, col_name=None,): + self.db_name = db_name + self.tbl_name = tbl_name + self.part_name = part_name + self.col_name = col_name def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -16070,15 +18773,23 @@ def read(self, iprot): if ftype == TType.STOP: break if fid == 1: - if ftype == TType.STRUCT: - self.new_index = Index() - self.new_index.read(iprot) + if ftype == TType.STRING: + self.db_name = iprot.readString(); else: iprot.skip(ftype) elif fid == 2: - if ftype == TType.STRUCT: - self.index_table = Table() - self.index_table.read(iprot) + if ftype == TType.STRING: + self.tbl_name = iprot.readString(); + else: + iprot.skip(ftype) + elif fid == 3: + if ftype == TType.STRING: + self.part_name = iprot.readString(); + else: + iprot.skip(ftype) + elif fid == 4: + if ftype == TType.STRING: + self.col_name = iprot.readString(); else: iprot.skip(ftype) else: @@ -16090,14 +18801,22 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('add_index_args') - if self.new_index is not None: - oprot.writeFieldBegin('new_index', TType.STRUCT, 1) - self.new_index.write(oprot) + oprot.writeStructBegin('delete_partition_column_statistics_args') + if self.db_name is not None: + oprot.writeFieldBegin('db_name', TType.STRING, 1) + oprot.writeString(self.db_name) oprot.writeFieldEnd() - if self.index_table is not None: - oprot.writeFieldBegin('index_table', TType.STRUCT, 2) - self.index_table.write(oprot) + if self.tbl_name is not None: + oprot.writeFieldBegin('tbl_name', TType.STRING, 2) + oprot.writeString(self.tbl_name) + oprot.writeFieldEnd() + if self.part_name is not None: + oprot.writeFieldBegin('part_name', TType.STRING, 3) + oprot.writeString(self.part_name) + oprot.writeFieldEnd() + if self.col_name is not None: + oprot.writeFieldBegin('col_name', TType.STRING, 4) + oprot.writeString(self.col_name) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -16117,27 +18836,30 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class add_index_result: +class delete_partition_column_statistics_result: """ Attributes: - success - o1 - o2 - o3 + - o4 """ thrift_spec = ( - (0, TType.STRUCT, 'success', (Index, Index.thrift_spec), None, ), # 0 - (1, TType.STRUCT, 'o1', (InvalidObjectException, InvalidObjectException.thrift_spec), None, ), # 1 - (2, TType.STRUCT, 'o2', (AlreadyExistsException, AlreadyExistsException.thrift_spec), None, ), # 2 - (3, TType.STRUCT, 'o3', (MetaException, MetaException.thrift_spec), None, ), # 3 + (0, TType.BOOL, 'success', None, None, ), # 0 + (1, TType.STRUCT, 'o1', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 1 + (2, TType.STRUCT, 'o2', (MetaException, MetaException.thrift_spec), None, ), # 2 + (3, TType.STRUCT, 'o3', (InvalidObjectException, InvalidObjectException.thrift_spec), None, ), # 3 + (4, TType.STRUCT, 'o4', (InvalidInputException, InvalidInputException.thrift_spec), None, ), # 4 ) - def __init__(self, success=None, o1=None, o2=None, o3=None,): + def __init__(self, success=None, o1=None, o2=None, o3=None, o4=None,): self.success = success self.o1 = o1 self.o2 = o2 self.o3 = o3 + self.o4 = o4 def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -16149,29 +18871,34 @@ def read(self, iprot): if ftype == TType.STOP: break if fid == 0: - if ftype == TType.STRUCT: - self.success = Index() - self.success.read(iprot) + if ftype == TType.BOOL: + self.success = iprot.readBool(); else: iprot.skip(ftype) elif fid == 1: if ftype == TType.STRUCT: - self.o1 = InvalidObjectException() + self.o1 = NoSuchObjectException() self.o1.read(iprot) else: iprot.skip(ftype) elif fid == 2: if ftype == TType.STRUCT: - self.o2 = AlreadyExistsException() + self.o2 = MetaException() self.o2.read(iprot) else: iprot.skip(ftype) elif fid == 3: if ftype == TType.STRUCT: - self.o3 = MetaException() + self.o3 = InvalidObjectException() self.o3.read(iprot) else: iprot.skip(ftype) + elif fid == 4: + if ftype == TType.STRUCT: + self.o4 = InvalidInputException() + self.o4.read(iprot) + else: + iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() @@ -16181,10 +18908,10 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('add_index_result') + oprot.writeStructBegin('delete_partition_column_statistics_result') if self.success is not None: - oprot.writeFieldBegin('success', TType.STRUCT, 0) - self.success.write(oprot) + oprot.writeFieldBegin('success', TType.BOOL, 0) + oprot.writeBool(self.success) oprot.writeFieldEnd() if self.o1 is not None: oprot.writeFieldBegin('o1', TType.STRUCT, 1) @@ -16198,6 +18925,10 @@ def write(self, oprot): oprot.writeFieldBegin('o3', TType.STRUCT, 3) self.o3.write(oprot) oprot.writeFieldEnd() + if self.o4 is not None: + oprot.writeFieldBegin('o4', TType.STRUCT, 4) + self.o4.write(oprot) + oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -16216,28 +18947,25 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class alter_index_args: +class delete_table_column_statistics_args: """ Attributes: - - dbname - - base_tbl_name - - idx_name - - new_idx + - db_name + - tbl_name + - col_name """ thrift_spec = ( None, # 0 - (1, TType.STRING, 'dbname', None, None, ), # 1 - (2, TType.STRING, 'base_tbl_name', None, None, ), # 2 - (3, TType.STRING, 'idx_name', None, None, ), # 3 - (4, TType.STRUCT, 'new_idx', (Index, Index.thrift_spec), None, ), # 4 + (1, TType.STRING, 'db_name', None, None, ), # 1 + (2, TType.STRING, 'tbl_name', None, None, ), # 2 + (3, TType.STRING, 'col_name', None, None, ), # 3 ) - def __init__(self, dbname=None, base_tbl_name=None, idx_name=None, new_idx=None,): - self.dbname = dbname - self.base_tbl_name = base_tbl_name - self.idx_name = idx_name - self.new_idx = new_idx + def __init__(self, db_name=None, tbl_name=None, col_name=None,): + self.db_name = db_name + self.tbl_name = tbl_name + self.col_name = col_name def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -16250,23 +18978,17 @@ def read(self, iprot): break if fid == 1: if ftype == TType.STRING: - self.dbname = iprot.readString(); + self.db_name = iprot.readString(); else: iprot.skip(ftype) elif fid == 2: if ftype == TType.STRING: - self.base_tbl_name = iprot.readString(); + self.tbl_name = iprot.readString(); else: iprot.skip(ftype) elif fid == 3: if ftype == TType.STRING: - self.idx_name = iprot.readString(); - else: - iprot.skip(ftype) - elif fid == 4: - if ftype == TType.STRUCT: - self.new_idx = Index() - self.new_idx.read(iprot) + self.col_name = iprot.readString(); else: iprot.skip(ftype) else: @@ -16278,22 +19000,18 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('alter_index_args') - if self.dbname is not None: - oprot.writeFieldBegin('dbname', TType.STRING, 1) - oprot.writeString(self.dbname) - oprot.writeFieldEnd() - if self.base_tbl_name is not None: - oprot.writeFieldBegin('base_tbl_name', TType.STRING, 2) - oprot.writeString(self.base_tbl_name) + oprot.writeStructBegin('delete_table_column_statistics_args') + if self.db_name is not None: + oprot.writeFieldBegin('db_name', TType.STRING, 1) + oprot.writeString(self.db_name) oprot.writeFieldEnd() - if self.idx_name is not None: - oprot.writeFieldBegin('idx_name', TType.STRING, 3) - oprot.writeString(self.idx_name) + if self.tbl_name is not None: + oprot.writeFieldBegin('tbl_name', TType.STRING, 2) + oprot.writeString(self.tbl_name) oprot.writeFieldEnd() - if self.new_idx is not None: - oprot.writeFieldBegin('new_idx', TType.STRUCT, 4) - self.new_idx.write(oprot) + if self.col_name is not None: + oprot.writeFieldBegin('col_name', TType.STRING, 3) + oprot.writeString(self.col_name) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -16313,22 +19031,30 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class alter_index_result: +class delete_table_column_statistics_result: """ Attributes: + - success - o1 - o2 + - o3 + - o4 """ thrift_spec = ( - None, # 0 - (1, TType.STRUCT, 'o1', (InvalidOperationException, InvalidOperationException.thrift_spec), None, ), # 1 + (0, TType.BOOL, 'success', None, None, ), # 0 + (1, TType.STRUCT, 'o1', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 1 (2, TType.STRUCT, 'o2', (MetaException, MetaException.thrift_spec), None, ), # 2 + (3, TType.STRUCT, 'o3', (InvalidObjectException, InvalidObjectException.thrift_spec), None, ), # 3 + (4, TType.STRUCT, 'o4', (InvalidInputException, InvalidInputException.thrift_spec), None, ), # 4 ) - def __init__(self, o1=None, o2=None,): + def __init__(self, success=None, o1=None, o2=None, o3=None, o4=None,): + self.success = success self.o1 = o1 self.o2 = o2 + self.o3 = o3 + self.o4 = o4 def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -16339,9 +19065,14 @@ def read(self, iprot): (fname, ftype, fid) = iprot.readFieldBegin() if ftype == TType.STOP: break - if fid == 1: + if fid == 0: + if ftype == TType.BOOL: + self.success = iprot.readBool(); + else: + iprot.skip(ftype) + elif fid == 1: if ftype == TType.STRUCT: - self.o1 = InvalidOperationException() + self.o1 = NoSuchObjectException() self.o1.read(iprot) else: iprot.skip(ftype) @@ -16351,6 +19082,18 @@ def read(self, iprot): self.o2.read(iprot) else: iprot.skip(ftype) + elif fid == 3: + if ftype == TType.STRUCT: + self.o3 = InvalidObjectException() + self.o3.read(iprot) + else: + iprot.skip(ftype) + elif fid == 4: + if ftype == TType.STRUCT: + self.o4 = InvalidInputException() + self.o4.read(iprot) + else: + iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() @@ -16360,7 +19103,11 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('alter_index_result') + oprot.writeStructBegin('delete_table_column_statistics_result') + if self.success is not None: + oprot.writeFieldBegin('success', TType.BOOL, 0) + oprot.writeBool(self.success) + oprot.writeFieldEnd() if self.o1 is not None: oprot.writeFieldBegin('o1', TType.STRUCT, 1) self.o1.write(oprot) @@ -16369,6 +19116,14 @@ def write(self, oprot): oprot.writeFieldBegin('o2', TType.STRUCT, 2) self.o2.write(oprot) oprot.writeFieldEnd() + if self.o3 is not None: + oprot.writeFieldBegin('o3', TType.STRUCT, 3) + self.o3.write(oprot) + oprot.writeFieldEnd() + if self.o4 is not None: + oprot.writeFieldBegin('o4', TType.STRUCT, 4) + self.o4.write(oprot) + oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -16387,28 +19142,19 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class drop_index_by_name_args: +class create_function_args: """ Attributes: - - db_name - - tbl_name - - index_name - - deleteData + - func """ thrift_spec = ( None, # 0 - (1, TType.STRING, 'db_name', None, None, ), # 1 - (2, TType.STRING, 'tbl_name', None, None, ), # 2 - (3, TType.STRING, 'index_name', None, None, ), # 3 - (4, TType.BOOL, 'deleteData', None, None, ), # 4 + (1, TType.STRUCT, 'func', (Function, Function.thrift_spec), None, ), # 1 ) - def __init__(self, db_name=None, tbl_name=None, index_name=None, deleteData=None,): - self.db_name = db_name - self.tbl_name = tbl_name - self.index_name = index_name - self.deleteData = deleteData + def __init__(self, func=None,): + self.func = func def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -16420,23 +19166,9 @@ def read(self, iprot): if ftype == TType.STOP: break if fid == 1: - if ftype == TType.STRING: - self.db_name = iprot.readString(); - else: - iprot.skip(ftype) - elif fid == 2: - if ftype == TType.STRING: - self.tbl_name = iprot.readString(); - else: - iprot.skip(ftype) - elif fid == 3: - if ftype == TType.STRING: - self.index_name = iprot.readString(); - else: - iprot.skip(ftype) - elif fid == 4: - if ftype == TType.BOOL: - self.deleteData = iprot.readBool(); + if ftype == TType.STRUCT: + self.func = Function() + self.func.read(iprot) else: iprot.skip(ftype) else: @@ -16448,22 +19180,10 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('drop_index_by_name_args') - if self.db_name is not None: - oprot.writeFieldBegin('db_name', TType.STRING, 1) - oprot.writeString(self.db_name) - oprot.writeFieldEnd() - if self.tbl_name is not None: - oprot.writeFieldBegin('tbl_name', TType.STRING, 2) - oprot.writeString(self.tbl_name) - oprot.writeFieldEnd() - if self.index_name is not None: - oprot.writeFieldBegin('index_name', TType.STRING, 3) - oprot.writeString(self.index_name) - oprot.writeFieldEnd() - if self.deleteData is not None: - oprot.writeFieldBegin('deleteData', TType.BOOL, 4) - oprot.writeBool(self.deleteData) + oprot.writeStructBegin('create_function_args') + if self.func is not None: + oprot.writeFieldBegin('func', TType.STRUCT, 1) + self.func.write(oprot) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -16483,24 +19203,28 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class drop_index_by_name_result: +class create_function_result: """ Attributes: - - success - o1 - o2 + - o3 + - o4 """ thrift_spec = ( - (0, TType.BOOL, 'success', None, None, ), # 0 - (1, TType.STRUCT, 'o1', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 1 - (2, TType.STRUCT, 'o2', (MetaException, MetaException.thrift_spec), None, ), # 2 + None, # 0 + (1, TType.STRUCT, 'o1', (AlreadyExistsException, AlreadyExistsException.thrift_spec), None, ), # 1 + (2, TType.STRUCT, 'o2', (InvalidObjectException, InvalidObjectException.thrift_spec), None, ), # 2 + (3, TType.STRUCT, 'o3', (MetaException, MetaException.thrift_spec), None, ), # 3 + (4, TType.STRUCT, 'o4', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 4 ) - def __init__(self, success=None, o1=None, o2=None,): - self.success = success + def __init__(self, o1=None, o2=None, o3=None, o4=None,): self.o1 = o1 self.o2 = o2 + self.o3 = o3 + self.o4 = o4 def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -16511,23 +19235,30 @@ def read(self, iprot): (fname, ftype, fid) = iprot.readFieldBegin() if ftype == TType.STOP: break - if fid == 0: - if ftype == TType.BOOL: - self.success = iprot.readBool(); - else: - iprot.skip(ftype) - elif fid == 1: + if fid == 1: if ftype == TType.STRUCT: - self.o1 = NoSuchObjectException() + self.o1 = AlreadyExistsException() self.o1.read(iprot) else: iprot.skip(ftype) elif fid == 2: if ftype == TType.STRUCT: - self.o2 = MetaException() + self.o2 = InvalidObjectException() self.o2.read(iprot) else: iprot.skip(ftype) + elif fid == 3: + if ftype == TType.STRUCT: + self.o3 = MetaException() + self.o3.read(iprot) + else: + iprot.skip(ftype) + elif fid == 4: + if ftype == TType.STRUCT: + self.o4 = NoSuchObjectException() + self.o4.read(iprot) + else: + iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() @@ -16537,11 +19268,7 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('drop_index_by_name_result') - if self.success is not None: - oprot.writeFieldBegin('success', TType.BOOL, 0) - oprot.writeBool(self.success) - oprot.writeFieldEnd() + oprot.writeStructBegin('create_function_result') if self.o1 is not None: oprot.writeFieldBegin('o1', TType.STRUCT, 1) self.o1.write(oprot) @@ -16550,6 +19277,14 @@ def write(self, oprot): oprot.writeFieldBegin('o2', TType.STRUCT, 2) self.o2.write(oprot) oprot.writeFieldEnd() + if self.o3 is not None: + oprot.writeFieldBegin('o3', TType.STRUCT, 3) + self.o3.write(oprot) + oprot.writeFieldEnd() + if self.o4 is not None: + oprot.writeFieldBegin('o4', TType.STRUCT, 4) + self.o4.write(oprot) + oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -16568,25 +19303,22 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class get_index_by_name_args: +class drop_function_args: """ Attributes: - - db_name - - tbl_name - - index_name + - dbName + - funcName """ thrift_spec = ( None, # 0 - (1, TType.STRING, 'db_name', None, None, ), # 1 - (2, TType.STRING, 'tbl_name', None, None, ), # 2 - (3, TType.STRING, 'index_name', None, None, ), # 3 + (1, TType.STRING, 'dbName', None, None, ), # 1 + (2, TType.STRING, 'funcName', None, None, ), # 2 ) - def __init__(self, db_name=None, tbl_name=None, index_name=None,): - self.db_name = db_name - self.tbl_name = tbl_name - self.index_name = index_name + def __init__(self, dbName=None, funcName=None,): + self.dbName = dbName + self.funcName = funcName def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -16599,17 +19331,12 @@ def read(self, iprot): break if fid == 1: if ftype == TType.STRING: - self.db_name = iprot.readString(); + self.dbName = iprot.readString(); else: iprot.skip(ftype) elif fid == 2: if ftype == TType.STRING: - self.tbl_name = iprot.readString(); - else: - iprot.skip(ftype) - elif fid == 3: - if ftype == TType.STRING: - self.index_name = iprot.readString(); + self.funcName = iprot.readString(); else: iprot.skip(ftype) else: @@ -16621,18 +19348,14 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('get_index_by_name_args') - if self.db_name is not None: - oprot.writeFieldBegin('db_name', TType.STRING, 1) - oprot.writeString(self.db_name) - oprot.writeFieldEnd() - if self.tbl_name is not None: - oprot.writeFieldBegin('tbl_name', TType.STRING, 2) - oprot.writeString(self.tbl_name) + oprot.writeStructBegin('drop_function_args') + if self.dbName is not None: + oprot.writeFieldBegin('dbName', TType.STRING, 1) + oprot.writeString(self.dbName) oprot.writeFieldEnd() - if self.index_name is not None: - oprot.writeFieldBegin('index_name', TType.STRING, 3) - oprot.writeString(self.index_name) + if self.funcName is not None: + oprot.writeFieldBegin('funcName', TType.STRING, 2) + oprot.writeString(self.funcName) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -16652,24 +19375,22 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class get_index_by_name_result: +class drop_function_result: """ Attributes: - - success - o1 - - o2 + - o3 """ thrift_spec = ( - (0, TType.STRUCT, 'success', (Index, Index.thrift_spec), None, ), # 0 - (1, TType.STRUCT, 'o1', (MetaException, MetaException.thrift_spec), None, ), # 1 - (2, TType.STRUCT, 'o2', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 2 + None, # 0 + (1, TType.STRUCT, 'o1', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 1 + (2, TType.STRUCT, 'o3', (MetaException, MetaException.thrift_spec), None, ), # 2 ) - def __init__(self, success=None, o1=None, o2=None,): - self.success = success + def __init__(self, o1=None, o3=None,): self.o1 = o1 - self.o2 = o2 + self.o3 = o3 def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -16680,22 +19401,16 @@ def read(self, iprot): (fname, ftype, fid) = iprot.readFieldBegin() if ftype == TType.STOP: break - if fid == 0: - if ftype == TType.STRUCT: - self.success = Index() - self.success.read(iprot) - else: - iprot.skip(ftype) - elif fid == 1: + if fid == 1: if ftype == TType.STRUCT: - self.o1 = MetaException() + self.o1 = NoSuchObjectException() self.o1.read(iprot) else: iprot.skip(ftype) elif fid == 2: if ftype == TType.STRUCT: - self.o2 = NoSuchObjectException() - self.o2.read(iprot) + self.o3 = MetaException() + self.o3.read(iprot) else: iprot.skip(ftype) else: @@ -16707,18 +19422,14 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('get_index_by_name_result') - if self.success is not None: - oprot.writeFieldBegin('success', TType.STRUCT, 0) - self.success.write(oprot) - oprot.writeFieldEnd() + oprot.writeStructBegin('drop_function_result') if self.o1 is not None: oprot.writeFieldBegin('o1', TType.STRUCT, 1) self.o1.write(oprot) oprot.writeFieldEnd() - if self.o2 is not None: - oprot.writeFieldBegin('o2', TType.STRUCT, 2) - self.o2.write(oprot) + if self.o3 is not None: + oprot.writeFieldBegin('o3', TType.STRUCT, 2) + self.o3.write(oprot) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -16738,25 +19449,25 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class get_indexes_args: +class alter_function_args: """ Attributes: - - db_name - - tbl_name - - max_indexes + - dbName + - funcName + - newFunc """ thrift_spec = ( None, # 0 - (1, TType.STRING, 'db_name', None, None, ), # 1 - (2, TType.STRING, 'tbl_name', None, None, ), # 2 - (3, TType.I16, 'max_indexes', None, -1, ), # 3 + (1, TType.STRING, 'dbName', None, None, ), # 1 + (2, TType.STRING, 'funcName', None, None, ), # 2 + (3, TType.STRUCT, 'newFunc', (Function, Function.thrift_spec), None, ), # 3 ) - def __init__(self, db_name=None, tbl_name=None, max_indexes=thrift_spec[3][4],): - self.db_name = db_name - self.tbl_name = tbl_name - self.max_indexes = max_indexes + def __init__(self, dbName=None, funcName=None, newFunc=None,): + self.dbName = dbName + self.funcName = funcName + self.newFunc = newFunc def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -16769,17 +19480,18 @@ def read(self, iprot): break if fid == 1: if ftype == TType.STRING: - self.db_name = iprot.readString(); + self.dbName = iprot.readString(); else: iprot.skip(ftype) elif fid == 2: if ftype == TType.STRING: - self.tbl_name = iprot.readString(); + self.funcName = iprot.readString(); else: iprot.skip(ftype) elif fid == 3: - if ftype == TType.I16: - self.max_indexes = iprot.readI16(); + if ftype == TType.STRUCT: + self.newFunc = Function() + self.newFunc.read(iprot) else: iprot.skip(ftype) else: @@ -16791,18 +19503,18 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('get_indexes_args') - if self.db_name is not None: - oprot.writeFieldBegin('db_name', TType.STRING, 1) - oprot.writeString(self.db_name) + oprot.writeStructBegin('alter_function_args') + if self.dbName is not None: + oprot.writeFieldBegin('dbName', TType.STRING, 1) + oprot.writeString(self.dbName) oprot.writeFieldEnd() - if self.tbl_name is not None: - oprot.writeFieldBegin('tbl_name', TType.STRING, 2) - oprot.writeString(self.tbl_name) + if self.funcName is not None: + oprot.writeFieldBegin('funcName', TType.STRING, 2) + oprot.writeString(self.funcName) oprot.writeFieldEnd() - if self.max_indexes is not None: - oprot.writeFieldBegin('max_indexes', TType.I16, 3) - oprot.writeI16(self.max_indexes) + if self.newFunc is not None: + oprot.writeFieldBegin('newFunc', TType.STRUCT, 3) + self.newFunc.write(oprot) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -16822,22 +19534,20 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class get_indexes_result: +class alter_function_result: """ Attributes: - - success - o1 - o2 """ thrift_spec = ( - (0, TType.LIST, 'success', (TType.STRUCT,(Index, Index.thrift_spec)), None, ), # 0 - (1, TType.STRUCT, 'o1', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 1 + None, # 0 + (1, TType.STRUCT, 'o1', (InvalidOperationException, InvalidOperationException.thrift_spec), None, ), # 1 (2, TType.STRUCT, 'o2', (MetaException, MetaException.thrift_spec), None, ), # 2 ) - def __init__(self, success=None, o1=None, o2=None,): - self.success = success + def __init__(self, o1=None, o2=None,): self.o1 = o1 self.o2 = o2 @@ -16850,20 +19560,9 @@ def read(self, iprot): (fname, ftype, fid) = iprot.readFieldBegin() if ftype == TType.STOP: break - if fid == 0: - if ftype == TType.LIST: - self.success = [] - (_etype613, _size610) = iprot.readListBegin() - for _i614 in xrange(_size610): - _elem615 = Index() - _elem615.read(iprot) - self.success.append(_elem615) - iprot.readListEnd() - else: - iprot.skip(ftype) - elif fid == 1: + if fid == 1: if ftype == TType.STRUCT: - self.o1 = NoSuchObjectException() + self.o1 = InvalidOperationException() self.o1.read(iprot) else: iprot.skip(ftype) @@ -16882,14 +19581,7 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('get_indexes_result') - if self.success is not None: - oprot.writeFieldBegin('success', TType.LIST, 0) - oprot.writeListBegin(TType.STRUCT, len(self.success)) - for iter616 in self.success: - iter616.write(oprot) - oprot.writeListEnd() - oprot.writeFieldEnd() + oprot.writeStructBegin('alter_function_result') if self.o1 is not None: oprot.writeFieldBegin('o1', TType.STRUCT, 1) self.o1.write(oprot) @@ -16916,25 +19608,22 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class get_index_names_args: +class get_functions_args: """ Attributes: - - db_name - - tbl_name - - max_indexes + - dbName + - pattern """ thrift_spec = ( None, # 0 - (1, TType.STRING, 'db_name', None, None, ), # 1 - (2, TType.STRING, 'tbl_name', None, None, ), # 2 - (3, TType.I16, 'max_indexes', None, -1, ), # 3 + (1, TType.STRING, 'dbName', None, None, ), # 1 + (2, TType.STRING, 'pattern', None, None, ), # 2 ) - def __init__(self, db_name=None, tbl_name=None, max_indexes=thrift_spec[3][4],): - self.db_name = db_name - self.tbl_name = tbl_name - self.max_indexes = max_indexes + def __init__(self, dbName=None, pattern=None,): + self.dbName = dbName + self.pattern = pattern def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -16947,17 +19636,12 @@ def read(self, iprot): break if fid == 1: if ftype == TType.STRING: - self.db_name = iprot.readString(); + self.dbName = iprot.readString(); else: iprot.skip(ftype) elif fid == 2: if ftype == TType.STRING: - self.tbl_name = iprot.readString(); - else: - iprot.skip(ftype) - elif fid == 3: - if ftype == TType.I16: - self.max_indexes = iprot.readI16(); + self.pattern = iprot.readString(); else: iprot.skip(ftype) else: @@ -16969,18 +19653,14 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('get_index_names_args') - if self.db_name is not None: - oprot.writeFieldBegin('db_name', TType.STRING, 1) - oprot.writeString(self.db_name) - oprot.writeFieldEnd() - if self.tbl_name is not None: - oprot.writeFieldBegin('tbl_name', TType.STRING, 2) - oprot.writeString(self.tbl_name) + oprot.writeStructBegin('get_functions_args') + if self.dbName is not None: + oprot.writeFieldBegin('dbName', TType.STRING, 1) + oprot.writeString(self.dbName) oprot.writeFieldEnd() - if self.max_indexes is not None: - oprot.writeFieldBegin('max_indexes', TType.I16, 3) - oprot.writeI16(self.max_indexes) + if self.pattern is not None: + oprot.writeFieldBegin('pattern', TType.STRING, 2) + oprot.writeString(self.pattern) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -17000,21 +19680,21 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class get_index_names_result: +class get_functions_result: """ Attributes: - success - - o2 + - o1 """ thrift_spec = ( (0, TType.LIST, 'success', (TType.STRING,None), None, ), # 0 - (1, TType.STRUCT, 'o2', (MetaException, MetaException.thrift_spec), None, ), # 1 + (1, TType.STRUCT, 'o1', (MetaException, MetaException.thrift_spec), None, ), # 1 ) - def __init__(self, success=None, o2=None,): + def __init__(self, success=None, o1=None,): self.success = success - self.o2 = o2 + self.o1 = o1 def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -17028,17 +19708,17 @@ def read(self, iprot): if fid == 0: if ftype == TType.LIST: self.success = [] - (_etype620, _size617) = iprot.readListBegin() - for _i621 in xrange(_size617): - _elem622 = iprot.readString(); - self.success.append(_elem622) + (_etype669, _size666) = iprot.readListBegin() + for _i670 in xrange(_size666): + _elem671 = iprot.readString(); + self.success.append(_elem671) iprot.readListEnd() else: iprot.skip(ftype) elif fid == 1: if ftype == TType.STRUCT: - self.o2 = MetaException() - self.o2.read(iprot) + self.o1 = MetaException() + self.o1.read(iprot) else: iprot.skip(ftype) else: @@ -17050,17 +19730,17 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('get_index_names_result') + oprot.writeStructBegin('get_functions_result') if self.success is not None: oprot.writeFieldBegin('success', TType.LIST, 0) oprot.writeListBegin(TType.STRING, len(self.success)) - for iter623 in self.success: - oprot.writeString(iter623) + for iter672 in self.success: + oprot.writeString(iter672) oprot.writeListEnd() oprot.writeFieldEnd() - if self.o2 is not None: - oprot.writeFieldBegin('o2', TType.STRUCT, 1) - self.o2.write(oprot) + if self.o1 is not None: + oprot.writeFieldBegin('o1', TType.STRUCT, 1) + self.o1.write(oprot) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -17080,19 +19760,22 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class update_table_column_statistics_args: +class get_function_args: """ Attributes: - - stats_obj + - dbName + - funcName """ thrift_spec = ( None, # 0 - (1, TType.STRUCT, 'stats_obj', (ColumnStatistics, ColumnStatistics.thrift_spec), None, ), # 1 + (1, TType.STRING, 'dbName', None, None, ), # 1 + (2, TType.STRING, 'funcName', None, None, ), # 2 ) - def __init__(self, stats_obj=None,): - self.stats_obj = stats_obj + def __init__(self, dbName=None, funcName=None,): + self.dbName = dbName + self.funcName = funcName def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -17104,9 +19787,13 @@ def read(self, iprot): if ftype == TType.STOP: break if fid == 1: - if ftype == TType.STRUCT: - self.stats_obj = ColumnStatistics() - self.stats_obj.read(iprot) + if ftype == TType.STRING: + self.dbName = iprot.readString(); + else: + iprot.skip(ftype) + elif fid == 2: + if ftype == TType.STRING: + self.funcName = iprot.readString(); else: iprot.skip(ftype) else: @@ -17118,10 +19805,14 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('update_table_column_statistics_args') - if self.stats_obj is not None: - oprot.writeFieldBegin('stats_obj', TType.STRUCT, 1) - self.stats_obj.write(oprot) + oprot.writeStructBegin('get_function_args') + if self.dbName is not None: + oprot.writeFieldBegin('dbName', TType.STRING, 1) + oprot.writeString(self.dbName) + oprot.writeFieldEnd() + if self.funcName is not None: + oprot.writeFieldBegin('funcName', TType.STRING, 2) + oprot.writeString(self.funcName) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -17141,30 +19832,24 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class update_table_column_statistics_result: +class get_function_result: """ Attributes: - success - o1 - o2 - - o3 - - o4 """ thrift_spec = ( - (0, TType.BOOL, 'success', None, None, ), # 0 - (1, TType.STRUCT, 'o1', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 1 - (2, TType.STRUCT, 'o2', (InvalidObjectException, InvalidObjectException.thrift_spec), None, ), # 2 - (3, TType.STRUCT, 'o3', (MetaException, MetaException.thrift_spec), None, ), # 3 - (4, TType.STRUCT, 'o4', (InvalidInputException, InvalidInputException.thrift_spec), None, ), # 4 + (0, TType.STRUCT, 'success', (Function, Function.thrift_spec), None, ), # 0 + (1, TType.STRUCT, 'o1', (MetaException, MetaException.thrift_spec), None, ), # 1 + (2, TType.STRUCT, 'o2', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 2 ) - def __init__(self, success=None, o1=None, o2=None, o3=None, o4=None,): + def __init__(self, success=None, o1=None, o2=None,): self.success = success self.o1 = o1 self.o2 = o2 - self.o3 = o3 - self.o4 = o4 def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -17176,34 +19861,23 @@ def read(self, iprot): if ftype == TType.STOP: break if fid == 0: - if ftype == TType.BOOL: - self.success = iprot.readBool(); + if ftype == TType.STRUCT: + self.success = Function() + self.success.read(iprot) else: iprot.skip(ftype) elif fid == 1: if ftype == TType.STRUCT: - self.o1 = NoSuchObjectException() + self.o1 = MetaException() self.o1.read(iprot) else: iprot.skip(ftype) elif fid == 2: if ftype == TType.STRUCT: - self.o2 = InvalidObjectException() + self.o2 = NoSuchObjectException() self.o2.read(iprot) else: iprot.skip(ftype) - elif fid == 3: - if ftype == TType.STRUCT: - self.o3 = MetaException() - self.o3.read(iprot) - else: - iprot.skip(ftype) - elif fid == 4: - if ftype == TType.STRUCT: - self.o4 = InvalidInputException() - self.o4.read(iprot) - else: - iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() @@ -17213,10 +19887,10 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('update_table_column_statistics_result') + oprot.writeStructBegin('get_function_result') if self.success is not None: - oprot.writeFieldBegin('success', TType.BOOL, 0) - oprot.writeBool(self.success) + oprot.writeFieldBegin('success', TType.STRUCT, 0) + self.success.write(oprot) oprot.writeFieldEnd() if self.o1 is not None: oprot.writeFieldBegin('o1', TType.STRUCT, 1) @@ -17226,14 +19900,6 @@ def write(self, oprot): oprot.writeFieldBegin('o2', TType.STRUCT, 2) self.o2.write(oprot) oprot.writeFieldEnd() - if self.o3 is not None: - oprot.writeFieldBegin('o3', TType.STRUCT, 3) - self.o3.write(oprot) - oprot.writeFieldEnd() - if self.o4 is not None: - oprot.writeFieldBegin('o4', TType.STRUCT, 4) - self.o4.write(oprot) - oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -17252,19 +19918,19 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class update_partition_column_statistics_args: +class create_role_args: """ Attributes: - - stats_obj + - role """ thrift_spec = ( None, # 0 - (1, TType.STRUCT, 'stats_obj', (ColumnStatistics, ColumnStatistics.thrift_spec), None, ), # 1 + (1, TType.STRUCT, 'role', (Role, Role.thrift_spec), None, ), # 1 ) - def __init__(self, stats_obj=None,): - self.stats_obj = stats_obj + def __init__(self, role=None,): + self.role = role def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -17277,8 +19943,8 @@ def read(self, iprot): break if fid == 1: if ftype == TType.STRUCT: - self.stats_obj = ColumnStatistics() - self.stats_obj.read(iprot) + self.role = Role() + self.role.read(iprot) else: iprot.skip(ftype) else: @@ -17290,10 +19956,10 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('update_partition_column_statistics_args') - if self.stats_obj is not None: - oprot.writeFieldBegin('stats_obj', TType.STRUCT, 1) - self.stats_obj.write(oprot) + oprot.writeStructBegin('create_role_args') + if self.role is not None: + oprot.writeFieldBegin('role', TType.STRUCT, 1) + self.role.write(oprot) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -17313,30 +19979,21 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class update_partition_column_statistics_result: +class create_role_result: """ Attributes: - success - o1 - - o2 - - o3 - - o4 """ thrift_spec = ( (0, TType.BOOL, 'success', None, None, ), # 0 - (1, TType.STRUCT, 'o1', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 1 - (2, TType.STRUCT, 'o2', (InvalidObjectException, InvalidObjectException.thrift_spec), None, ), # 2 - (3, TType.STRUCT, 'o3', (MetaException, MetaException.thrift_spec), None, ), # 3 - (4, TType.STRUCT, 'o4', (InvalidInputException, InvalidInputException.thrift_spec), None, ), # 4 + (1, TType.STRUCT, 'o1', (MetaException, MetaException.thrift_spec), None, ), # 1 ) - def __init__(self, success=None, o1=None, o2=None, o3=None, o4=None,): + def __init__(self, success=None, o1=None,): self.success = success self.o1 = o1 - self.o2 = o2 - self.o3 = o3 - self.o4 = o4 def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -17352,28 +20009,10 @@ def read(self, iprot): self.success = iprot.readBool(); else: iprot.skip(ftype) - elif fid == 1: - if ftype == TType.STRUCT: - self.o1 = NoSuchObjectException() - self.o1.read(iprot) - else: - iprot.skip(ftype) - elif fid == 2: - if ftype == TType.STRUCT: - self.o2 = InvalidObjectException() - self.o2.read(iprot) - else: - iprot.skip(ftype) - elif fid == 3: - if ftype == TType.STRUCT: - self.o3 = MetaException() - self.o3.read(iprot) - else: - iprot.skip(ftype) - elif fid == 4: + elif fid == 1: if ftype == TType.STRUCT: - self.o4 = InvalidInputException() - self.o4.read(iprot) + self.o1 = MetaException() + self.o1.read(iprot) else: iprot.skip(ftype) else: @@ -17385,7 +20024,7 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('update_partition_column_statistics_result') + oprot.writeStructBegin('create_role_result') if self.success is not None: oprot.writeFieldBegin('success', TType.BOOL, 0) oprot.writeBool(self.success) @@ -17394,18 +20033,6 @@ def write(self, oprot): oprot.writeFieldBegin('o1', TType.STRUCT, 1) self.o1.write(oprot) oprot.writeFieldEnd() - if self.o2 is not None: - oprot.writeFieldBegin('o2', TType.STRUCT, 2) - self.o2.write(oprot) - oprot.writeFieldEnd() - if self.o3 is not None: - oprot.writeFieldBegin('o3', TType.STRUCT, 3) - self.o3.write(oprot) - oprot.writeFieldEnd() - if self.o4 is not None: - oprot.writeFieldBegin('o4', TType.STRUCT, 4) - self.o4.write(oprot) - oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -17424,25 +20051,19 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class get_table_column_statistics_args: +class drop_role_args: """ Attributes: - - db_name - - tbl_name - - col_name + - role_name """ thrift_spec = ( None, # 0 - (1, TType.STRING, 'db_name', None, None, ), # 1 - (2, TType.STRING, 'tbl_name', None, None, ), # 2 - (3, TType.STRING, 'col_name', None, None, ), # 3 + (1, TType.STRING, 'role_name', None, None, ), # 1 ) - def __init__(self, db_name=None, tbl_name=None, col_name=None,): - self.db_name = db_name - self.tbl_name = tbl_name - self.col_name = col_name + def __init__(self, role_name=None,): + self.role_name = role_name def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -17455,17 +20076,7 @@ def read(self, iprot): break if fid == 1: if ftype == TType.STRING: - self.db_name = iprot.readString(); - else: - iprot.skip(ftype) - elif fid == 2: - if ftype == TType.STRING: - self.tbl_name = iprot.readString(); - else: - iprot.skip(ftype) - elif fid == 3: - if ftype == TType.STRING: - self.col_name = iprot.readString(); + self.role_name = iprot.readString(); else: iprot.skip(ftype) else: @@ -17477,18 +20088,10 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('get_table_column_statistics_args') - if self.db_name is not None: - oprot.writeFieldBegin('db_name', TType.STRING, 1) - oprot.writeString(self.db_name) - oprot.writeFieldEnd() - if self.tbl_name is not None: - oprot.writeFieldBegin('tbl_name', TType.STRING, 2) - oprot.writeString(self.tbl_name) - oprot.writeFieldEnd() - if self.col_name is not None: - oprot.writeFieldBegin('col_name', TType.STRING, 3) - oprot.writeString(self.col_name) + oprot.writeStructBegin('drop_role_args') + if self.role_name is not None: + oprot.writeFieldBegin('role_name', TType.STRING, 1) + oprot.writeString(self.role_name) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -17508,30 +20111,21 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class get_table_column_statistics_result: +class drop_role_result: """ Attributes: - success - o1 - - o2 - - o3 - - o4 """ thrift_spec = ( - (0, TType.STRUCT, 'success', (ColumnStatistics, ColumnStatistics.thrift_spec), None, ), # 0 - (1, TType.STRUCT, 'o1', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 1 - (2, TType.STRUCT, 'o2', (MetaException, MetaException.thrift_spec), None, ), # 2 - (3, TType.STRUCT, 'o3', (InvalidInputException, InvalidInputException.thrift_spec), None, ), # 3 - (4, TType.STRUCT, 'o4', (InvalidObjectException, InvalidObjectException.thrift_spec), None, ), # 4 + (0, TType.BOOL, 'success', None, None, ), # 0 + (1, TType.STRUCT, 'o1', (MetaException, MetaException.thrift_spec), None, ), # 1 ) - def __init__(self, success=None, o1=None, o2=None, o3=None, o4=None,): + def __init__(self, success=None, o1=None,): self.success = success self.o1 = o1 - self.o2 = o2 - self.o3 = o3 - self.o4 = o4 def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -17543,35 +20137,16 @@ def read(self, iprot): if ftype == TType.STOP: break if fid == 0: - if ftype == TType.STRUCT: - self.success = ColumnStatistics() - self.success.read(iprot) + if ftype == TType.BOOL: + self.success = iprot.readBool(); else: iprot.skip(ftype) elif fid == 1: if ftype == TType.STRUCT: - self.o1 = NoSuchObjectException() + self.o1 = MetaException() self.o1.read(iprot) else: iprot.skip(ftype) - elif fid == 2: - if ftype == TType.STRUCT: - self.o2 = MetaException() - self.o2.read(iprot) - else: - iprot.skip(ftype) - elif fid == 3: - if ftype == TType.STRUCT: - self.o3 = InvalidInputException() - self.o3.read(iprot) - else: - iprot.skip(ftype) - elif fid == 4: - if ftype == TType.STRUCT: - self.o4 = InvalidObjectException() - self.o4.read(iprot) - else: - iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() @@ -17581,27 +20156,15 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('get_table_column_statistics_result') + oprot.writeStructBegin('drop_role_result') if self.success is not None: - oprot.writeFieldBegin('success', TType.STRUCT, 0) - self.success.write(oprot) + oprot.writeFieldBegin('success', TType.BOOL, 0) + oprot.writeBool(self.success) oprot.writeFieldEnd() if self.o1 is not None: oprot.writeFieldBegin('o1', TType.STRUCT, 1) self.o1.write(oprot) oprot.writeFieldEnd() - if self.o2 is not None: - oprot.writeFieldBegin('o2', TType.STRUCT, 2) - self.o2.write(oprot) - oprot.writeFieldEnd() - if self.o3 is not None: - oprot.writeFieldBegin('o3', TType.STRUCT, 3) - self.o3.write(oprot) - oprot.writeFieldEnd() - if self.o4 is not None: - oprot.writeFieldBegin('o4', TType.STRUCT, 4) - self.o4.write(oprot) - oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -17620,29 +20183,11 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class get_partition_column_statistics_args: - """ - Attributes: - - db_name - - tbl_name - - part_name - - col_name - """ +class get_role_names_args: thrift_spec = ( - None, # 0 - (1, TType.STRING, 'db_name', None, None, ), # 1 - (2, TType.STRING, 'tbl_name', None, None, ), # 2 - (3, TType.STRING, 'part_name', None, None, ), # 3 - (4, TType.STRING, 'col_name', None, None, ), # 4 ) - def __init__(self, db_name=None, tbl_name=None, part_name=None, col_name=None,): - self.db_name = db_name - self.tbl_name = tbl_name - self.part_name = part_name - self.col_name = col_name - def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) @@ -17652,26 +20197,6 @@ def read(self, iprot): (fname, ftype, fid) = iprot.readFieldBegin() if ftype == TType.STOP: break - if fid == 1: - if ftype == TType.STRING: - self.db_name = iprot.readString(); - else: - iprot.skip(ftype) - elif fid == 2: - if ftype == TType.STRING: - self.tbl_name = iprot.readString(); - else: - iprot.skip(ftype) - elif fid == 3: - if ftype == TType.STRING: - self.part_name = iprot.readString(); - else: - iprot.skip(ftype) - elif fid == 4: - if ftype == TType.STRING: - self.col_name = iprot.readString(); - else: - iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() @@ -17681,23 +20206,7 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('get_partition_column_statistics_args') - if self.db_name is not None: - oprot.writeFieldBegin('db_name', TType.STRING, 1) - oprot.writeString(self.db_name) - oprot.writeFieldEnd() - if self.tbl_name is not None: - oprot.writeFieldBegin('tbl_name', TType.STRING, 2) - oprot.writeString(self.tbl_name) - oprot.writeFieldEnd() - if self.part_name is not None: - oprot.writeFieldBegin('part_name', TType.STRING, 3) - oprot.writeString(self.part_name) - oprot.writeFieldEnd() - if self.col_name is not None: - oprot.writeFieldBegin('col_name', TType.STRING, 4) - oprot.writeString(self.col_name) - oprot.writeFieldEnd() + oprot.writeStructBegin('get_role_names_args') oprot.writeFieldStop() oprot.writeStructEnd() @@ -17716,30 +20225,21 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class get_partition_column_statistics_result: +class get_role_names_result: """ Attributes: - success - o1 - - o2 - - o3 - - o4 """ thrift_spec = ( - (0, TType.STRUCT, 'success', (ColumnStatistics, ColumnStatistics.thrift_spec), None, ), # 0 - (1, TType.STRUCT, 'o1', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 1 - (2, TType.STRUCT, 'o2', (MetaException, MetaException.thrift_spec), None, ), # 2 - (3, TType.STRUCT, 'o3', (InvalidInputException, InvalidInputException.thrift_spec), None, ), # 3 - (4, TType.STRUCT, 'o4', (InvalidObjectException, InvalidObjectException.thrift_spec), None, ), # 4 + (0, TType.LIST, 'success', (TType.STRING,None), None, ), # 0 + (1, TType.STRUCT, 'o1', (MetaException, MetaException.thrift_spec), None, ), # 1 ) - def __init__(self, success=None, o1=None, o2=None, o3=None, o4=None,): + def __init__(self, success=None, o1=None,): self.success = success self.o1 = o1 - self.o2 = o2 - self.o3 = o3 - self.o4 = o4 def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -17751,35 +20251,21 @@ def read(self, iprot): if ftype == TType.STOP: break if fid == 0: - if ftype == TType.STRUCT: - self.success = ColumnStatistics() - self.success.read(iprot) + if ftype == TType.LIST: + self.success = [] + (_etype676, _size673) = iprot.readListBegin() + for _i677 in xrange(_size673): + _elem678 = iprot.readString(); + self.success.append(_elem678) + iprot.readListEnd() else: iprot.skip(ftype) elif fid == 1: if ftype == TType.STRUCT: - self.o1 = NoSuchObjectException() + self.o1 = MetaException() self.o1.read(iprot) else: iprot.skip(ftype) - elif fid == 2: - if ftype == TType.STRUCT: - self.o2 = MetaException() - self.o2.read(iprot) - else: - iprot.skip(ftype) - elif fid == 3: - if ftype == TType.STRUCT: - self.o3 = InvalidInputException() - self.o3.read(iprot) - else: - iprot.skip(ftype) - elif fid == 4: - if ftype == TType.STRUCT: - self.o4 = InvalidObjectException() - self.o4.read(iprot) - else: - iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() @@ -17789,27 +20275,18 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('get_partition_column_statistics_result') + oprot.writeStructBegin('get_role_names_result') if self.success is not None: - oprot.writeFieldBegin('success', TType.STRUCT, 0) - self.success.write(oprot) + oprot.writeFieldBegin('success', TType.LIST, 0) + oprot.writeListBegin(TType.STRING, len(self.success)) + for iter679 in self.success: + oprot.writeString(iter679) + oprot.writeListEnd() oprot.writeFieldEnd() if self.o1 is not None: oprot.writeFieldBegin('o1', TType.STRUCT, 1) self.o1.write(oprot) oprot.writeFieldEnd() - if self.o2 is not None: - oprot.writeFieldBegin('o2', TType.STRUCT, 2) - self.o2.write(oprot) - oprot.writeFieldEnd() - if self.o3 is not None: - oprot.writeFieldBegin('o3', TType.STRUCT, 3) - self.o3.write(oprot) - oprot.writeFieldEnd() - if self.o4 is not None: - oprot.writeFieldBegin('o4', TType.STRUCT, 4) - self.o4.write(oprot) - oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -17828,19 +20305,34 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class get_table_statistics_req_args: +class grant_role_args: """ Attributes: - - request + - role_name + - principal_name + - principal_type + - grantor + - grantorType + - grant_option """ thrift_spec = ( None, # 0 - (1, TType.STRUCT, 'request', (TableStatsRequest, TableStatsRequest.thrift_spec), None, ), # 1 + (1, TType.STRING, 'role_name', None, None, ), # 1 + (2, TType.STRING, 'principal_name', None, None, ), # 2 + (3, TType.I32, 'principal_type', None, None, ), # 3 + (4, TType.STRING, 'grantor', None, None, ), # 4 + (5, TType.I32, 'grantorType', None, None, ), # 5 + (6, TType.BOOL, 'grant_option', None, None, ), # 6 ) - def __init__(self, request=None,): - self.request = request + def __init__(self, role_name=None, principal_name=None, principal_type=None, grantor=None, grantorType=None, grant_option=None,): + self.role_name = role_name + self.principal_name = principal_name + self.principal_type = principal_type + self.grantor = grantor + self.grantorType = grantorType + self.grant_option = grant_option def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -17852,9 +20344,33 @@ def read(self, iprot): if ftype == TType.STOP: break if fid == 1: - if ftype == TType.STRUCT: - self.request = TableStatsRequest() - self.request.read(iprot) + if ftype == TType.STRING: + self.role_name = iprot.readString(); + else: + iprot.skip(ftype) + elif fid == 2: + if ftype == TType.STRING: + self.principal_name = iprot.readString(); + else: + iprot.skip(ftype) + elif fid == 3: + if ftype == TType.I32: + self.principal_type = iprot.readI32(); + else: + iprot.skip(ftype) + elif fid == 4: + if ftype == TType.STRING: + self.grantor = iprot.readString(); + else: + iprot.skip(ftype) + elif fid == 5: + if ftype == TType.I32: + self.grantorType = iprot.readI32(); + else: + iprot.skip(ftype) + elif fid == 6: + if ftype == TType.BOOL: + self.grant_option = iprot.readBool(); else: iprot.skip(ftype) else: @@ -17866,10 +20382,30 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('get_table_statistics_req_args') - if self.request is not None: - oprot.writeFieldBegin('request', TType.STRUCT, 1) - self.request.write(oprot) + oprot.writeStructBegin('grant_role_args') + if self.role_name is not None: + oprot.writeFieldBegin('role_name', TType.STRING, 1) + oprot.writeString(self.role_name) + oprot.writeFieldEnd() + if self.principal_name is not None: + oprot.writeFieldBegin('principal_name', TType.STRING, 2) + oprot.writeString(self.principal_name) + oprot.writeFieldEnd() + if self.principal_type is not None: + oprot.writeFieldBegin('principal_type', TType.I32, 3) + oprot.writeI32(self.principal_type) + oprot.writeFieldEnd() + if self.grantor is not None: + oprot.writeFieldBegin('grantor', TType.STRING, 4) + oprot.writeString(self.grantor) + oprot.writeFieldEnd() + if self.grantorType is not None: + oprot.writeFieldBegin('grantorType', TType.I32, 5) + oprot.writeI32(self.grantorType) + oprot.writeFieldEnd() + if self.grant_option is not None: + oprot.writeFieldBegin('grant_option', TType.BOOL, 6) + oprot.writeBool(self.grant_option) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -17889,24 +20425,21 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class get_table_statistics_req_result: +class grant_role_result: """ Attributes: - success - o1 - - o2 """ thrift_spec = ( - (0, TType.STRUCT, 'success', (TableStatsResult, TableStatsResult.thrift_spec), None, ), # 0 - (1, TType.STRUCT, 'o1', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 1 - (2, TType.STRUCT, 'o2', (MetaException, MetaException.thrift_spec), None, ), # 2 + (0, TType.BOOL, 'success', None, None, ), # 0 + (1, TType.STRUCT, 'o1', (MetaException, MetaException.thrift_spec), None, ), # 1 ) - def __init__(self, success=None, o1=None, o2=None,): + def __init__(self, success=None, o1=None,): self.success = success self.o1 = o1 - self.o2 = o2 def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -17918,23 +20451,16 @@ def read(self, iprot): if ftype == TType.STOP: break if fid == 0: - if ftype == TType.STRUCT: - self.success = TableStatsResult() - self.success.read(iprot) + if ftype == TType.BOOL: + self.success = iprot.readBool(); else: iprot.skip(ftype) elif fid == 1: if ftype == TType.STRUCT: - self.o1 = NoSuchObjectException() + self.o1 = MetaException() self.o1.read(iprot) else: iprot.skip(ftype) - elif fid == 2: - if ftype == TType.STRUCT: - self.o2 = MetaException() - self.o2.read(iprot) - else: - iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() @@ -17944,19 +20470,15 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('get_table_statistics_req_result') + oprot.writeStructBegin('grant_role_result') if self.success is not None: - oprot.writeFieldBegin('success', TType.STRUCT, 0) - self.success.write(oprot) + oprot.writeFieldBegin('success', TType.BOOL, 0) + oprot.writeBool(self.success) oprot.writeFieldEnd() if self.o1 is not None: oprot.writeFieldBegin('o1', TType.STRUCT, 1) self.o1.write(oprot) oprot.writeFieldEnd() - if self.o2 is not None: - oprot.writeFieldBegin('o2', TType.STRUCT, 2) - self.o2.write(oprot) - oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -17975,19 +20497,25 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class get_partitions_statistics_req_args: +class revoke_role_args: """ Attributes: - - request + - role_name + - principal_name + - principal_type """ thrift_spec = ( None, # 0 - (1, TType.STRUCT, 'request', (PartitionsStatsRequest, PartitionsStatsRequest.thrift_spec), None, ), # 1 + (1, TType.STRING, 'role_name', None, None, ), # 1 + (2, TType.STRING, 'principal_name', None, None, ), # 2 + (3, TType.I32, 'principal_type', None, None, ), # 3 ) - def __init__(self, request=None,): - self.request = request + def __init__(self, role_name=None, principal_name=None, principal_type=None,): + self.role_name = role_name + self.principal_name = principal_name + self.principal_type = principal_type def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -17999,9 +20527,18 @@ def read(self, iprot): if ftype == TType.STOP: break if fid == 1: - if ftype == TType.STRUCT: - self.request = PartitionsStatsRequest() - self.request.read(iprot) + if ftype == TType.STRING: + self.role_name = iprot.readString(); + else: + iprot.skip(ftype) + elif fid == 2: + if ftype == TType.STRING: + self.principal_name = iprot.readString(); + else: + iprot.skip(ftype) + elif fid == 3: + if ftype == TType.I32: + self.principal_type = iprot.readI32(); else: iprot.skip(ftype) else: @@ -18013,10 +20550,18 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('get_partitions_statistics_req_args') - if self.request is not None: - oprot.writeFieldBegin('request', TType.STRUCT, 1) - self.request.write(oprot) + oprot.writeStructBegin('revoke_role_args') + if self.role_name is not None: + oprot.writeFieldBegin('role_name', TType.STRING, 1) + oprot.writeString(self.role_name) + oprot.writeFieldEnd() + if self.principal_name is not None: + oprot.writeFieldBegin('principal_name', TType.STRING, 2) + oprot.writeString(self.principal_name) + oprot.writeFieldEnd() + if self.principal_type is not None: + oprot.writeFieldBegin('principal_type', TType.I32, 3) + oprot.writeI32(self.principal_type) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -18036,24 +20581,21 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class get_partitions_statistics_req_result: +class revoke_role_result: """ Attributes: - success - o1 - - o2 """ thrift_spec = ( - (0, TType.STRUCT, 'success', (PartitionsStatsResult, PartitionsStatsResult.thrift_spec), None, ), # 0 - (1, TType.STRUCT, 'o1', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 1 - (2, TType.STRUCT, 'o2', (MetaException, MetaException.thrift_spec), None, ), # 2 + (0, TType.BOOL, 'success', None, None, ), # 0 + (1, TType.STRUCT, 'o1', (MetaException, MetaException.thrift_spec), None, ), # 1 ) - def __init__(self, success=None, o1=None, o2=None,): + def __init__(self, success=None, o1=None,): self.success = success self.o1 = o1 - self.o2 = o2 def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -18065,23 +20607,16 @@ def read(self, iprot): if ftype == TType.STOP: break if fid == 0: - if ftype == TType.STRUCT: - self.success = PartitionsStatsResult() - self.success.read(iprot) + if ftype == TType.BOOL: + self.success = iprot.readBool(); else: iprot.skip(ftype) elif fid == 1: if ftype == TType.STRUCT: - self.o1 = NoSuchObjectException() + self.o1 = MetaException() self.o1.read(iprot) else: iprot.skip(ftype) - elif fid == 2: - if ftype == TType.STRUCT: - self.o2 = MetaException() - self.o2.read(iprot) - else: - iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() @@ -18091,19 +20626,15 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('get_partitions_statistics_req_result') + oprot.writeStructBegin('revoke_role_result') if self.success is not None: - oprot.writeFieldBegin('success', TType.STRUCT, 0) - self.success.write(oprot) + oprot.writeFieldBegin('success', TType.BOOL, 0) + oprot.writeBool(self.success) oprot.writeFieldEnd() if self.o1 is not None: oprot.writeFieldBegin('o1', TType.STRUCT, 1) self.o1.write(oprot) oprot.writeFieldEnd() - if self.o2 is not None: - oprot.writeFieldBegin('o2', TType.STRUCT, 2) - self.o2.write(oprot) - oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -18122,28 +20653,22 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class delete_partition_column_statistics_args: +class list_roles_args: """ Attributes: - - db_name - - tbl_name - - part_name - - col_name + - principal_name + - principal_type """ thrift_spec = ( None, # 0 - (1, TType.STRING, 'db_name', None, None, ), # 1 - (2, TType.STRING, 'tbl_name', None, None, ), # 2 - (3, TType.STRING, 'part_name', None, None, ), # 3 - (4, TType.STRING, 'col_name', None, None, ), # 4 + (1, TType.STRING, 'principal_name', None, None, ), # 1 + (2, TType.I32, 'principal_type', None, None, ), # 2 ) - def __init__(self, db_name=None, tbl_name=None, part_name=None, col_name=None,): - self.db_name = db_name - self.tbl_name = tbl_name - self.part_name = part_name - self.col_name = col_name + def __init__(self, principal_name=None, principal_type=None,): + self.principal_name = principal_name + self.principal_type = principal_type def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -18156,22 +20681,12 @@ def read(self, iprot): break if fid == 1: if ftype == TType.STRING: - self.db_name = iprot.readString(); + self.principal_name = iprot.readString(); else: iprot.skip(ftype) elif fid == 2: - if ftype == TType.STRING: - self.tbl_name = iprot.readString(); - else: - iprot.skip(ftype) - elif fid == 3: - if ftype == TType.STRING: - self.part_name = iprot.readString(); - else: - iprot.skip(ftype) - elif fid == 4: - if ftype == TType.STRING: - self.col_name = iprot.readString(); + if ftype == TType.I32: + self.principal_type = iprot.readI32(); else: iprot.skip(ftype) else: @@ -18183,22 +20698,14 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('delete_partition_column_statistics_args') - if self.db_name is not None: - oprot.writeFieldBegin('db_name', TType.STRING, 1) - oprot.writeString(self.db_name) - oprot.writeFieldEnd() - if self.tbl_name is not None: - oprot.writeFieldBegin('tbl_name', TType.STRING, 2) - oprot.writeString(self.tbl_name) - oprot.writeFieldEnd() - if self.part_name is not None: - oprot.writeFieldBegin('part_name', TType.STRING, 3) - oprot.writeString(self.part_name) + oprot.writeStructBegin('list_roles_args') + if self.principal_name is not None: + oprot.writeFieldBegin('principal_name', TType.STRING, 1) + oprot.writeString(self.principal_name) oprot.writeFieldEnd() - if self.col_name is not None: - oprot.writeFieldBegin('col_name', TType.STRING, 4) - oprot.writeString(self.col_name) + if self.principal_type is not None: + oprot.writeFieldBegin('principal_type', TType.I32, 2) + oprot.writeI32(self.principal_type) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -18218,30 +20725,21 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class delete_partition_column_statistics_result: +class list_roles_result: """ Attributes: - success - o1 - - o2 - - o3 - - o4 """ thrift_spec = ( - (0, TType.BOOL, 'success', None, None, ), # 0 - (1, TType.STRUCT, 'o1', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 1 - (2, TType.STRUCT, 'o2', (MetaException, MetaException.thrift_spec), None, ), # 2 - (3, TType.STRUCT, 'o3', (InvalidObjectException, InvalidObjectException.thrift_spec), None, ), # 3 - (4, TType.STRUCT, 'o4', (InvalidInputException, InvalidInputException.thrift_spec), None, ), # 4 + (0, TType.LIST, 'success', (TType.STRUCT,(Role, Role.thrift_spec)), None, ), # 0 + (1, TType.STRUCT, 'o1', (MetaException, MetaException.thrift_spec), None, ), # 1 ) - def __init__(self, success=None, o1=None, o2=None, o3=None, o4=None,): + def __init__(self, success=None, o1=None,): self.success = success self.o1 = o1 - self.o2 = o2 - self.o3 = o3 - self.o4 = o4 def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -18253,34 +20751,22 @@ def read(self, iprot): if ftype == TType.STOP: break if fid == 0: - if ftype == TType.BOOL: - self.success = iprot.readBool(); + if ftype == TType.LIST: + self.success = [] + (_etype683, _size680) = iprot.readListBegin() + for _i684 in xrange(_size680): + _elem685 = Role() + _elem685.read(iprot) + self.success.append(_elem685) + iprot.readListEnd() else: iprot.skip(ftype) elif fid == 1: if ftype == TType.STRUCT: - self.o1 = NoSuchObjectException() + self.o1 = MetaException() self.o1.read(iprot) else: iprot.skip(ftype) - elif fid == 2: - if ftype == TType.STRUCT: - self.o2 = MetaException() - self.o2.read(iprot) - else: - iprot.skip(ftype) - elif fid == 3: - if ftype == TType.STRUCT: - self.o3 = InvalidObjectException() - self.o3.read(iprot) - else: - iprot.skip(ftype) - elif fid == 4: - if ftype == TType.STRUCT: - self.o4 = InvalidInputException() - self.o4.read(iprot) - else: - iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() @@ -18290,27 +20776,18 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('delete_partition_column_statistics_result') + oprot.writeStructBegin('list_roles_result') if self.success is not None: - oprot.writeFieldBegin('success', TType.BOOL, 0) - oprot.writeBool(self.success) + oprot.writeFieldBegin('success', TType.LIST, 0) + oprot.writeListBegin(TType.STRUCT, len(self.success)) + for iter686 in self.success: + iter686.write(oprot) + oprot.writeListEnd() oprot.writeFieldEnd() if self.o1 is not None: oprot.writeFieldBegin('o1', TType.STRUCT, 1) self.o1.write(oprot) oprot.writeFieldEnd() - if self.o2 is not None: - oprot.writeFieldBegin('o2', TType.STRUCT, 2) - self.o2.write(oprot) - oprot.writeFieldEnd() - if self.o3 is not None: - oprot.writeFieldBegin('o3', TType.STRUCT, 3) - self.o3.write(oprot) - oprot.writeFieldEnd() - if self.o4 is not None: - oprot.writeFieldBegin('o4', TType.STRUCT, 4) - self.o4.write(oprot) - oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -18329,25 +20806,25 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class delete_table_column_statistics_args: +class get_privilege_set_args: """ Attributes: - - db_name - - tbl_name - - col_name + - hiveObject + - user_name + - group_names """ thrift_spec = ( None, # 0 - (1, TType.STRING, 'db_name', None, None, ), # 1 - (2, TType.STRING, 'tbl_name', None, None, ), # 2 - (3, TType.STRING, 'col_name', None, None, ), # 3 + (1, TType.STRUCT, 'hiveObject', (HiveObjectRef, HiveObjectRef.thrift_spec), None, ), # 1 + (2, TType.STRING, 'user_name', None, None, ), # 2 + (3, TType.LIST, 'group_names', (TType.STRING,None), None, ), # 3 ) - def __init__(self, db_name=None, tbl_name=None, col_name=None,): - self.db_name = db_name - self.tbl_name = tbl_name - self.col_name = col_name + def __init__(self, hiveObject=None, user_name=None, group_names=None,): + self.hiveObject = hiveObject + self.user_name = user_name + self.group_names = group_names def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -18359,18 +20836,24 @@ def read(self, iprot): if ftype == TType.STOP: break if fid == 1: - if ftype == TType.STRING: - self.db_name = iprot.readString(); + if ftype == TType.STRUCT: + self.hiveObject = HiveObjectRef() + self.hiveObject.read(iprot) else: iprot.skip(ftype) elif fid == 2: if ftype == TType.STRING: - self.tbl_name = iprot.readString(); + self.user_name = iprot.readString(); else: iprot.skip(ftype) elif fid == 3: - if ftype == TType.STRING: - self.col_name = iprot.readString(); + if ftype == TType.LIST: + self.group_names = [] + (_etype690, _size687) = iprot.readListBegin() + for _i691 in xrange(_size687): + _elem692 = iprot.readString(); + self.group_names.append(_elem692) + iprot.readListEnd() else: iprot.skip(ftype) else: @@ -18382,18 +20865,21 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('delete_table_column_statistics_args') - if self.db_name is not None: - oprot.writeFieldBegin('db_name', TType.STRING, 1) - oprot.writeString(self.db_name) + oprot.writeStructBegin('get_privilege_set_args') + if self.hiveObject is not None: + oprot.writeFieldBegin('hiveObject', TType.STRUCT, 1) + self.hiveObject.write(oprot) oprot.writeFieldEnd() - if self.tbl_name is not None: - oprot.writeFieldBegin('tbl_name', TType.STRING, 2) - oprot.writeString(self.tbl_name) + if self.user_name is not None: + oprot.writeFieldBegin('user_name', TType.STRING, 2) + oprot.writeString(self.user_name) oprot.writeFieldEnd() - if self.col_name is not None: - oprot.writeFieldBegin('col_name', TType.STRING, 3) - oprot.writeString(self.col_name) + if self.group_names is not None: + oprot.writeFieldBegin('group_names', TType.LIST, 3) + oprot.writeListBegin(TType.STRING, len(self.group_names)) + for iter693 in self.group_names: + oprot.writeString(iter693) + oprot.writeListEnd() oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -18413,30 +20899,21 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class delete_table_column_statistics_result: +class get_privilege_set_result: """ Attributes: - success - o1 - - o2 - - o3 - - o4 """ thrift_spec = ( - (0, TType.BOOL, 'success', None, None, ), # 0 - (1, TType.STRUCT, 'o1', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 1 - (2, TType.STRUCT, 'o2', (MetaException, MetaException.thrift_spec), None, ), # 2 - (3, TType.STRUCT, 'o3', (InvalidObjectException, InvalidObjectException.thrift_spec), None, ), # 3 - (4, TType.STRUCT, 'o4', (InvalidInputException, InvalidInputException.thrift_spec), None, ), # 4 + (0, TType.STRUCT, 'success', (PrincipalPrivilegeSet, PrincipalPrivilegeSet.thrift_spec), None, ), # 0 + (1, TType.STRUCT, 'o1', (MetaException, MetaException.thrift_spec), None, ), # 1 ) - def __init__(self, success=None, o1=None, o2=None, o3=None, o4=None,): + def __init__(self, success=None, o1=None,): self.success = success self.o1 = o1 - self.o2 = o2 - self.o3 = o3 - self.o4 = o4 def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -18448,34 +20925,17 @@ def read(self, iprot): if ftype == TType.STOP: break if fid == 0: - if ftype == TType.BOOL: - self.success = iprot.readBool(); + if ftype == TType.STRUCT: + self.success = PrincipalPrivilegeSet() + self.success.read(iprot) else: iprot.skip(ftype) elif fid == 1: if ftype == TType.STRUCT: - self.o1 = NoSuchObjectException() + self.o1 = MetaException() self.o1.read(iprot) else: iprot.skip(ftype) - elif fid == 2: - if ftype == TType.STRUCT: - self.o2 = MetaException() - self.o2.read(iprot) - else: - iprot.skip(ftype) - elif fid == 3: - if ftype == TType.STRUCT: - self.o3 = InvalidObjectException() - self.o3.read(iprot) - else: - iprot.skip(ftype) - elif fid == 4: - if ftype == TType.STRUCT: - self.o4 = InvalidInputException() - self.o4.read(iprot) - else: - iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() @@ -18485,27 +20945,15 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('delete_table_column_statistics_result') + oprot.writeStructBegin('get_privilege_set_result') if self.success is not None: - oprot.writeFieldBegin('success', TType.BOOL, 0) - oprot.writeBool(self.success) + oprot.writeFieldBegin('success', TType.STRUCT, 0) + self.success.write(oprot) oprot.writeFieldEnd() if self.o1 is not None: oprot.writeFieldBegin('o1', TType.STRUCT, 1) self.o1.write(oprot) oprot.writeFieldEnd() - if self.o2 is not None: - oprot.writeFieldBegin('o2', TType.STRUCT, 2) - self.o2.write(oprot) - oprot.writeFieldEnd() - if self.o3 is not None: - oprot.writeFieldBegin('o3', TType.STRUCT, 3) - self.o3.write(oprot) - oprot.writeFieldEnd() - if self.o4 is not None: - oprot.writeFieldBegin('o4', TType.STRUCT, 4) - self.o4.write(oprot) - oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -18524,19 +20972,25 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class create_function_args: +class list_privileges_args: """ Attributes: - - func + - principal_name + - principal_type + - hiveObject """ thrift_spec = ( None, # 0 - (1, TType.STRUCT, 'func', (Function, Function.thrift_spec), None, ), # 1 + (1, TType.STRING, 'principal_name', None, None, ), # 1 + (2, TType.I32, 'principal_type', None, None, ), # 2 + (3, TType.STRUCT, 'hiveObject', (HiveObjectRef, HiveObjectRef.thrift_spec), None, ), # 3 ) - def __init__(self, func=None,): - self.func = func + def __init__(self, principal_name=None, principal_type=None, hiveObject=None,): + self.principal_name = principal_name + self.principal_type = principal_type + self.hiveObject = hiveObject def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -18548,9 +21002,19 @@ def read(self, iprot): if ftype == TType.STOP: break if fid == 1: + if ftype == TType.STRING: + self.principal_name = iprot.readString(); + else: + iprot.skip(ftype) + elif fid == 2: + if ftype == TType.I32: + self.principal_type = iprot.readI32(); + else: + iprot.skip(ftype) + elif fid == 3: if ftype == TType.STRUCT: - self.func = Function() - self.func.read(iprot) + self.hiveObject = HiveObjectRef() + self.hiveObject.read(iprot) else: iprot.skip(ftype) else: @@ -18562,10 +21026,18 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('create_function_args') - if self.func is not None: - oprot.writeFieldBegin('func', TType.STRUCT, 1) - self.func.write(oprot) + oprot.writeStructBegin('list_privileges_args') + if self.principal_name is not None: + oprot.writeFieldBegin('principal_name', TType.STRING, 1) + oprot.writeString(self.principal_name) + oprot.writeFieldEnd() + if self.principal_type is not None: + oprot.writeFieldBegin('principal_type', TType.I32, 2) + oprot.writeI32(self.principal_type) + oprot.writeFieldEnd() + if self.hiveObject is not None: + oprot.writeFieldBegin('hiveObject', TType.STRUCT, 3) + self.hiveObject.write(oprot) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -18585,28 +21057,21 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class create_function_result: +class list_privileges_result: """ Attributes: + - success - o1 - - o2 - - o3 - - o4 """ thrift_spec = ( - None, # 0 - (1, TType.STRUCT, 'o1', (AlreadyExistsException, AlreadyExistsException.thrift_spec), None, ), # 1 - (2, TType.STRUCT, 'o2', (InvalidObjectException, InvalidObjectException.thrift_spec), None, ), # 2 - (3, TType.STRUCT, 'o3', (MetaException, MetaException.thrift_spec), None, ), # 3 - (4, TType.STRUCT, 'o4', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 4 + (0, TType.LIST, 'success', (TType.STRUCT,(HiveObjectPrivilege, HiveObjectPrivilege.thrift_spec)), None, ), # 0 + (1, TType.STRUCT, 'o1', (MetaException, MetaException.thrift_spec), None, ), # 1 ) - def __init__(self, o1=None, o2=None, o3=None, o4=None,): + def __init__(self, success=None, o1=None,): + self.success = success self.o1 = o1 - self.o2 = o2 - self.o3 = o3 - self.o4 = o4 def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -18617,28 +21082,21 @@ def read(self, iprot): (fname, ftype, fid) = iprot.readFieldBegin() if ftype == TType.STOP: break - if fid == 1: - if ftype == TType.STRUCT: - self.o1 = AlreadyExistsException() - self.o1.read(iprot) - else: - iprot.skip(ftype) - elif fid == 2: - if ftype == TType.STRUCT: - self.o2 = InvalidObjectException() - self.o2.read(iprot) - else: - iprot.skip(ftype) - elif fid == 3: - if ftype == TType.STRUCT: - self.o3 = MetaException() - self.o3.read(iprot) + if fid == 0: + if ftype == TType.LIST: + self.success = [] + (_etype697, _size694) = iprot.readListBegin() + for _i698 in xrange(_size694): + _elem699 = HiveObjectPrivilege() + _elem699.read(iprot) + self.success.append(_elem699) + iprot.readListEnd() else: iprot.skip(ftype) - elif fid == 4: + elif fid == 1: if ftype == TType.STRUCT: - self.o4 = NoSuchObjectException() - self.o4.read(iprot) + self.o1 = MetaException() + self.o1.read(iprot) else: iprot.skip(ftype) else: @@ -18650,23 +21108,18 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('create_function_result') + oprot.writeStructBegin('list_privileges_result') + if self.success is not None: + oprot.writeFieldBegin('success', TType.LIST, 0) + oprot.writeListBegin(TType.STRUCT, len(self.success)) + for iter700 in self.success: + iter700.write(oprot) + oprot.writeListEnd() + oprot.writeFieldEnd() if self.o1 is not None: oprot.writeFieldBegin('o1', TType.STRUCT, 1) self.o1.write(oprot) oprot.writeFieldEnd() - if self.o2 is not None: - oprot.writeFieldBegin('o2', TType.STRUCT, 2) - self.o2.write(oprot) - oprot.writeFieldEnd() - if self.o3 is not None: - oprot.writeFieldBegin('o3', TType.STRUCT, 3) - self.o3.write(oprot) - oprot.writeFieldEnd() - if self.o4 is not None: - oprot.writeFieldBegin('o4', TType.STRUCT, 4) - self.o4.write(oprot) - oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -18685,22 +21138,19 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class drop_function_args: +class grant_privileges_args: """ Attributes: - - dbName - - funcName + - privileges """ thrift_spec = ( None, # 0 - (1, TType.STRING, 'dbName', None, None, ), # 1 - (2, TType.STRING, 'funcName', None, None, ), # 2 + (1, TType.STRUCT, 'privileges', (PrivilegeBag, PrivilegeBag.thrift_spec), None, ), # 1 ) - def __init__(self, dbName=None, funcName=None,): - self.dbName = dbName - self.funcName = funcName + def __init__(self, privileges=None,): + self.privileges = privileges def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -18712,13 +21162,9 @@ def read(self, iprot): if ftype == TType.STOP: break if fid == 1: - if ftype == TType.STRING: - self.dbName = iprot.readString(); - else: - iprot.skip(ftype) - elif fid == 2: - if ftype == TType.STRING: - self.funcName = iprot.readString(); + if ftype == TType.STRUCT: + self.privileges = PrivilegeBag() + self.privileges.read(iprot) else: iprot.skip(ftype) else: @@ -18730,14 +21176,10 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('drop_function_args') - if self.dbName is not None: - oprot.writeFieldBegin('dbName', TType.STRING, 1) - oprot.writeString(self.dbName) - oprot.writeFieldEnd() - if self.funcName is not None: - oprot.writeFieldBegin('funcName', TType.STRING, 2) - oprot.writeString(self.funcName) + oprot.writeStructBegin('grant_privileges_args') + if self.privileges is not None: + oprot.writeFieldBegin('privileges', TType.STRUCT, 1) + self.privileges.write(oprot) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -18757,22 +21199,21 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class drop_function_result: +class grant_privileges_result: """ Attributes: + - success - o1 - - o3 """ thrift_spec = ( - None, # 0 - (1, TType.STRUCT, 'o1', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 1 - (2, TType.STRUCT, 'o3', (MetaException, MetaException.thrift_spec), None, ), # 2 + (0, TType.BOOL, 'success', None, None, ), # 0 + (1, TType.STRUCT, 'o1', (MetaException, MetaException.thrift_spec), None, ), # 1 ) - def __init__(self, o1=None, o3=None,): + def __init__(self, success=None, o1=None,): + self.success = success self.o1 = o1 - self.o3 = o3 def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -18783,16 +21224,15 @@ def read(self, iprot): (fname, ftype, fid) = iprot.readFieldBegin() if ftype == TType.STOP: break - if fid == 1: - if ftype == TType.STRUCT: - self.o1 = NoSuchObjectException() - self.o1.read(iprot) + if fid == 0: + if ftype == TType.BOOL: + self.success = iprot.readBool(); else: iprot.skip(ftype) - elif fid == 2: + elif fid == 1: if ftype == TType.STRUCT: - self.o3 = MetaException() - self.o3.read(iprot) + self.o1 = MetaException() + self.o1.read(iprot) else: iprot.skip(ftype) else: @@ -18804,15 +21244,15 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('drop_function_result') + oprot.writeStructBegin('grant_privileges_result') + if self.success is not None: + oprot.writeFieldBegin('success', TType.BOOL, 0) + oprot.writeBool(self.success) + oprot.writeFieldEnd() if self.o1 is not None: oprot.writeFieldBegin('o1', TType.STRUCT, 1) self.o1.write(oprot) oprot.writeFieldEnd() - if self.o3 is not None: - oprot.writeFieldBegin('o3', TType.STRUCT, 2) - self.o3.write(oprot) - oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -18831,25 +21271,19 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class alter_function_args: +class revoke_privileges_args: """ Attributes: - - dbName - - funcName - - newFunc + - privileges """ thrift_spec = ( None, # 0 - (1, TType.STRING, 'dbName', None, None, ), # 1 - (2, TType.STRING, 'funcName', None, None, ), # 2 - (3, TType.STRUCT, 'newFunc', (Function, Function.thrift_spec), None, ), # 3 + (1, TType.STRUCT, 'privileges', (PrivilegeBag, PrivilegeBag.thrift_spec), None, ), # 1 ) - def __init__(self, dbName=None, funcName=None, newFunc=None,): - self.dbName = dbName - self.funcName = funcName - self.newFunc = newFunc + def __init__(self, privileges=None,): + self.privileges = privileges def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -18861,19 +21295,9 @@ def read(self, iprot): if ftype == TType.STOP: break if fid == 1: - if ftype == TType.STRING: - self.dbName = iprot.readString(); - else: - iprot.skip(ftype) - elif fid == 2: - if ftype == TType.STRING: - self.funcName = iprot.readString(); - else: - iprot.skip(ftype) - elif fid == 3: if ftype == TType.STRUCT: - self.newFunc = Function() - self.newFunc.read(iprot) + self.privileges = PrivilegeBag() + self.privileges.read(iprot) else: iprot.skip(ftype) else: @@ -18885,18 +21309,10 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('alter_function_args') - if self.dbName is not None: - oprot.writeFieldBegin('dbName', TType.STRING, 1) - oprot.writeString(self.dbName) - oprot.writeFieldEnd() - if self.funcName is not None: - oprot.writeFieldBegin('funcName', TType.STRING, 2) - oprot.writeString(self.funcName) - oprot.writeFieldEnd() - if self.newFunc is not None: - oprot.writeFieldBegin('newFunc', TType.STRUCT, 3) - self.newFunc.write(oprot) + oprot.writeStructBegin('revoke_privileges_args') + if self.privileges is not None: + oprot.writeFieldBegin('privileges', TType.STRUCT, 1) + self.privileges.write(oprot) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -18916,22 +21332,21 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class alter_function_result: +class revoke_privileges_result: """ Attributes: + - success - o1 - - o2 """ thrift_spec = ( - None, # 0 - (1, TType.STRUCT, 'o1', (InvalidOperationException, InvalidOperationException.thrift_spec), None, ), # 1 - (2, TType.STRUCT, 'o2', (MetaException, MetaException.thrift_spec), None, ), # 2 + (0, TType.BOOL, 'success', None, None, ), # 0 + (1, TType.STRUCT, 'o1', (MetaException, MetaException.thrift_spec), None, ), # 1 ) - def __init__(self, o1=None, o2=None,): + def __init__(self, success=None, o1=None,): + self.success = success self.o1 = o1 - self.o2 = o2 def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -18942,16 +21357,15 @@ def read(self, iprot): (fname, ftype, fid) = iprot.readFieldBegin() if ftype == TType.STOP: break - if fid == 1: - if ftype == TType.STRUCT: - self.o1 = InvalidOperationException() - self.o1.read(iprot) + if fid == 0: + if ftype == TType.BOOL: + self.success = iprot.readBool(); else: iprot.skip(ftype) - elif fid == 2: + elif fid == 1: if ftype == TType.STRUCT: - self.o2 = MetaException() - self.o2.read(iprot) + self.o1 = MetaException() + self.o1.read(iprot) else: iprot.skip(ftype) else: @@ -18963,15 +21377,15 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('alter_function_result') + oprot.writeStructBegin('revoke_privileges_result') + if self.success is not None: + oprot.writeFieldBegin('success', TType.BOOL, 0) + oprot.writeBool(self.success) + oprot.writeFieldEnd() if self.o1 is not None: oprot.writeFieldBegin('o1', TType.STRUCT, 1) self.o1.write(oprot) oprot.writeFieldEnd() - if self.o2 is not None: - oprot.writeFieldBegin('o2', TType.STRUCT, 2) - self.o2.write(oprot) - oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -18990,22 +21404,22 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class get_functions_args: +class set_ugi_args: """ Attributes: - - dbName - - pattern + - user_name + - group_names """ thrift_spec = ( None, # 0 - (1, TType.STRING, 'dbName', None, None, ), # 1 - (2, TType.STRING, 'pattern', None, None, ), # 2 + (1, TType.STRING, 'user_name', None, None, ), # 1 + (2, TType.LIST, 'group_names', (TType.STRING,None), None, ), # 2 ) - def __init__(self, dbName=None, pattern=None,): - self.dbName = dbName - self.pattern = pattern + def __init__(self, user_name=None, group_names=None,): + self.user_name = user_name + self.group_names = group_names def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -19018,12 +21432,17 @@ def read(self, iprot): break if fid == 1: if ftype == TType.STRING: - self.dbName = iprot.readString(); + self.user_name = iprot.readString(); else: iprot.skip(ftype) elif fid == 2: - if ftype == TType.STRING: - self.pattern = iprot.readString(); + if ftype == TType.LIST: + self.group_names = [] + (_etype704, _size701) = iprot.readListBegin() + for _i705 in xrange(_size701): + _elem706 = iprot.readString(); + self.group_names.append(_elem706) + iprot.readListEnd() else: iprot.skip(ftype) else: @@ -19035,14 +21454,17 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('get_functions_args') - if self.dbName is not None: - oprot.writeFieldBegin('dbName', TType.STRING, 1) - oprot.writeString(self.dbName) + oprot.writeStructBegin('set_ugi_args') + if self.user_name is not None: + oprot.writeFieldBegin('user_name', TType.STRING, 1) + oprot.writeString(self.user_name) oprot.writeFieldEnd() - if self.pattern is not None: - oprot.writeFieldBegin('pattern', TType.STRING, 2) - oprot.writeString(self.pattern) + if self.group_names is not None: + oprot.writeFieldBegin('group_names', TType.LIST, 2) + oprot.writeListBegin(TType.STRING, len(self.group_names)) + for iter707 in self.group_names: + oprot.writeString(iter707) + oprot.writeListEnd() oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -19062,7 +21484,7 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class get_functions_result: +class set_ugi_result: """ Attributes: - success @@ -19090,10 +21512,10 @@ def read(self, iprot): if fid == 0: if ftype == TType.LIST: self.success = [] - (_etype627, _size624) = iprot.readListBegin() - for _i628 in xrange(_size624): - _elem629 = iprot.readString(); - self.success.append(_elem629) + (_etype711, _size708) = iprot.readListBegin() + for _i712 in xrange(_size708): + _elem713 = iprot.readString(); + self.success.append(_elem713) iprot.readListEnd() else: iprot.skip(ftype) @@ -19112,12 +21534,12 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('get_functions_result') + oprot.writeStructBegin('set_ugi_result') if self.success is not None: oprot.writeFieldBegin('success', TType.LIST, 0) oprot.writeListBegin(TType.STRING, len(self.success)) - for iter630 in self.success: - oprot.writeString(iter630) + for iter714 in self.success: + oprot.writeString(iter714) oprot.writeListEnd() oprot.writeFieldEnd() if self.o1 is not None: @@ -19142,22 +21564,22 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class get_function_args: +class get_delegation_token_args: """ Attributes: - - dbName - - funcName + - token_owner + - renewer_kerberos_principal_name """ thrift_spec = ( None, # 0 - (1, TType.STRING, 'dbName', None, None, ), # 1 - (2, TType.STRING, 'funcName', None, None, ), # 2 + (1, TType.STRING, 'token_owner', None, None, ), # 1 + (2, TType.STRING, 'renewer_kerberos_principal_name', None, None, ), # 2 ) - def __init__(self, dbName=None, funcName=None,): - self.dbName = dbName - self.funcName = funcName + def __init__(self, token_owner=None, renewer_kerberos_principal_name=None,): + self.token_owner = token_owner + self.renewer_kerberos_principal_name = renewer_kerberos_principal_name def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -19170,12 +21592,12 @@ def read(self, iprot): break if fid == 1: if ftype == TType.STRING: - self.dbName = iprot.readString(); + self.token_owner = iprot.readString(); else: iprot.skip(ftype) elif fid == 2: if ftype == TType.STRING: - self.funcName = iprot.readString(); + self.renewer_kerberos_principal_name = iprot.readString(); else: iprot.skip(ftype) else: @@ -19187,14 +21609,14 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('get_function_args') - if self.dbName is not None: - oprot.writeFieldBegin('dbName', TType.STRING, 1) - oprot.writeString(self.dbName) + oprot.writeStructBegin('get_delegation_token_args') + if self.token_owner is not None: + oprot.writeFieldBegin('token_owner', TType.STRING, 1) + oprot.writeString(self.token_owner) oprot.writeFieldEnd() - if self.funcName is not None: - oprot.writeFieldBegin('funcName', TType.STRING, 2) - oprot.writeString(self.funcName) + if self.renewer_kerberos_principal_name is not None: + oprot.writeFieldBegin('renewer_kerberos_principal_name', TType.STRING, 2) + oprot.writeString(self.renewer_kerberos_principal_name) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -19214,24 +21636,21 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class get_function_result: +class get_delegation_token_result: """ Attributes: - success - o1 - - o2 """ thrift_spec = ( - (0, TType.STRUCT, 'success', (Function, Function.thrift_spec), None, ), # 0 + (0, TType.STRING, 'success', None, None, ), # 0 (1, TType.STRUCT, 'o1', (MetaException, MetaException.thrift_spec), None, ), # 1 - (2, TType.STRUCT, 'o2', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 2 ) - def __init__(self, success=None, o1=None, o2=None,): + def __init__(self, success=None, o1=None,): self.success = success self.o1 = o1 - self.o2 = o2 def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -19243,9 +21662,8 @@ def read(self, iprot): if ftype == TType.STOP: break if fid == 0: - if ftype == TType.STRUCT: - self.success = Function() - self.success.read(iprot) + if ftype == TType.STRING: + self.success = iprot.readString(); else: iprot.skip(ftype) elif fid == 1: @@ -19254,12 +21672,6 @@ def read(self, iprot): self.o1.read(iprot) else: iprot.skip(ftype) - elif fid == 2: - if ftype == TType.STRUCT: - self.o2 = NoSuchObjectException() - self.o2.read(iprot) - else: - iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() @@ -19269,19 +21681,15 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('get_function_result') + oprot.writeStructBegin('get_delegation_token_result') if self.success is not None: - oprot.writeFieldBegin('success', TType.STRUCT, 0) - self.success.write(oprot) + oprot.writeFieldBegin('success', TType.STRING, 0) + oprot.writeString(self.success) oprot.writeFieldEnd() if self.o1 is not None: oprot.writeFieldBegin('o1', TType.STRUCT, 1) self.o1.write(oprot) oprot.writeFieldEnd() - if self.o2 is not None: - oprot.writeFieldBegin('o2', TType.STRUCT, 2) - self.o2.write(oprot) - oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -19300,19 +21708,19 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class create_role_args: +class renew_delegation_token_args: """ Attributes: - - role + - token_str_form """ thrift_spec = ( None, # 0 - (1, TType.STRUCT, 'role', (Role, Role.thrift_spec), None, ), # 1 + (1, TType.STRING, 'token_str_form', None, None, ), # 1 ) - def __init__(self, role=None,): - self.role = role + def __init__(self, token_str_form=None,): + self.token_str_form = token_str_form def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -19324,9 +21732,8 @@ def read(self, iprot): if ftype == TType.STOP: break if fid == 1: - if ftype == TType.STRUCT: - self.role = Role() - self.role.read(iprot) + if ftype == TType.STRING: + self.token_str_form = iprot.readString(); else: iprot.skip(ftype) else: @@ -19338,10 +21745,10 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('create_role_args') - if self.role is not None: - oprot.writeFieldBegin('role', TType.STRUCT, 1) - self.role.write(oprot) + oprot.writeStructBegin('renew_delegation_token_args') + if self.token_str_form is not None: + oprot.writeFieldBegin('token_str_form', TType.STRING, 1) + oprot.writeString(self.token_str_form) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -19361,7 +21768,7 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class create_role_result: +class renew_delegation_token_result: """ Attributes: - success @@ -19369,7 +21776,7 @@ class create_role_result: """ thrift_spec = ( - (0, TType.BOOL, 'success', None, None, ), # 0 + (0, TType.I64, 'success', None, None, ), # 0 (1, TType.STRUCT, 'o1', (MetaException, MetaException.thrift_spec), None, ), # 1 ) @@ -19387,8 +21794,8 @@ def read(self, iprot): if ftype == TType.STOP: break if fid == 0: - if ftype == TType.BOOL: - self.success = iprot.readBool(); + if ftype == TType.I64: + self.success = iprot.readI64(); else: iprot.skip(ftype) elif fid == 1: @@ -19406,10 +21813,10 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('create_role_result') + oprot.writeStructBegin('renew_delegation_token_result') if self.success is not None: - oprot.writeFieldBegin('success', TType.BOOL, 0) - oprot.writeBool(self.success) + oprot.writeFieldBegin('success', TType.I64, 0) + oprot.writeI64(self.success) oprot.writeFieldEnd() if self.o1 is not None: oprot.writeFieldBegin('o1', TType.STRUCT, 1) @@ -19433,19 +21840,19 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class drop_role_args: +class cancel_delegation_token_args: """ Attributes: - - role_name + - token_str_form """ thrift_spec = ( None, # 0 - (1, TType.STRING, 'role_name', None, None, ), # 1 + (1, TType.STRING, 'token_str_form', None, None, ), # 1 ) - def __init__(self, role_name=None,): - self.role_name = role_name + def __init__(self, token_str_form=None,): + self.token_str_form = token_str_form def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -19458,7 +21865,7 @@ def read(self, iprot): break if fid == 1: if ftype == TType.STRING: - self.role_name = iprot.readString(); + self.token_str_form = iprot.readString(); else: iprot.skip(ftype) else: @@ -19470,10 +21877,10 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('drop_role_args') - if self.role_name is not None: - oprot.writeFieldBegin('role_name', TType.STRING, 1) - oprot.writeString(self.role_name) + oprot.writeStructBegin('cancel_delegation_token_args') + if self.token_str_form is not None: + oprot.writeFieldBegin('token_str_form', TType.STRING, 1) + oprot.writeString(self.token_str_form) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -19493,20 +21900,18 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class drop_role_result: +class cancel_delegation_token_result: """ Attributes: - - success - o1 """ thrift_spec = ( - (0, TType.BOOL, 'success', None, None, ), # 0 + None, # 0 (1, TType.STRUCT, 'o1', (MetaException, MetaException.thrift_spec), None, ), # 1 ) - def __init__(self, success=None, o1=None,): - self.success = success + def __init__(self, o1=None,): self.o1 = o1 def read(self, iprot): @@ -19518,12 +21923,7 @@ def read(self, iprot): (fname, ftype, fid) = iprot.readFieldBegin() if ftype == TType.STOP: break - if fid == 0: - if ftype == TType.BOOL: - self.success = iprot.readBool(); - else: - iprot.skip(ftype) - elif fid == 1: + if fid == 1: if ftype == TType.STRUCT: self.o1 = MetaException() self.o1.read(iprot) @@ -19538,11 +21938,7 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('drop_role_result') - if self.success is not None: - oprot.writeFieldBegin('success', TType.BOOL, 0) - oprot.writeBool(self.success) - oprot.writeFieldEnd() + oprot.writeStructBegin('cancel_delegation_token_result') if self.o1 is not None: oprot.writeFieldBegin('o1', TType.STRUCT, 1) self.o1.write(oprot) @@ -19565,7 +21961,7 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class get_role_names_args: +class get_open_txns_args: thrift_spec = ( ) @@ -19588,7 +21984,7 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('get_role_names_args') + oprot.writeStructBegin('get_open_txns_args') oprot.writeFieldStop() oprot.writeStructEnd() @@ -19607,21 +22003,18 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class get_role_names_result: +class get_open_txns_result: """ Attributes: - success - - o1 """ thrift_spec = ( - (0, TType.LIST, 'success', (TType.STRING,None), None, ), # 0 - (1, TType.STRUCT, 'o1', (MetaException, MetaException.thrift_spec), None, ), # 1 + (0, TType.STRUCT, 'success', (GetOpenTxnsResponse, GetOpenTxnsResponse.thrift_spec), None, ), # 0 ) - def __init__(self, success=None, o1=None,): + def __init__(self, success=None,): self.success = success - self.o1 = o1 def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -19633,19 +22026,9 @@ def read(self, iprot): if ftype == TType.STOP: break if fid == 0: - if ftype == TType.LIST: - self.success = [] - (_etype634, _size631) = iprot.readListBegin() - for _i635 in xrange(_size631): - _elem636 = iprot.readString(); - self.success.append(_elem636) - iprot.readListEnd() - else: - iprot.skip(ftype) - elif fid == 1: if ftype == TType.STRUCT: - self.o1 = MetaException() - self.o1.read(iprot) + self.success = GetOpenTxnsResponse() + self.success.read(iprot) else: iprot.skip(ftype) else: @@ -19657,17 +22040,10 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('get_role_names_result') + oprot.writeStructBegin('get_open_txns_result') if self.success is not None: - oprot.writeFieldBegin('success', TType.LIST, 0) - oprot.writeListBegin(TType.STRING, len(self.success)) - for iter637 in self.success: - oprot.writeString(iter637) - oprot.writeListEnd() - oprot.writeFieldEnd() - if self.o1 is not None: - oprot.writeFieldBegin('o1', TType.STRUCT, 1) - self.o1.write(oprot) + oprot.writeFieldBegin('success', TType.STRUCT, 0) + self.success.write(oprot) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -19687,35 +22063,11 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class grant_role_args: - """ - Attributes: - - role_name - - principal_name - - principal_type - - grantor - - grantorType - - grant_option - """ +class get_open_txns_info_args: thrift_spec = ( - None, # 0 - (1, TType.STRING, 'role_name', None, None, ), # 1 - (2, TType.STRING, 'principal_name', None, None, ), # 2 - (3, TType.I32, 'principal_type', None, None, ), # 3 - (4, TType.STRING, 'grantor', None, None, ), # 4 - (5, TType.I32, 'grantorType', None, None, ), # 5 - (6, TType.BOOL, 'grant_option', None, None, ), # 6 ) - def __init__(self, role_name=None, principal_name=None, principal_type=None, grantor=None, grantorType=None, grant_option=None,): - self.role_name = role_name - self.principal_name = principal_name - self.principal_type = principal_type - self.grantor = grantor - self.grantorType = grantorType - self.grant_option = grant_option - def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) @@ -19725,36 +22077,6 @@ def read(self, iprot): (fname, ftype, fid) = iprot.readFieldBegin() if ftype == TType.STOP: break - if fid == 1: - if ftype == TType.STRING: - self.role_name = iprot.readString(); - else: - iprot.skip(ftype) - elif fid == 2: - if ftype == TType.STRING: - self.principal_name = iprot.readString(); - else: - iprot.skip(ftype) - elif fid == 3: - if ftype == TType.I32: - self.principal_type = iprot.readI32(); - else: - iprot.skip(ftype) - elif fid == 4: - if ftype == TType.STRING: - self.grantor = iprot.readString(); - else: - iprot.skip(ftype) - elif fid == 5: - if ftype == TType.I32: - self.grantorType = iprot.readI32(); - else: - iprot.skip(ftype) - elif fid == 6: - if ftype == TType.BOOL: - self.grant_option = iprot.readBool(); - else: - iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() @@ -19764,31 +22086,7 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('grant_role_args') - if self.role_name is not None: - oprot.writeFieldBegin('role_name', TType.STRING, 1) - oprot.writeString(self.role_name) - oprot.writeFieldEnd() - if self.principal_name is not None: - oprot.writeFieldBegin('principal_name', TType.STRING, 2) - oprot.writeString(self.principal_name) - oprot.writeFieldEnd() - if self.principal_type is not None: - oprot.writeFieldBegin('principal_type', TType.I32, 3) - oprot.writeI32(self.principal_type) - oprot.writeFieldEnd() - if self.grantor is not None: - oprot.writeFieldBegin('grantor', TType.STRING, 4) - oprot.writeString(self.grantor) - oprot.writeFieldEnd() - if self.grantorType is not None: - oprot.writeFieldBegin('grantorType', TType.I32, 5) - oprot.writeI32(self.grantorType) - oprot.writeFieldEnd() - if self.grant_option is not None: - oprot.writeFieldBegin('grant_option', TType.BOOL, 6) - oprot.writeBool(self.grant_option) - oprot.writeFieldEnd() + oprot.writeStructBegin('get_open_txns_info_args') oprot.writeFieldStop() oprot.writeStructEnd() @@ -19807,21 +22105,18 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class grant_role_result: +class get_open_txns_info_result: """ Attributes: - success - - o1 """ thrift_spec = ( - (0, TType.BOOL, 'success', None, None, ), # 0 - (1, TType.STRUCT, 'o1', (MetaException, MetaException.thrift_spec), None, ), # 1 + (0, TType.STRUCT, 'success', (GetOpenTxnsInfoResponse, GetOpenTxnsInfoResponse.thrift_spec), None, ), # 0 ) - def __init__(self, success=None, o1=None,): + def __init__(self, success=None,): self.success = success - self.o1 = o1 def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -19833,14 +22128,9 @@ def read(self, iprot): if ftype == TType.STOP: break if fid == 0: - if ftype == TType.BOOL: - self.success = iprot.readBool(); - else: - iprot.skip(ftype) - elif fid == 1: if ftype == TType.STRUCT: - self.o1 = MetaException() - self.o1.read(iprot) + self.success = GetOpenTxnsInfoResponse() + self.success.read(iprot) else: iprot.skip(ftype) else: @@ -19852,14 +22142,10 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('grant_role_result') + oprot.writeStructBegin('get_open_txns_info_result') if self.success is not None: - oprot.writeFieldBegin('success', TType.BOOL, 0) - oprot.writeBool(self.success) - oprot.writeFieldEnd() - if self.o1 is not None: - oprot.writeFieldBegin('o1', TType.STRUCT, 1) - self.o1.write(oprot) + oprot.writeFieldBegin('success', TType.STRUCT, 0) + self.success.write(oprot) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -19879,25 +22165,19 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class revoke_role_args: +class open_txns_args: """ Attributes: - - role_name - - principal_name - - principal_type + - rqst """ thrift_spec = ( None, # 0 - (1, TType.STRING, 'role_name', None, None, ), # 1 - (2, TType.STRING, 'principal_name', None, None, ), # 2 - (3, TType.I32, 'principal_type', None, None, ), # 3 + (1, TType.STRUCT, 'rqst', (OpenTxnRequest, OpenTxnRequest.thrift_spec), None, ), # 1 ) - def __init__(self, role_name=None, principal_name=None, principal_type=None,): - self.role_name = role_name - self.principal_name = principal_name - self.principal_type = principal_type + def __init__(self, rqst=None,): + self.rqst = rqst def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -19906,21 +22186,12 @@ def read(self, iprot): iprot.readStructBegin() while True: (fname, ftype, fid) = iprot.readFieldBegin() - if ftype == TType.STOP: - break - if fid == 1: - if ftype == TType.STRING: - self.role_name = iprot.readString(); - else: - iprot.skip(ftype) - elif fid == 2: - if ftype == TType.STRING: - self.principal_name = iprot.readString(); - else: - iprot.skip(ftype) - elif fid == 3: - if ftype == TType.I32: - self.principal_type = iprot.readI32(); + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.STRUCT: + self.rqst = OpenTxnRequest() + self.rqst.read(iprot) else: iprot.skip(ftype) else: @@ -19932,18 +22203,10 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('revoke_role_args') - if self.role_name is not None: - oprot.writeFieldBegin('role_name', TType.STRING, 1) - oprot.writeString(self.role_name) - oprot.writeFieldEnd() - if self.principal_name is not None: - oprot.writeFieldBegin('principal_name', TType.STRING, 2) - oprot.writeString(self.principal_name) - oprot.writeFieldEnd() - if self.principal_type is not None: - oprot.writeFieldBegin('principal_type', TType.I32, 3) - oprot.writeI32(self.principal_type) + oprot.writeStructBegin('open_txns_args') + if self.rqst is not None: + oprot.writeFieldBegin('rqst', TType.STRUCT, 1) + self.rqst.write(oprot) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -19963,21 +22226,18 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class revoke_role_result: +class open_txns_result: """ Attributes: - success - - o1 """ thrift_spec = ( - (0, TType.BOOL, 'success', None, None, ), # 0 - (1, TType.STRUCT, 'o1', (MetaException, MetaException.thrift_spec), None, ), # 1 + (0, TType.STRUCT, 'success', (OpenTxnsResponse, OpenTxnsResponse.thrift_spec), None, ), # 0 ) - def __init__(self, success=None, o1=None,): + def __init__(self, success=None,): self.success = success - self.o1 = o1 def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -19989,14 +22249,9 @@ def read(self, iprot): if ftype == TType.STOP: break if fid == 0: - if ftype == TType.BOOL: - self.success = iprot.readBool(); - else: - iprot.skip(ftype) - elif fid == 1: if ftype == TType.STRUCT: - self.o1 = MetaException() - self.o1.read(iprot) + self.success = OpenTxnsResponse() + self.success.read(iprot) else: iprot.skip(ftype) else: @@ -20008,14 +22263,10 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('revoke_role_result') + oprot.writeStructBegin('open_txns_result') if self.success is not None: - oprot.writeFieldBegin('success', TType.BOOL, 0) - oprot.writeBool(self.success) - oprot.writeFieldEnd() - if self.o1 is not None: - oprot.writeFieldBegin('o1', TType.STRUCT, 1) - self.o1.write(oprot) + oprot.writeFieldBegin('success', TType.STRUCT, 0) + self.success.write(oprot) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -20035,22 +22286,19 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class list_roles_args: +class abort_txn_args: """ Attributes: - - principal_name - - principal_type + - rqst """ thrift_spec = ( None, # 0 - (1, TType.STRING, 'principal_name', None, None, ), # 1 - (2, TType.I32, 'principal_type', None, None, ), # 2 + (1, TType.STRUCT, 'rqst', (AbortTxnRequest, AbortTxnRequest.thrift_spec), None, ), # 1 ) - def __init__(self, principal_name=None, principal_type=None,): - self.principal_name = principal_name - self.principal_type = principal_type + def __init__(self, rqst=None,): + self.rqst = rqst def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -20062,13 +22310,9 @@ def read(self, iprot): if ftype == TType.STOP: break if fid == 1: - if ftype == TType.STRING: - self.principal_name = iprot.readString(); - else: - iprot.skip(ftype) - elif fid == 2: - if ftype == TType.I32: - self.principal_type = iprot.readI32(); + if ftype == TType.STRUCT: + self.rqst = AbortTxnRequest() + self.rqst.read(iprot) else: iprot.skip(ftype) else: @@ -20080,14 +22324,10 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('list_roles_args') - if self.principal_name is not None: - oprot.writeFieldBegin('principal_name', TType.STRING, 1) - oprot.writeString(self.principal_name) - oprot.writeFieldEnd() - if self.principal_type is not None: - oprot.writeFieldBegin('principal_type', TType.I32, 2) - oprot.writeI32(self.principal_type) + oprot.writeStructBegin('abort_txn_args') + if self.rqst is not None: + oprot.writeFieldBegin('rqst', TType.STRUCT, 1) + self.rqst.write(oprot) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -20107,20 +22347,18 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class list_roles_result: +class abort_txn_result: """ Attributes: - - success - o1 """ thrift_spec = ( - (0, TType.LIST, 'success', (TType.STRUCT,(Role, Role.thrift_spec)), None, ), # 0 - (1, TType.STRUCT, 'o1', (MetaException, MetaException.thrift_spec), None, ), # 1 + None, # 0 + (1, TType.STRUCT, 'o1', (NoSuchTxnException, NoSuchTxnException.thrift_spec), None, ), # 1 ) - def __init__(self, success=None, o1=None,): - self.success = success + def __init__(self, o1=None,): self.o1 = o1 def read(self, iprot): @@ -20132,20 +22370,9 @@ def read(self, iprot): (fname, ftype, fid) = iprot.readFieldBegin() if ftype == TType.STOP: break - if fid == 0: - if ftype == TType.LIST: - self.success = [] - (_etype641, _size638) = iprot.readListBegin() - for _i642 in xrange(_size638): - _elem643 = Role() - _elem643.read(iprot) - self.success.append(_elem643) - iprot.readListEnd() - else: - iprot.skip(ftype) - elif fid == 1: + if fid == 1: if ftype == TType.STRUCT: - self.o1 = MetaException() + self.o1 = NoSuchTxnException() self.o1.read(iprot) else: iprot.skip(ftype) @@ -20158,14 +22385,7 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('list_roles_result') - if self.success is not None: - oprot.writeFieldBegin('success', TType.LIST, 0) - oprot.writeListBegin(TType.STRUCT, len(self.success)) - for iter644 in self.success: - iter644.write(oprot) - oprot.writeListEnd() - oprot.writeFieldEnd() + oprot.writeStructBegin('abort_txn_result') if self.o1 is not None: oprot.writeFieldBegin('o1', TType.STRUCT, 1) self.o1.write(oprot) @@ -20188,25 +22408,19 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class get_privilege_set_args: +class commit_txn_args: """ Attributes: - - hiveObject - - user_name - - group_names + - rqst """ thrift_spec = ( None, # 0 - (1, TType.STRUCT, 'hiveObject', (HiveObjectRef, HiveObjectRef.thrift_spec), None, ), # 1 - (2, TType.STRING, 'user_name', None, None, ), # 2 - (3, TType.LIST, 'group_names', (TType.STRING,None), None, ), # 3 + (1, TType.STRUCT, 'rqst', (CommitTxnRequest, CommitTxnRequest.thrift_spec), None, ), # 1 ) - def __init__(self, hiveObject=None, user_name=None, group_names=None,): - self.hiveObject = hiveObject - self.user_name = user_name - self.group_names = group_names + def __init__(self, rqst=None,): + self.rqst = rqst def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -20219,23 +22433,8 @@ def read(self, iprot): break if fid == 1: if ftype == TType.STRUCT: - self.hiveObject = HiveObjectRef() - self.hiveObject.read(iprot) - else: - iprot.skip(ftype) - elif fid == 2: - if ftype == TType.STRING: - self.user_name = iprot.readString(); - else: - iprot.skip(ftype) - elif fid == 3: - if ftype == TType.LIST: - self.group_names = [] - (_etype648, _size645) = iprot.readListBegin() - for _i649 in xrange(_size645): - _elem650 = iprot.readString(); - self.group_names.append(_elem650) - iprot.readListEnd() + self.rqst = CommitTxnRequest() + self.rqst.read(iprot) else: iprot.skip(ftype) else: @@ -20247,21 +22446,10 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('get_privilege_set_args') - if self.hiveObject is not None: - oprot.writeFieldBegin('hiveObject', TType.STRUCT, 1) - self.hiveObject.write(oprot) - oprot.writeFieldEnd() - if self.user_name is not None: - oprot.writeFieldBegin('user_name', TType.STRING, 2) - oprot.writeString(self.user_name) - oprot.writeFieldEnd() - if self.group_names is not None: - oprot.writeFieldBegin('group_names', TType.LIST, 3) - oprot.writeListBegin(TType.STRING, len(self.group_names)) - for iter651 in self.group_names: - oprot.writeString(iter651) - oprot.writeListEnd() + oprot.writeStructBegin('commit_txn_args') + if self.rqst is not None: + oprot.writeFieldBegin('rqst', TType.STRUCT, 1) + self.rqst.write(oprot) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -20281,21 +22469,22 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class get_privilege_set_result: +class commit_txn_result: """ Attributes: - - success - o1 + - o2 """ thrift_spec = ( - (0, TType.STRUCT, 'success', (PrincipalPrivilegeSet, PrincipalPrivilegeSet.thrift_spec), None, ), # 0 - (1, TType.STRUCT, 'o1', (MetaException, MetaException.thrift_spec), None, ), # 1 + None, # 0 + (1, TType.STRUCT, 'o1', (NoSuchTxnException, NoSuchTxnException.thrift_spec), None, ), # 1 + (2, TType.STRUCT, 'o2', (TxnAbortedException, TxnAbortedException.thrift_spec), None, ), # 2 ) - def __init__(self, success=None, o1=None,): - self.success = success + def __init__(self, o1=None, o2=None,): self.o1 = o1 + self.o2 = o2 def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -20306,16 +22495,16 @@ def read(self, iprot): (fname, ftype, fid) = iprot.readFieldBegin() if ftype == TType.STOP: break - if fid == 0: + if fid == 1: if ftype == TType.STRUCT: - self.success = PrincipalPrivilegeSet() - self.success.read(iprot) + self.o1 = NoSuchTxnException() + self.o1.read(iprot) else: iprot.skip(ftype) - elif fid == 1: + elif fid == 2: if ftype == TType.STRUCT: - self.o1 = MetaException() - self.o1.read(iprot) + self.o2 = TxnAbortedException() + self.o2.read(iprot) else: iprot.skip(ftype) else: @@ -20327,15 +22516,15 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('get_privilege_set_result') - if self.success is not None: - oprot.writeFieldBegin('success', TType.STRUCT, 0) - self.success.write(oprot) - oprot.writeFieldEnd() + oprot.writeStructBegin('commit_txn_result') if self.o1 is not None: oprot.writeFieldBegin('o1', TType.STRUCT, 1) self.o1.write(oprot) oprot.writeFieldEnd() + if self.o2 is not None: + oprot.writeFieldBegin('o2', TType.STRUCT, 2) + self.o2.write(oprot) + oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -20354,25 +22543,19 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class list_privileges_args: +class lock_args: """ Attributes: - - principal_name - - principal_type - - hiveObject + - rqst """ thrift_spec = ( None, # 0 - (1, TType.STRING, 'principal_name', None, None, ), # 1 - (2, TType.I32, 'principal_type', None, None, ), # 2 - (3, TType.STRUCT, 'hiveObject', (HiveObjectRef, HiveObjectRef.thrift_spec), None, ), # 3 + (1, TType.STRUCT, 'rqst', (LockRequest, LockRequest.thrift_spec), None, ), # 1 ) - def __init__(self, principal_name=None, principal_type=None, hiveObject=None,): - self.principal_name = principal_name - self.principal_type = principal_type - self.hiveObject = hiveObject + def __init__(self, rqst=None,): + self.rqst = rqst def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -20384,19 +22567,9 @@ def read(self, iprot): if ftype == TType.STOP: break if fid == 1: - if ftype == TType.STRING: - self.principal_name = iprot.readString(); - else: - iprot.skip(ftype) - elif fid == 2: - if ftype == TType.I32: - self.principal_type = iprot.readI32(); - else: - iprot.skip(ftype) - elif fid == 3: if ftype == TType.STRUCT: - self.hiveObject = HiveObjectRef() - self.hiveObject.read(iprot) + self.rqst = LockRequest() + self.rqst.read(iprot) else: iprot.skip(ftype) else: @@ -20408,18 +22581,10 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('list_privileges_args') - if self.principal_name is not None: - oprot.writeFieldBegin('principal_name', TType.STRING, 1) - oprot.writeString(self.principal_name) - oprot.writeFieldEnd() - if self.principal_type is not None: - oprot.writeFieldBegin('principal_type', TType.I32, 2) - oprot.writeI32(self.principal_type) - oprot.writeFieldEnd() - if self.hiveObject is not None: - oprot.writeFieldBegin('hiveObject', TType.STRUCT, 3) - self.hiveObject.write(oprot) + oprot.writeStructBegin('lock_args') + if self.rqst is not None: + oprot.writeFieldBegin('rqst', TType.STRUCT, 1) + self.rqst.write(oprot) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -20439,21 +22604,24 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class list_privileges_result: +class lock_result: """ Attributes: - success - o1 + - o2 """ thrift_spec = ( - (0, TType.LIST, 'success', (TType.STRUCT,(HiveObjectPrivilege, HiveObjectPrivilege.thrift_spec)), None, ), # 0 - (1, TType.STRUCT, 'o1', (MetaException, MetaException.thrift_spec), None, ), # 1 + (0, TType.STRUCT, 'success', (LockResponse, LockResponse.thrift_spec), None, ), # 0 + (1, TType.STRUCT, 'o1', (NoSuchTxnException, NoSuchTxnException.thrift_spec), None, ), # 1 + (2, TType.STRUCT, 'o2', (TxnAbortedException, TxnAbortedException.thrift_spec), None, ), # 2 ) - def __init__(self, success=None, o1=None,): + def __init__(self, success=None, o1=None, o2=None,): self.success = success self.o1 = o1 + self.o2 = o2 def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -20465,22 +22633,23 @@ def read(self, iprot): if ftype == TType.STOP: break if fid == 0: - if ftype == TType.LIST: - self.success = [] - (_etype655, _size652) = iprot.readListBegin() - for _i656 in xrange(_size652): - _elem657 = HiveObjectPrivilege() - _elem657.read(iprot) - self.success.append(_elem657) - iprot.readListEnd() + if ftype == TType.STRUCT: + self.success = LockResponse() + self.success.read(iprot) else: iprot.skip(ftype) elif fid == 1: if ftype == TType.STRUCT: - self.o1 = MetaException() + self.o1 = NoSuchTxnException() self.o1.read(iprot) else: iprot.skip(ftype) + elif fid == 2: + if ftype == TType.STRUCT: + self.o2 = TxnAbortedException() + self.o2.read(iprot) + else: + iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() @@ -20490,18 +22659,19 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('list_privileges_result') + oprot.writeStructBegin('lock_result') if self.success is not None: - oprot.writeFieldBegin('success', TType.LIST, 0) - oprot.writeListBegin(TType.STRUCT, len(self.success)) - for iter658 in self.success: - iter658.write(oprot) - oprot.writeListEnd() + oprot.writeFieldBegin('success', TType.STRUCT, 0) + self.success.write(oprot) oprot.writeFieldEnd() if self.o1 is not None: oprot.writeFieldBegin('o1', TType.STRUCT, 1) self.o1.write(oprot) oprot.writeFieldEnd() + if self.o2 is not None: + oprot.writeFieldBegin('o2', TType.STRUCT, 2) + self.o2.write(oprot) + oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -20520,19 +22690,19 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class grant_privileges_args: +class check_lock_args: """ Attributes: - - privileges + - rqst """ thrift_spec = ( None, # 0 - (1, TType.STRUCT, 'privileges', (PrivilegeBag, PrivilegeBag.thrift_spec), None, ), # 1 + (1, TType.STRUCT, 'rqst', (CheckLockRequest, CheckLockRequest.thrift_spec), None, ), # 1 ) - def __init__(self, privileges=None,): - self.privileges = privileges + def __init__(self, rqst=None,): + self.rqst = rqst def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -20545,8 +22715,8 @@ def read(self, iprot): break if fid == 1: if ftype == TType.STRUCT: - self.privileges = PrivilegeBag() - self.privileges.read(iprot) + self.rqst = CheckLockRequest() + self.rqst.read(iprot) else: iprot.skip(ftype) else: @@ -20558,10 +22728,10 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('grant_privileges_args') - if self.privileges is not None: - oprot.writeFieldBegin('privileges', TType.STRUCT, 1) - self.privileges.write(oprot) + oprot.writeStructBegin('check_lock_args') + if self.rqst is not None: + oprot.writeFieldBegin('rqst', TType.STRUCT, 1) + self.rqst.write(oprot) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -20581,21 +22751,27 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class grant_privileges_result: +class check_lock_result: """ Attributes: - success - o1 + - o2 + - o3 """ thrift_spec = ( - (0, TType.BOOL, 'success', None, None, ), # 0 - (1, TType.STRUCT, 'o1', (MetaException, MetaException.thrift_spec), None, ), # 1 + (0, TType.STRUCT, 'success', (LockResponse, LockResponse.thrift_spec), None, ), # 0 + (1, TType.STRUCT, 'o1', (NoSuchTxnException, NoSuchTxnException.thrift_spec), None, ), # 1 + (2, TType.STRUCT, 'o2', (TxnAbortedException, TxnAbortedException.thrift_spec), None, ), # 2 + (3, TType.STRUCT, 'o3', (NoSuchLockException, NoSuchLockException.thrift_spec), None, ), # 3 ) - def __init__(self, success=None, o1=None,): + def __init__(self, success=None, o1=None, o2=None, o3=None,): self.success = success self.o1 = o1 + self.o2 = o2 + self.o3 = o3 def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -20607,16 +22783,29 @@ def read(self, iprot): if ftype == TType.STOP: break if fid == 0: - if ftype == TType.BOOL: - self.success = iprot.readBool(); + if ftype == TType.STRUCT: + self.success = LockResponse() + self.success.read(iprot) else: iprot.skip(ftype) elif fid == 1: if ftype == TType.STRUCT: - self.o1 = MetaException() + self.o1 = NoSuchTxnException() self.o1.read(iprot) else: iprot.skip(ftype) + elif fid == 2: + if ftype == TType.STRUCT: + self.o2 = TxnAbortedException() + self.o2.read(iprot) + else: + iprot.skip(ftype) + elif fid == 3: + if ftype == TType.STRUCT: + self.o3 = NoSuchLockException() + self.o3.read(iprot) + else: + iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() @@ -20626,15 +22815,23 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('grant_privileges_result') + oprot.writeStructBegin('check_lock_result') if self.success is not None: - oprot.writeFieldBegin('success', TType.BOOL, 0) - oprot.writeBool(self.success) + oprot.writeFieldBegin('success', TType.STRUCT, 0) + self.success.write(oprot) oprot.writeFieldEnd() if self.o1 is not None: oprot.writeFieldBegin('o1', TType.STRUCT, 1) self.o1.write(oprot) oprot.writeFieldEnd() + if self.o2 is not None: + oprot.writeFieldBegin('o2', TType.STRUCT, 2) + self.o2.write(oprot) + oprot.writeFieldEnd() + if self.o3 is not None: + oprot.writeFieldBegin('o3', TType.STRUCT, 3) + self.o3.write(oprot) + oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -20653,19 +22850,19 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class revoke_privileges_args: +class unlock_args: """ Attributes: - - privileges + - rqst """ thrift_spec = ( None, # 0 - (1, TType.STRUCT, 'privileges', (PrivilegeBag, PrivilegeBag.thrift_spec), None, ), # 1 + (1, TType.STRUCT, 'rqst', (UnlockRequest, UnlockRequest.thrift_spec), None, ), # 1 ) - def __init__(self, privileges=None,): - self.privileges = privileges + def __init__(self, rqst=None,): + self.rqst = rqst def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -20678,8 +22875,8 @@ def read(self, iprot): break if fid == 1: if ftype == TType.STRUCT: - self.privileges = PrivilegeBag() - self.privileges.read(iprot) + self.rqst = UnlockRequest() + self.rqst.read(iprot) else: iprot.skip(ftype) else: @@ -20691,10 +22888,10 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('revoke_privileges_args') - if self.privileges is not None: - oprot.writeFieldBegin('privileges', TType.STRUCT, 1) - self.privileges.write(oprot) + oprot.writeStructBegin('unlock_args') + if self.rqst is not None: + oprot.writeFieldBegin('rqst', TType.STRUCT, 1) + self.rqst.write(oprot) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -20714,21 +22911,22 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class revoke_privileges_result: +class unlock_result: """ Attributes: - - success - o1 + - o2 """ thrift_spec = ( - (0, TType.BOOL, 'success', None, None, ), # 0 - (1, TType.STRUCT, 'o1', (MetaException, MetaException.thrift_spec), None, ), # 1 + None, # 0 + (1, TType.STRUCT, 'o1', (NoSuchLockException, NoSuchLockException.thrift_spec), None, ), # 1 + (2, TType.STRUCT, 'o2', (TxnOpenException, TxnOpenException.thrift_spec), None, ), # 2 ) - def __init__(self, success=None, o1=None,): - self.success = success + def __init__(self, o1=None, o2=None,): self.o1 = o1 + self.o2 = o2 def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -20739,15 +22937,16 @@ def read(self, iprot): (fname, ftype, fid) = iprot.readFieldBegin() if ftype == TType.STOP: break - if fid == 0: - if ftype == TType.BOOL: - self.success = iprot.readBool(); + if fid == 1: + if ftype == TType.STRUCT: + self.o1 = NoSuchLockException() + self.o1.read(iprot) else: iprot.skip(ftype) - elif fid == 1: + elif fid == 2: if ftype == TType.STRUCT: - self.o1 = MetaException() - self.o1.read(iprot) + self.o2 = TxnOpenException() + self.o2.read(iprot) else: iprot.skip(ftype) else: @@ -20759,15 +22958,15 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('revoke_privileges_result') - if self.success is not None: - oprot.writeFieldBegin('success', TType.BOOL, 0) - oprot.writeBool(self.success) - oprot.writeFieldEnd() + oprot.writeStructBegin('unlock_result') if self.o1 is not None: oprot.writeFieldBegin('o1', TType.STRUCT, 1) self.o1.write(oprot) oprot.writeFieldEnd() + if self.o2 is not None: + oprot.writeFieldBegin('o2', TType.STRUCT, 2) + self.o2.write(oprot) + oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -20786,22 +22985,19 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class set_ugi_args: +class show_locks_args: """ Attributes: - - user_name - - group_names + - rqst """ thrift_spec = ( None, # 0 - (1, TType.STRING, 'user_name', None, None, ), # 1 - (2, TType.LIST, 'group_names', (TType.STRING,None), None, ), # 2 + (1, TType.STRUCT, 'rqst', (ShowLocksRequest, ShowLocksRequest.thrift_spec), None, ), # 1 ) - def __init__(self, user_name=None, group_names=None,): - self.user_name = user_name - self.group_names = group_names + def __init__(self, rqst=None,): + self.rqst = rqst def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -20813,18 +23009,9 @@ def read(self, iprot): if ftype == TType.STOP: break if fid == 1: - if ftype == TType.STRING: - self.user_name = iprot.readString(); - else: - iprot.skip(ftype) - elif fid == 2: - if ftype == TType.LIST: - self.group_names = [] - (_etype662, _size659) = iprot.readListBegin() - for _i663 in xrange(_size659): - _elem664 = iprot.readString(); - self.group_names.append(_elem664) - iprot.readListEnd() + if ftype == TType.STRUCT: + self.rqst = ShowLocksRequest() + self.rqst.read(iprot) else: iprot.skip(ftype) else: @@ -20836,17 +23023,10 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('set_ugi_args') - if self.user_name is not None: - oprot.writeFieldBegin('user_name', TType.STRING, 1) - oprot.writeString(self.user_name) - oprot.writeFieldEnd() - if self.group_names is not None: - oprot.writeFieldBegin('group_names', TType.LIST, 2) - oprot.writeListBegin(TType.STRING, len(self.group_names)) - for iter665 in self.group_names: - oprot.writeString(iter665) - oprot.writeListEnd() + oprot.writeStructBegin('show_locks_args') + if self.rqst is not None: + oprot.writeFieldBegin('rqst', TType.STRUCT, 1) + self.rqst.write(oprot) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -20866,21 +23046,18 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class set_ugi_result: +class show_locks_result: """ Attributes: - success - - o1 """ thrift_spec = ( - (0, TType.LIST, 'success', (TType.STRING,None), None, ), # 0 - (1, TType.STRUCT, 'o1', (MetaException, MetaException.thrift_spec), None, ), # 1 + (0, TType.STRUCT, 'success', (ShowLocksResponse, ShowLocksResponse.thrift_spec), None, ), # 0 ) - def __init__(self, success=None, o1=None,): + def __init__(self, success=None,): self.success = success - self.o1 = o1 def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -20892,19 +23069,9 @@ def read(self, iprot): if ftype == TType.STOP: break if fid == 0: - if ftype == TType.LIST: - self.success = [] - (_etype669, _size666) = iprot.readListBegin() - for _i670 in xrange(_size666): - _elem671 = iprot.readString(); - self.success.append(_elem671) - iprot.readListEnd() - else: - iprot.skip(ftype) - elif fid == 1: if ftype == TType.STRUCT: - self.o1 = MetaException() - self.o1.read(iprot) + self.success = ShowLocksResponse() + self.success.read(iprot) else: iprot.skip(ftype) else: @@ -20916,17 +23083,10 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('set_ugi_result') + oprot.writeStructBegin('show_locks_result') if self.success is not None: - oprot.writeFieldBegin('success', TType.LIST, 0) - oprot.writeListBegin(TType.STRING, len(self.success)) - for iter672 in self.success: - oprot.writeString(iter672) - oprot.writeListEnd() - oprot.writeFieldEnd() - if self.o1 is not None: - oprot.writeFieldBegin('o1', TType.STRUCT, 1) - self.o1.write(oprot) + oprot.writeFieldBegin('success', TType.STRUCT, 0) + self.success.write(oprot) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -20946,22 +23106,19 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class get_delegation_token_args: +class heartbeat_args: """ Attributes: - - token_owner - - renewer_kerberos_principal_name + - ids """ thrift_spec = ( None, # 0 - (1, TType.STRING, 'token_owner', None, None, ), # 1 - (2, TType.STRING, 'renewer_kerberos_principal_name', None, None, ), # 2 + (1, TType.STRUCT, 'ids', (HeartbeatRequest, HeartbeatRequest.thrift_spec), None, ), # 1 ) - def __init__(self, token_owner=None, renewer_kerberos_principal_name=None,): - self.token_owner = token_owner - self.renewer_kerberos_principal_name = renewer_kerberos_principal_name + def __init__(self, ids=None,): + self.ids = ids def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -20973,13 +23130,9 @@ def read(self, iprot): if ftype == TType.STOP: break if fid == 1: - if ftype == TType.STRING: - self.token_owner = iprot.readString(); - else: - iprot.skip(ftype) - elif fid == 2: - if ftype == TType.STRING: - self.renewer_kerberos_principal_name = iprot.readString(); + if ftype == TType.STRUCT: + self.ids = HeartbeatRequest() + self.ids.read(iprot) else: iprot.skip(ftype) else: @@ -20991,14 +23144,10 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('get_delegation_token_args') - if self.token_owner is not None: - oprot.writeFieldBegin('token_owner', TType.STRING, 1) - oprot.writeString(self.token_owner) - oprot.writeFieldEnd() - if self.renewer_kerberos_principal_name is not None: - oprot.writeFieldBegin('renewer_kerberos_principal_name', TType.STRING, 2) - oprot.writeString(self.renewer_kerberos_principal_name) + oprot.writeStructBegin('heartbeat_args') + if self.ids is not None: + oprot.writeFieldBegin('ids', TType.STRUCT, 1) + self.ids.write(oprot) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -21018,21 +23167,25 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class get_delegation_token_result: +class heartbeat_result: """ Attributes: - - success - o1 + - o2 + - o3 """ thrift_spec = ( - (0, TType.STRING, 'success', None, None, ), # 0 - (1, TType.STRUCT, 'o1', (MetaException, MetaException.thrift_spec), None, ), # 1 + None, # 0 + (1, TType.STRUCT, 'o1', (NoSuchLockException, NoSuchLockException.thrift_spec), None, ), # 1 + (2, TType.STRUCT, 'o2', (NoSuchTxnException, NoSuchTxnException.thrift_spec), None, ), # 2 + (3, TType.STRUCT, 'o3', (TxnAbortedException, TxnAbortedException.thrift_spec), None, ), # 3 ) - def __init__(self, success=None, o1=None,): - self.success = success + def __init__(self, o1=None, o2=None, o3=None,): self.o1 = o1 + self.o2 = o2 + self.o3 = o3 def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -21043,15 +23196,22 @@ def read(self, iprot): (fname, ftype, fid) = iprot.readFieldBegin() if ftype == TType.STOP: break - if fid == 0: - if ftype == TType.STRING: - self.success = iprot.readString(); + if fid == 1: + if ftype == TType.STRUCT: + self.o1 = NoSuchLockException() + self.o1.read(iprot) else: iprot.skip(ftype) - elif fid == 1: + elif fid == 2: if ftype == TType.STRUCT: - self.o1 = MetaException() - self.o1.read(iprot) + self.o2 = NoSuchTxnException() + self.o2.read(iprot) + else: + iprot.skip(ftype) + elif fid == 3: + if ftype == TType.STRUCT: + self.o3 = TxnAbortedException() + self.o3.read(iprot) else: iprot.skip(ftype) else: @@ -21063,15 +23223,19 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('get_delegation_token_result') - if self.success is not None: - oprot.writeFieldBegin('success', TType.STRING, 0) - oprot.writeString(self.success) - oprot.writeFieldEnd() + oprot.writeStructBegin('heartbeat_result') if self.o1 is not None: oprot.writeFieldBegin('o1', TType.STRUCT, 1) self.o1.write(oprot) oprot.writeFieldEnd() + if self.o2 is not None: + oprot.writeFieldBegin('o2', TType.STRUCT, 2) + self.o2.write(oprot) + oprot.writeFieldEnd() + if self.o3 is not None: + oprot.writeFieldBegin('o3', TType.STRUCT, 3) + self.o3.write(oprot) + oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -21090,19 +23254,19 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class renew_delegation_token_args: +class compact_args: """ Attributes: - - token_str_form + - rqst """ thrift_spec = ( None, # 0 - (1, TType.STRING, 'token_str_form', None, None, ), # 1 + (1, TType.STRUCT, 'rqst', (CompactionRequest, CompactionRequest.thrift_spec), None, ), # 1 ) - def __init__(self, token_str_form=None,): - self.token_str_form = token_str_form + def __init__(self, rqst=None,): + self.rqst = rqst def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -21114,8 +23278,9 @@ def read(self, iprot): if ftype == TType.STOP: break if fid == 1: - if ftype == TType.STRING: - self.token_str_form = iprot.readString(); + if ftype == TType.STRUCT: + self.rqst = CompactionRequest() + self.rqst.read(iprot) else: iprot.skip(ftype) else: @@ -21127,10 +23292,10 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('renew_delegation_token_args') - if self.token_str_form is not None: - oprot.writeFieldBegin('token_str_form', TType.STRING, 1) - oprot.writeString(self.token_str_form) + oprot.writeStructBegin('compact_args') + if self.rqst is not None: + oprot.writeFieldBegin('rqst', TType.STRUCT, 1) + self.rqst.write(oprot) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -21150,22 +23315,11 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class renew_delegation_token_result: - """ - Attributes: - - success - - o1 - """ +class compact_result: thrift_spec = ( - (0, TType.I64, 'success', None, None, ), # 0 - (1, TType.STRUCT, 'o1', (MetaException, MetaException.thrift_spec), None, ), # 1 ) - def __init__(self, success=None, o1=None,): - self.success = success - self.o1 = o1 - def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) @@ -21175,17 +23329,6 @@ def read(self, iprot): (fname, ftype, fid) = iprot.readFieldBegin() if ftype == TType.STOP: break - if fid == 0: - if ftype == TType.I64: - self.success = iprot.readI64(); - else: - iprot.skip(ftype) - elif fid == 1: - if ftype == TType.STRUCT: - self.o1 = MetaException() - self.o1.read(iprot) - else: - iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() @@ -21195,15 +23338,7 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('renew_delegation_token_result') - if self.success is not None: - oprot.writeFieldBegin('success', TType.I64, 0) - oprot.writeI64(self.success) - oprot.writeFieldEnd() - if self.o1 is not None: - oprot.writeFieldBegin('o1', TType.STRUCT, 1) - self.o1.write(oprot) - oprot.writeFieldEnd() + oprot.writeStructBegin('compact_result') oprot.writeFieldStop() oprot.writeStructEnd() @@ -21222,19 +23357,19 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class cancel_delegation_token_args: +class show_compact_args: """ Attributes: - - token_str_form + - rqst """ thrift_spec = ( None, # 0 - (1, TType.STRING, 'token_str_form', None, None, ), # 1 + (1, TType.STRUCT, 'rqst', (ShowCompactRequest, ShowCompactRequest.thrift_spec), None, ), # 1 ) - def __init__(self, token_str_form=None,): - self.token_str_form = token_str_form + def __init__(self, rqst=None,): + self.rqst = rqst def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -21246,8 +23381,9 @@ def read(self, iprot): if ftype == TType.STOP: break if fid == 1: - if ftype == TType.STRING: - self.token_str_form = iprot.readString(); + if ftype == TType.STRUCT: + self.rqst = ShowCompactRequest() + self.rqst.read(iprot) else: iprot.skip(ftype) else: @@ -21259,10 +23395,10 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('cancel_delegation_token_args') - if self.token_str_form is not None: - oprot.writeFieldBegin('token_str_form', TType.STRING, 1) - oprot.writeString(self.token_str_form) + oprot.writeStructBegin('show_compact_args') + if self.rqst is not None: + oprot.writeFieldBegin('rqst', TType.STRUCT, 1) + self.rqst.write(oprot) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -21282,19 +23418,18 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class cancel_delegation_token_result: +class show_compact_result: """ Attributes: - - o1 + - success """ thrift_spec = ( - None, # 0 - (1, TType.STRUCT, 'o1', (MetaException, MetaException.thrift_spec), None, ), # 1 + (0, TType.STRUCT, 'success', (ShowCompactResponse, ShowCompactResponse.thrift_spec), None, ), # 0 ) - def __init__(self, o1=None,): - self.o1 = o1 + def __init__(self, success=None,): + self.success = success def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -21305,10 +23440,10 @@ def read(self, iprot): (fname, ftype, fid) = iprot.readFieldBegin() if ftype == TType.STOP: break - if fid == 1: + if fid == 0: if ftype == TType.STRUCT: - self.o1 = MetaException() - self.o1.read(iprot) + self.success = ShowCompactResponse() + self.success.read(iprot) else: iprot.skip(ftype) else: @@ -21320,10 +23455,10 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('cancel_delegation_token_result') - if self.o1 is not None: - oprot.writeFieldBegin('o1', TType.STRUCT, 1) - self.o1.write(oprot) + oprot.writeStructBegin('show_compact_result') + if self.success is not None: + oprot.writeFieldBegin('success', TType.STRUCT, 0) + self.success.write(oprot) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() diff --git metastore/src/gen/thrift/gen-py/hive_metastore/ttypes.py metastore/src/gen/thrift/gen-py/hive_metastore/ttypes.py index 8ab1f03..33c221d 100644 --- metastore/src/gen/thrift/gen-py/hive_metastore/ttypes.py +++ metastore/src/gen/thrift/gen-py/hive_metastore/ttypes.py @@ -69,6 +69,91 @@ class PartitionEventType: "LOAD_DONE": 1, } +class TxnState: + COMMITTED = 1 + ABORTED = 2 + OPEN = 3 + + _VALUES_TO_NAMES = { + 1: "COMMITTED", + 2: "ABORTED", + 3: "OPEN", + } + + _NAMES_TO_VALUES = { + "COMMITTED": 1, + "ABORTED": 2, + "OPEN": 3, + } + +class LockLevel: + DB = 1 + TABLE = 2 + PARTITION = 3 + + _VALUES_TO_NAMES = { + 1: "DB", + 2: "TABLE", + 3: "PARTITION", + } + + _NAMES_TO_VALUES = { + "DB": 1, + "TABLE": 2, + "PARTITION": 3, + } + +class LockState: + ACQUIRED = 1 + WAITING = 2 + ABORT = 3 + NOT_ACQUIRED = 4 + + _VALUES_TO_NAMES = { + 1: "ACQUIRED", + 2: "WAITING", + 3: "ABORT", + 4: "NOT_ACQUIRED", + } + + _NAMES_TO_VALUES = { + "ACQUIRED": 1, + "WAITING": 2, + "ABORT": 3, + "NOT_ACQUIRED": 4, + } + +class LockType: + SHARED_READ = 1 + SHARED_WRITE = 2 + EXCLUSIVE = 3 + + _VALUES_TO_NAMES = { + 1: "SHARED_READ", + 2: "SHARED_WRITE", + 3: "EXCLUSIVE", + } + + _NAMES_TO_VALUES = { + "SHARED_READ": 1, + "SHARED_WRITE": 2, + "EXCLUSIVE": 3, + } + +class CompactionType: + MINOR = 1 + MAJOR = 2 + + _VALUES_TO_NAMES = { + 1: "MINOR", + 2: "MAJOR", + } + + _NAMES_TO_VALUES = { + "MINOR": 1, + "MAJOR": 2, + } + class FunctionType: JAVA = 1 @@ -4697,19 +4782,28 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class MetaException(TException): +class TxnInfo: """ Attributes: - - message + - id + - state + - user + - hostname """ thrift_spec = ( None, # 0 - (1, TType.STRING, 'message', None, None, ), # 1 + (1, TType.I64, 'id', None, None, ), # 1 + (2, TType.I32, 'state', None, None, ), # 2 + (3, TType.STRING, 'user', None, None, ), # 3 + (4, TType.STRING, 'hostname', None, None, ), # 4 ) - def __init__(self, message=None,): - self.message = message + def __init__(self, id=None, state=None, user=None, hostname=None,): + self.id = id + self.state = state + self.user = user + self.hostname = hostname def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -4721,8 +4815,23 @@ def read(self, iprot): if ftype == TType.STOP: break if fid == 1: + if ftype == TType.I64: + self.id = iprot.readI64(); + else: + iprot.skip(ftype) + elif fid == 2: + if ftype == TType.I32: + self.state = iprot.readI32(); + else: + iprot.skip(ftype) + elif fid == 3: if ftype == TType.STRING: - self.message = iprot.readString(); + self.user = iprot.readString(); + else: + iprot.skip(ftype) + elif fid == 4: + if ftype == TType.STRING: + self.hostname = iprot.readString(); else: iprot.skip(ftype) else: @@ -4734,21 +4843,38 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('MetaException') - if self.message is not None: - oprot.writeFieldBegin('message', TType.STRING, 1) - oprot.writeString(self.message) + oprot.writeStructBegin('TxnInfo') + if self.id is not None: + oprot.writeFieldBegin('id', TType.I64, 1) + oprot.writeI64(self.id) + oprot.writeFieldEnd() + if self.state is not None: + oprot.writeFieldBegin('state', TType.I32, 2) + oprot.writeI32(self.state) + oprot.writeFieldEnd() + if self.user is not None: + oprot.writeFieldBegin('user', TType.STRING, 3) + oprot.writeString(self.user) + oprot.writeFieldEnd() + if self.hostname is not None: + oprot.writeFieldBegin('hostname', TType.STRING, 4) + oprot.writeString(self.hostname) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() def validate(self): + if self.id is None: + raise TProtocol.TProtocolException(message='Required field id is unset!') + if self.state is None: + raise TProtocol.TProtocolException(message='Required field state is unset!') + if self.user is None: + raise TProtocol.TProtocolException(message='Required field user is unset!') + if self.hostname is None: + raise TProtocol.TProtocolException(message='Required field hostname is unset!') return - def __str__(self): - return repr(self) - def __repr__(self): L = ['%s=%r' % (key, value) for key, value in self.__dict__.iteritems()] @@ -4760,19 +4886,22 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class UnknownTableException(TException): +class GetOpenTxnsInfoResponse: """ Attributes: - - message + - txn_high_water_mark + - open_txns """ thrift_spec = ( None, # 0 - (1, TType.STRING, 'message', None, None, ), # 1 + (1, TType.I64, 'txn_high_water_mark', None, None, ), # 1 + (2, TType.LIST, 'open_txns', (TType.STRUCT,(TxnInfo, TxnInfo.thrift_spec)), None, ), # 2 ) - def __init__(self, message=None,): - self.message = message + def __init__(self, txn_high_water_mark=None, open_txns=None,): + self.txn_high_water_mark = txn_high_water_mark + self.open_txns = open_txns def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -4784,8 +4913,19 @@ def read(self, iprot): if ftype == TType.STOP: break if fid == 1: - if ftype == TType.STRING: - self.message = iprot.readString(); + if ftype == TType.I64: + self.txn_high_water_mark = iprot.readI64(); + else: + iprot.skip(ftype) + elif fid == 2: + if ftype == TType.LIST: + self.open_txns = [] + (_etype323, _size320) = iprot.readListBegin() + for _i324 in xrange(_size320): + _elem325 = TxnInfo() + _elem325.read(iprot) + self.open_txns.append(_elem325) + iprot.readListEnd() else: iprot.skip(ftype) else: @@ -4797,21 +4937,29 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('UnknownTableException') - if self.message is not None: - oprot.writeFieldBegin('message', TType.STRING, 1) - oprot.writeString(self.message) + oprot.writeStructBegin('GetOpenTxnsInfoResponse') + if self.txn_high_water_mark is not None: + oprot.writeFieldBegin('txn_high_water_mark', TType.I64, 1) + oprot.writeI64(self.txn_high_water_mark) + oprot.writeFieldEnd() + if self.open_txns is not None: + oprot.writeFieldBegin('open_txns', TType.LIST, 2) + oprot.writeListBegin(TType.STRUCT, len(self.open_txns)) + for iter326 in self.open_txns: + iter326.write(oprot) + oprot.writeListEnd() oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() def validate(self): + if self.txn_high_water_mark is None: + raise TProtocol.TProtocolException(message='Required field txn_high_water_mark is unset!') + if self.open_txns is None: + raise TProtocol.TProtocolException(message='Required field open_txns is unset!') return - def __str__(self): - return repr(self) - def __repr__(self): L = ['%s=%r' % (key, value) for key, value in self.__dict__.iteritems()] @@ -4823,19 +4971,22 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class UnknownDBException(TException): +class GetOpenTxnsResponse: """ Attributes: - - message + - txn_high_water_mark + - open_txns """ thrift_spec = ( None, # 0 - (1, TType.STRING, 'message', None, None, ), # 1 + (1, TType.I64, 'txn_high_water_mark', None, None, ), # 1 + (2, TType.SET, 'open_txns', (TType.I64,None), None, ), # 2 ) - def __init__(self, message=None,): - self.message = message + def __init__(self, txn_high_water_mark=None, open_txns=None,): + self.txn_high_water_mark = txn_high_water_mark + self.open_txns = open_txns def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -4847,8 +4998,18 @@ def read(self, iprot): if ftype == TType.STOP: break if fid == 1: - if ftype == TType.STRING: - self.message = iprot.readString(); + if ftype == TType.I64: + self.txn_high_water_mark = iprot.readI64(); + else: + iprot.skip(ftype) + elif fid == 2: + if ftype == TType.SET: + self.open_txns = set() + (_etype330, _size327) = iprot.readSetBegin() + for _i331 in xrange(_size327): + _elem332 = iprot.readI64(); + self.open_txns.add(_elem332) + iprot.readSetEnd() else: iprot.skip(ftype) else: @@ -4860,21 +5021,29 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('UnknownDBException') - if self.message is not None: - oprot.writeFieldBegin('message', TType.STRING, 1) - oprot.writeString(self.message) + oprot.writeStructBegin('GetOpenTxnsResponse') + if self.txn_high_water_mark is not None: + oprot.writeFieldBegin('txn_high_water_mark', TType.I64, 1) + oprot.writeI64(self.txn_high_water_mark) + oprot.writeFieldEnd() + if self.open_txns is not None: + oprot.writeFieldBegin('open_txns', TType.SET, 2) + oprot.writeSetBegin(TType.I64, len(self.open_txns)) + for iter333 in self.open_txns: + oprot.writeI64(iter333) + oprot.writeSetEnd() oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() def validate(self): + if self.txn_high_water_mark is None: + raise TProtocol.TProtocolException(message='Required field txn_high_water_mark is unset!') + if self.open_txns is None: + raise TProtocol.TProtocolException(message='Required field open_txns is unset!') return - def __str__(self): - return repr(self) - def __repr__(self): L = ['%s=%r' % (key, value) for key, value in self.__dict__.iteritems()] @@ -4886,19 +5055,25 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class AlreadyExistsException(TException): +class OpenTxnRequest: """ Attributes: - - message + - num_txns + - user + - hostname """ thrift_spec = ( None, # 0 - (1, TType.STRING, 'message', None, None, ), # 1 + (1, TType.I32, 'num_txns', None, None, ), # 1 + (2, TType.STRING, 'user', None, None, ), # 2 + (3, TType.STRING, 'hostname', None, None, ), # 3 ) - def __init__(self, message=None,): - self.message = message + def __init__(self, num_txns=None, user=None, hostname=None,): + self.num_txns = num_txns + self.user = user + self.hostname = hostname def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -4910,8 +5085,18 @@ def read(self, iprot): if ftype == TType.STOP: break if fid == 1: + if ftype == TType.I32: + self.num_txns = iprot.readI32(); + else: + iprot.skip(ftype) + elif fid == 2: if ftype == TType.STRING: - self.message = iprot.readString(); + self.user = iprot.readString(); + else: + iprot.skip(ftype) + elif fid == 3: + if ftype == TType.STRING: + self.hostname = iprot.readString(); else: iprot.skip(ftype) else: @@ -4923,21 +5108,32 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('AlreadyExistsException') - if self.message is not None: - oprot.writeFieldBegin('message', TType.STRING, 1) - oprot.writeString(self.message) + oprot.writeStructBegin('OpenTxnRequest') + if self.num_txns is not None: + oprot.writeFieldBegin('num_txns', TType.I32, 1) + oprot.writeI32(self.num_txns) + oprot.writeFieldEnd() + if self.user is not None: + oprot.writeFieldBegin('user', TType.STRING, 2) + oprot.writeString(self.user) + oprot.writeFieldEnd() + if self.hostname is not None: + oprot.writeFieldBegin('hostname', TType.STRING, 3) + oprot.writeString(self.hostname) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() def validate(self): + if self.num_txns is None: + raise TProtocol.TProtocolException(message='Required field num_txns is unset!') + if self.user is None: + raise TProtocol.TProtocolException(message='Required field user is unset!') + if self.hostname is None: + raise TProtocol.TProtocolException(message='Required field hostname is unset!') return - def __str__(self): - return repr(self) - def __repr__(self): L = ['%s=%r' % (key, value) for key, value in self.__dict__.iteritems()] @@ -4949,19 +5145,19 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class InvalidPartitionException(TException): +class OpenTxnsResponse: """ Attributes: - - message + - txn_ids """ thrift_spec = ( None, # 0 - (1, TType.STRING, 'message', None, None, ), # 1 + (1, TType.LIST, 'txn_ids', (TType.I64,None), None, ), # 1 ) - def __init__(self, message=None,): - self.message = message + def __init__(self, txn_ids=None,): + self.txn_ids = txn_ids def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -4973,8 +5169,13 @@ def read(self, iprot): if ftype == TType.STOP: break if fid == 1: - if ftype == TType.STRING: - self.message = iprot.readString(); + if ftype == TType.LIST: + self.txn_ids = [] + (_etype337, _size334) = iprot.readListBegin() + for _i338 in xrange(_size334): + _elem339 = iprot.readI64(); + self.txn_ids.append(_elem339) + iprot.readListEnd() else: iprot.skip(ftype) else: @@ -4986,21 +5187,23 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('InvalidPartitionException') - if self.message is not None: - oprot.writeFieldBegin('message', TType.STRING, 1) - oprot.writeString(self.message) + oprot.writeStructBegin('OpenTxnsResponse') + if self.txn_ids is not None: + oprot.writeFieldBegin('txn_ids', TType.LIST, 1) + oprot.writeListBegin(TType.I64, len(self.txn_ids)) + for iter340 in self.txn_ids: + oprot.writeI64(iter340) + oprot.writeListEnd() oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() def validate(self): + if self.txn_ids is None: + raise TProtocol.TProtocolException(message='Required field txn_ids is unset!') return - def __str__(self): - return repr(self) - def __repr__(self): L = ['%s=%r' % (key, value) for key, value in self.__dict__.iteritems()] @@ -5012,19 +5215,19 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class UnknownPartitionException(TException): +class AbortTxnRequest: """ Attributes: - - message + - txnid """ thrift_spec = ( None, # 0 - (1, TType.STRING, 'message', None, None, ), # 1 + (1, TType.I64, 'txnid', None, None, ), # 1 ) - def __init__(self, message=None,): - self.message = message + def __init__(self, txnid=None,): + self.txnid = txnid def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -5036,8 +5239,8 @@ def read(self, iprot): if ftype == TType.STOP: break if fid == 1: - if ftype == TType.STRING: - self.message = iprot.readString(); + if ftype == TType.I64: + self.txnid = iprot.readI64(); else: iprot.skip(ftype) else: @@ -5049,21 +5252,20 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('UnknownPartitionException') - if self.message is not None: - oprot.writeFieldBegin('message', TType.STRING, 1) - oprot.writeString(self.message) + oprot.writeStructBegin('AbortTxnRequest') + if self.txnid is not None: + oprot.writeFieldBegin('txnid', TType.I64, 1) + oprot.writeI64(self.txnid) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() def validate(self): + if self.txnid is None: + raise TProtocol.TProtocolException(message='Required field txnid is unset!') return - def __str__(self): - return repr(self) - def __repr__(self): L = ['%s=%r' % (key, value) for key, value in self.__dict__.iteritems()] @@ -5075,19 +5277,19 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class InvalidObjectException(TException): +class CommitTxnRequest: """ Attributes: - - message + - txnid """ thrift_spec = ( None, # 0 - (1, TType.STRING, 'message', None, None, ), # 1 + (1, TType.I64, 'txnid', None, None, ), # 1 ) - def __init__(self, message=None,): - self.message = message + def __init__(self, txnid=None,): + self.txnid = txnid def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -5099,8 +5301,8 @@ def read(self, iprot): if ftype == TType.STOP: break if fid == 1: - if ftype == TType.STRING: - self.message = iprot.readString(); + if ftype == TType.I64: + self.txnid = iprot.readI64(); else: iprot.skip(ftype) else: @@ -5112,21 +5314,20 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('InvalidObjectException') - if self.message is not None: - oprot.writeFieldBegin('message', TType.STRING, 1) - oprot.writeString(self.message) + oprot.writeStructBegin('CommitTxnRequest') + if self.txnid is not None: + oprot.writeFieldBegin('txnid', TType.I64, 1) + oprot.writeI64(self.txnid) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() def validate(self): + if self.txnid is None: + raise TProtocol.TProtocolException(message='Required field txnid is unset!') return - def __str__(self): - return repr(self) - def __repr__(self): L = ['%s=%r' % (key, value) for key, value in self.__dict__.iteritems()] @@ -5138,19 +5339,31 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class NoSuchObjectException(TException): +class LockComponent: """ Attributes: - - message + - type + - level + - dbname + - tablename + - partitionname """ thrift_spec = ( None, # 0 - (1, TType.STRING, 'message', None, None, ), # 1 + (1, TType.I32, 'type', None, None, ), # 1 + (2, TType.I32, 'level', None, None, ), # 2 + (3, TType.STRING, 'dbname', None, None, ), # 3 + (4, TType.STRING, 'tablename', None, None, ), # 4 + (5, TType.STRING, 'partitionname', None, None, ), # 5 ) - def __init__(self, message=None,): - self.message = message + def __init__(self, type=None, level=None, dbname=None, tablename=None, partitionname=None,): + self.type = type + self.level = level + self.dbname = dbname + self.tablename = tablename + self.partitionname = partitionname def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -5162,8 +5375,28 @@ def read(self, iprot): if ftype == TType.STOP: break if fid == 1: + if ftype == TType.I32: + self.type = iprot.readI32(); + else: + iprot.skip(ftype) + elif fid == 2: + if ftype == TType.I32: + self.level = iprot.readI32(); + else: + iprot.skip(ftype) + elif fid == 3: if ftype == TType.STRING: - self.message = iprot.readString(); + self.dbname = iprot.readString(); + else: + iprot.skip(ftype) + elif fid == 4: + if ftype == TType.STRING: + self.tablename = iprot.readString(); + else: + iprot.skip(ftype) + elif fid == 5: + if ftype == TType.STRING: + self.partitionname = iprot.readString(); else: iprot.skip(ftype) else: @@ -5175,21 +5408,40 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('NoSuchObjectException') - if self.message is not None: - oprot.writeFieldBegin('message', TType.STRING, 1) - oprot.writeString(self.message) + oprot.writeStructBegin('LockComponent') + if self.type is not None: + oprot.writeFieldBegin('type', TType.I32, 1) + oprot.writeI32(self.type) + oprot.writeFieldEnd() + if self.level is not None: + oprot.writeFieldBegin('level', TType.I32, 2) + oprot.writeI32(self.level) + oprot.writeFieldEnd() + if self.dbname is not None: + oprot.writeFieldBegin('dbname', TType.STRING, 3) + oprot.writeString(self.dbname) + oprot.writeFieldEnd() + if self.tablename is not None: + oprot.writeFieldBegin('tablename', TType.STRING, 4) + oprot.writeString(self.tablename) + oprot.writeFieldEnd() + if self.partitionname is not None: + oprot.writeFieldBegin('partitionname', TType.STRING, 5) + oprot.writeString(self.partitionname) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() def validate(self): + if self.type is None: + raise TProtocol.TProtocolException(message='Required field type is unset!') + if self.level is None: + raise TProtocol.TProtocolException(message='Required field level is unset!') + if self.dbname is None: + raise TProtocol.TProtocolException(message='Required field dbname is unset!') return - def __str__(self): - return repr(self) - def __repr__(self): L = ['%s=%r' % (key, value) for key, value in self.__dict__.iteritems()] @@ -5201,19 +5453,28 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class IndexAlreadyExistsException(TException): +class LockRequest: """ Attributes: - - message + - component + - txnid + - user + - hostname """ thrift_spec = ( None, # 0 - (1, TType.STRING, 'message', None, None, ), # 1 + (1, TType.LIST, 'component', (TType.STRUCT,(LockComponent, LockComponent.thrift_spec)), None, ), # 1 + (2, TType.I64, 'txnid', None, None, ), # 2 + (3, TType.STRING, 'user', None, None, ), # 3 + (4, TType.STRING, 'hostname', None, None, ), # 4 ) - def __init__(self, message=None,): - self.message = message + def __init__(self, component=None, txnid=None, user=None, hostname=None,): + self.component = component + self.txnid = txnid + self.user = user + self.hostname = hostname def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -5225,8 +5486,29 @@ def read(self, iprot): if ftype == TType.STOP: break if fid == 1: + if ftype == TType.LIST: + self.component = [] + (_etype344, _size341) = iprot.readListBegin() + for _i345 in xrange(_size341): + _elem346 = LockComponent() + _elem346.read(iprot) + self.component.append(_elem346) + iprot.readListEnd() + else: + iprot.skip(ftype) + elif fid == 2: + if ftype == TType.I64: + self.txnid = iprot.readI64(); + else: + iprot.skip(ftype) + elif fid == 3: if ftype == TType.STRING: - self.message = iprot.readString(); + self.user = iprot.readString(); + else: + iprot.skip(ftype) + elif fid == 4: + if ftype == TType.STRING: + self.hostname = iprot.readString(); else: iprot.skip(ftype) else: @@ -5238,21 +5520,39 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('IndexAlreadyExistsException') - if self.message is not None: - oprot.writeFieldBegin('message', TType.STRING, 1) - oprot.writeString(self.message) + oprot.writeStructBegin('LockRequest') + if self.component is not None: + oprot.writeFieldBegin('component', TType.LIST, 1) + oprot.writeListBegin(TType.STRUCT, len(self.component)) + for iter347 in self.component: + iter347.write(oprot) + oprot.writeListEnd() + oprot.writeFieldEnd() + if self.txnid is not None: + oprot.writeFieldBegin('txnid', TType.I64, 2) + oprot.writeI64(self.txnid) + oprot.writeFieldEnd() + if self.user is not None: + oprot.writeFieldBegin('user', TType.STRING, 3) + oprot.writeString(self.user) + oprot.writeFieldEnd() + if self.hostname is not None: + oprot.writeFieldBegin('hostname', TType.STRING, 4) + oprot.writeString(self.hostname) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() def validate(self): + if self.component is None: + raise TProtocol.TProtocolException(message='Required field component is unset!') + if self.user is None: + raise TProtocol.TProtocolException(message='Required field user is unset!') + if self.hostname is None: + raise TProtocol.TProtocolException(message='Required field hostname is unset!') return - def __str__(self): - return repr(self) - def __repr__(self): L = ['%s=%r' % (key, value) for key, value in self.__dict__.iteritems()] @@ -5264,19 +5564,22 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class InvalidOperationException(TException): +class LockResponse: """ Attributes: - - message + - lockid + - state """ thrift_spec = ( None, # 0 - (1, TType.STRING, 'message', None, None, ), # 1 + (1, TType.I64, 'lockid', None, None, ), # 1 + (2, TType.I32, 'state', None, None, ), # 2 ) - def __init__(self, message=None,): - self.message = message + def __init__(self, lockid=None, state=None,): + self.lockid = lockid + self.state = state def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -5288,8 +5591,13 @@ def read(self, iprot): if ftype == TType.STOP: break if fid == 1: - if ftype == TType.STRING: - self.message = iprot.readString(); + if ftype == TType.I64: + self.lockid = iprot.readI64(); + else: + iprot.skip(ftype) + elif fid == 2: + if ftype == TType.I32: + self.state = iprot.readI32(); else: iprot.skip(ftype) else: @@ -5301,19 +5609,1542 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('InvalidOperationException') - if self.message is not None: - oprot.writeFieldBegin('message', TType.STRING, 1) - oprot.writeString(self.message) + oprot.writeStructBegin('LockResponse') + if self.lockid is not None: + oprot.writeFieldBegin('lockid', TType.I64, 1) + oprot.writeI64(self.lockid) + oprot.writeFieldEnd() + if self.state is not None: + oprot.writeFieldBegin('state', TType.I32, 2) + oprot.writeI32(self.state) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() def validate(self): + if self.lockid is None: + raise TProtocol.TProtocolException(message='Required field lockid is unset!') + if self.state is None: + raise TProtocol.TProtocolException(message='Required field state is unset!') return - def __str__(self): + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class CheckLockRequest: + """ + Attributes: + - lockid + """ + + thrift_spec = ( + None, # 0 + (1, TType.I64, 'lockid', None, None, ), # 1 + ) + + def __init__(self, lockid=None,): + self.lockid = lockid + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.I64: + self.lockid = iprot.readI64(); + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('CheckLockRequest') + if self.lockid is not None: + oprot.writeFieldBegin('lockid', TType.I64, 1) + oprot.writeI64(self.lockid) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + if self.lockid is None: + raise TProtocol.TProtocolException(message='Required field lockid is unset!') + return + + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class UnlockRequest: + """ + Attributes: + - lockid + """ + + thrift_spec = ( + None, # 0 + (1, TType.I64, 'lockid', None, None, ), # 1 + ) + + def __init__(self, lockid=None,): + self.lockid = lockid + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.I64: + self.lockid = iprot.readI64(); + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('UnlockRequest') + if self.lockid is not None: + oprot.writeFieldBegin('lockid', TType.I64, 1) + oprot.writeI64(self.lockid) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + if self.lockid is None: + raise TProtocol.TProtocolException(message='Required field lockid is unset!') + return + + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class ShowLocksRequest: + + thrift_spec = ( + ) + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('ShowLocksRequest') + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class ShowLocksResponseElement: + """ + Attributes: + - lockid + - dbname + - tablename + - partname + - state + - type + - txnid + - lastheartbeat + - acquiredat + - user + - hostname + """ + + thrift_spec = ( + None, # 0 + (1, TType.I64, 'lockid', None, None, ), # 1 + (2, TType.STRING, 'dbname', None, None, ), # 2 + (3, TType.STRING, 'tablename', None, None, ), # 3 + (4, TType.STRING, 'partname', None, None, ), # 4 + (5, TType.I32, 'state', None, None, ), # 5 + (6, TType.I32, 'type', None, None, ), # 6 + (7, TType.I64, 'txnid', None, None, ), # 7 + (8, TType.I64, 'lastheartbeat', None, None, ), # 8 + (9, TType.I64, 'acquiredat', None, None, ), # 9 + (10, TType.STRING, 'user', None, None, ), # 10 + (11, TType.STRING, 'hostname', None, None, ), # 11 + ) + + def __init__(self, lockid=None, dbname=None, tablename=None, partname=None, state=None, type=None, txnid=None, lastheartbeat=None, acquiredat=None, user=None, hostname=None,): + self.lockid = lockid + self.dbname = dbname + self.tablename = tablename + self.partname = partname + self.state = state + self.type = type + self.txnid = txnid + self.lastheartbeat = lastheartbeat + self.acquiredat = acquiredat + self.user = user + self.hostname = hostname + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.I64: + self.lockid = iprot.readI64(); + else: + iprot.skip(ftype) + elif fid == 2: + if ftype == TType.STRING: + self.dbname = iprot.readString(); + else: + iprot.skip(ftype) + elif fid == 3: + if ftype == TType.STRING: + self.tablename = iprot.readString(); + else: + iprot.skip(ftype) + elif fid == 4: + if ftype == TType.STRING: + self.partname = iprot.readString(); + else: + iprot.skip(ftype) + elif fid == 5: + if ftype == TType.I32: + self.state = iprot.readI32(); + else: + iprot.skip(ftype) + elif fid == 6: + if ftype == TType.I32: + self.type = iprot.readI32(); + else: + iprot.skip(ftype) + elif fid == 7: + if ftype == TType.I64: + self.txnid = iprot.readI64(); + else: + iprot.skip(ftype) + elif fid == 8: + if ftype == TType.I64: + self.lastheartbeat = iprot.readI64(); + else: + iprot.skip(ftype) + elif fid == 9: + if ftype == TType.I64: + self.acquiredat = iprot.readI64(); + else: + iprot.skip(ftype) + elif fid == 10: + if ftype == TType.STRING: + self.user = iprot.readString(); + else: + iprot.skip(ftype) + elif fid == 11: + if ftype == TType.STRING: + self.hostname = iprot.readString(); + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('ShowLocksResponseElement') + if self.lockid is not None: + oprot.writeFieldBegin('lockid', TType.I64, 1) + oprot.writeI64(self.lockid) + oprot.writeFieldEnd() + if self.dbname is not None: + oprot.writeFieldBegin('dbname', TType.STRING, 2) + oprot.writeString(self.dbname) + oprot.writeFieldEnd() + if self.tablename is not None: + oprot.writeFieldBegin('tablename', TType.STRING, 3) + oprot.writeString(self.tablename) + oprot.writeFieldEnd() + if self.partname is not None: + oprot.writeFieldBegin('partname', TType.STRING, 4) + oprot.writeString(self.partname) + oprot.writeFieldEnd() + if self.state is not None: + oprot.writeFieldBegin('state', TType.I32, 5) + oprot.writeI32(self.state) + oprot.writeFieldEnd() + if self.type is not None: + oprot.writeFieldBegin('type', TType.I32, 6) + oprot.writeI32(self.type) + oprot.writeFieldEnd() + if self.txnid is not None: + oprot.writeFieldBegin('txnid', TType.I64, 7) + oprot.writeI64(self.txnid) + oprot.writeFieldEnd() + if self.lastheartbeat is not None: + oprot.writeFieldBegin('lastheartbeat', TType.I64, 8) + oprot.writeI64(self.lastheartbeat) + oprot.writeFieldEnd() + if self.acquiredat is not None: + oprot.writeFieldBegin('acquiredat', TType.I64, 9) + oprot.writeI64(self.acquiredat) + oprot.writeFieldEnd() + if self.user is not None: + oprot.writeFieldBegin('user', TType.STRING, 10) + oprot.writeString(self.user) + oprot.writeFieldEnd() + if self.hostname is not None: + oprot.writeFieldBegin('hostname', TType.STRING, 11) + oprot.writeString(self.hostname) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + if self.lockid is None: + raise TProtocol.TProtocolException(message='Required field lockid is unset!') + if self.dbname is None: + raise TProtocol.TProtocolException(message='Required field dbname is unset!') + if self.state is None: + raise TProtocol.TProtocolException(message='Required field state is unset!') + if self.type is None: + raise TProtocol.TProtocolException(message='Required field type is unset!') + if self.lastheartbeat is None: + raise TProtocol.TProtocolException(message='Required field lastheartbeat is unset!') + if self.user is None: + raise TProtocol.TProtocolException(message='Required field user is unset!') + if self.hostname is None: + raise TProtocol.TProtocolException(message='Required field hostname is unset!') + return + + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class ShowLocksResponse: + """ + Attributes: + - locks + """ + + thrift_spec = ( + None, # 0 + (1, TType.LIST, 'locks', (TType.STRUCT,(ShowLocksResponseElement, ShowLocksResponseElement.thrift_spec)), None, ), # 1 + ) + + def __init__(self, locks=None,): + self.locks = locks + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.LIST: + self.locks = [] + (_etype351, _size348) = iprot.readListBegin() + for _i352 in xrange(_size348): + _elem353 = ShowLocksResponseElement() + _elem353.read(iprot) + self.locks.append(_elem353) + iprot.readListEnd() + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('ShowLocksResponse') + if self.locks is not None: + oprot.writeFieldBegin('locks', TType.LIST, 1) + oprot.writeListBegin(TType.STRUCT, len(self.locks)) + for iter354 in self.locks: + iter354.write(oprot) + oprot.writeListEnd() + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class HeartbeatRequest: + """ + Attributes: + - lockid + - txnid + """ + + thrift_spec = ( + None, # 0 + (1, TType.I64, 'lockid', None, None, ), # 1 + (2, TType.I64, 'txnid', None, None, ), # 2 + ) + + def __init__(self, lockid=None, txnid=None,): + self.lockid = lockid + self.txnid = txnid + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.I64: + self.lockid = iprot.readI64(); + else: + iprot.skip(ftype) + elif fid == 2: + if ftype == TType.I64: + self.txnid = iprot.readI64(); + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('HeartbeatRequest') + if self.lockid is not None: + oprot.writeFieldBegin('lockid', TType.I64, 1) + oprot.writeI64(self.lockid) + oprot.writeFieldEnd() + if self.txnid is not None: + oprot.writeFieldBegin('txnid', TType.I64, 2) + oprot.writeI64(self.txnid) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class CompactionRequest: + """ + Attributes: + - dbname + - tablename + - partitionname + - type + - runas + """ + + thrift_spec = ( + None, # 0 + (1, TType.STRING, 'dbname', None, None, ), # 1 + (2, TType.STRING, 'tablename', None, None, ), # 2 + (3, TType.STRING, 'partitionname', None, None, ), # 3 + (4, TType.I32, 'type', None, None, ), # 4 + (5, TType.STRING, 'runas', None, None, ), # 5 + ) + + def __init__(self, dbname=None, tablename=None, partitionname=None, type=None, runas=None,): + self.dbname = dbname + self.tablename = tablename + self.partitionname = partitionname + self.type = type + self.runas = runas + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.STRING: + self.dbname = iprot.readString(); + else: + iprot.skip(ftype) + elif fid == 2: + if ftype == TType.STRING: + self.tablename = iprot.readString(); + else: + iprot.skip(ftype) + elif fid == 3: + if ftype == TType.STRING: + self.partitionname = iprot.readString(); + else: + iprot.skip(ftype) + elif fid == 4: + if ftype == TType.I32: + self.type = iprot.readI32(); + else: + iprot.skip(ftype) + elif fid == 5: + if ftype == TType.STRING: + self.runas = iprot.readString(); + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('CompactionRequest') + if self.dbname is not None: + oprot.writeFieldBegin('dbname', TType.STRING, 1) + oprot.writeString(self.dbname) + oprot.writeFieldEnd() + if self.tablename is not None: + oprot.writeFieldBegin('tablename', TType.STRING, 2) + oprot.writeString(self.tablename) + oprot.writeFieldEnd() + if self.partitionname is not None: + oprot.writeFieldBegin('partitionname', TType.STRING, 3) + oprot.writeString(self.partitionname) + oprot.writeFieldEnd() + if self.type is not None: + oprot.writeFieldBegin('type', TType.I32, 4) + oprot.writeI32(self.type) + oprot.writeFieldEnd() + if self.runas is not None: + oprot.writeFieldBegin('runas', TType.STRING, 5) + oprot.writeString(self.runas) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + if self.dbname is None: + raise TProtocol.TProtocolException(message='Required field dbname is unset!') + if self.tablename is None: + raise TProtocol.TProtocolException(message='Required field tablename is unset!') + if self.type is None: + raise TProtocol.TProtocolException(message='Required field type is unset!') + return + + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class ShowCompactRequest: + + thrift_spec = ( + ) + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('ShowCompactRequest') + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class ShowCompactResponseElement: + """ + Attributes: + - dbname + - tablename + - partitionname + - type + - state + - workerid + - start + - runAs + """ + + thrift_spec = ( + None, # 0 + (1, TType.STRING, 'dbname', None, None, ), # 1 + (2, TType.STRING, 'tablename', None, None, ), # 2 + (3, TType.STRING, 'partitionname', None, None, ), # 3 + (4, TType.I32, 'type', None, None, ), # 4 + (5, TType.STRING, 'state', None, None, ), # 5 + (6, TType.STRING, 'workerid', None, None, ), # 6 + (7, TType.I64, 'start', None, None, ), # 7 + (8, TType.STRING, 'runAs', None, None, ), # 8 + ) + + def __init__(self, dbname=None, tablename=None, partitionname=None, type=None, state=None, workerid=None, start=None, runAs=None,): + self.dbname = dbname + self.tablename = tablename + self.partitionname = partitionname + self.type = type + self.state = state + self.workerid = workerid + self.start = start + self.runAs = runAs + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.STRING: + self.dbname = iprot.readString(); + else: + iprot.skip(ftype) + elif fid == 2: + if ftype == TType.STRING: + self.tablename = iprot.readString(); + else: + iprot.skip(ftype) + elif fid == 3: + if ftype == TType.STRING: + self.partitionname = iprot.readString(); + else: + iprot.skip(ftype) + elif fid == 4: + if ftype == TType.I32: + self.type = iprot.readI32(); + else: + iprot.skip(ftype) + elif fid == 5: + if ftype == TType.STRING: + self.state = iprot.readString(); + else: + iprot.skip(ftype) + elif fid == 6: + if ftype == TType.STRING: + self.workerid = iprot.readString(); + else: + iprot.skip(ftype) + elif fid == 7: + if ftype == TType.I64: + self.start = iprot.readI64(); + else: + iprot.skip(ftype) + elif fid == 8: + if ftype == TType.STRING: + self.runAs = iprot.readString(); + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('ShowCompactResponseElement') + if self.dbname is not None: + oprot.writeFieldBegin('dbname', TType.STRING, 1) + oprot.writeString(self.dbname) + oprot.writeFieldEnd() + if self.tablename is not None: + oprot.writeFieldBegin('tablename', TType.STRING, 2) + oprot.writeString(self.tablename) + oprot.writeFieldEnd() + if self.partitionname is not None: + oprot.writeFieldBegin('partitionname', TType.STRING, 3) + oprot.writeString(self.partitionname) + oprot.writeFieldEnd() + if self.type is not None: + oprot.writeFieldBegin('type', TType.I32, 4) + oprot.writeI32(self.type) + oprot.writeFieldEnd() + if self.state is not None: + oprot.writeFieldBegin('state', TType.STRING, 5) + oprot.writeString(self.state) + oprot.writeFieldEnd() + if self.workerid is not None: + oprot.writeFieldBegin('workerid', TType.STRING, 6) + oprot.writeString(self.workerid) + oprot.writeFieldEnd() + if self.start is not None: + oprot.writeFieldBegin('start', TType.I64, 7) + oprot.writeI64(self.start) + oprot.writeFieldEnd() + if self.runAs is not None: + oprot.writeFieldBegin('runAs', TType.STRING, 8) + oprot.writeString(self.runAs) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + if self.dbname is None: + raise TProtocol.TProtocolException(message='Required field dbname is unset!') + if self.tablename is None: + raise TProtocol.TProtocolException(message='Required field tablename is unset!') + if self.partitionname is None: + raise TProtocol.TProtocolException(message='Required field partitionname is unset!') + if self.type is None: + raise TProtocol.TProtocolException(message='Required field type is unset!') + if self.state is None: + raise TProtocol.TProtocolException(message='Required field state is unset!') + if self.workerid is None: + raise TProtocol.TProtocolException(message='Required field workerid is unset!') + if self.start is None: + raise TProtocol.TProtocolException(message='Required field start is unset!') + if self.runAs is None: + raise TProtocol.TProtocolException(message='Required field runAs is unset!') + return + + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class ShowCompactResponse: + """ + Attributes: + - compacts + """ + + thrift_spec = ( + None, # 0 + (1, TType.LIST, 'compacts', (TType.STRUCT,(ShowCompactResponseElement, ShowCompactResponseElement.thrift_spec)), None, ), # 1 + ) + + def __init__(self, compacts=None,): + self.compacts = compacts + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.LIST: + self.compacts = [] + (_etype358, _size355) = iprot.readListBegin() + for _i359 in xrange(_size355): + _elem360 = ShowCompactResponseElement() + _elem360.read(iprot) + self.compacts.append(_elem360) + iprot.readListEnd() + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('ShowCompactResponse') + if self.compacts is not None: + oprot.writeFieldBegin('compacts', TType.LIST, 1) + oprot.writeListBegin(TType.STRUCT, len(self.compacts)) + for iter361 in self.compacts: + iter361.write(oprot) + oprot.writeListEnd() + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + if self.compacts is None: + raise TProtocol.TProtocolException(message='Required field compacts is unset!') + return + + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class MetaException(TException): + """ + Attributes: + - message + """ + + thrift_spec = ( + None, # 0 + (1, TType.STRING, 'message', None, None, ), # 1 + ) + + def __init__(self, message=None,): + self.message = message + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.STRING: + self.message = iprot.readString(); + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('MetaException') + if self.message is not None: + oprot.writeFieldBegin('message', TType.STRING, 1) + oprot.writeString(self.message) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __str__(self): + return repr(self) + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class UnknownTableException(TException): + """ + Attributes: + - message + """ + + thrift_spec = ( + None, # 0 + (1, TType.STRING, 'message', None, None, ), # 1 + ) + + def __init__(self, message=None,): + self.message = message + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.STRING: + self.message = iprot.readString(); + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('UnknownTableException') + if self.message is not None: + oprot.writeFieldBegin('message', TType.STRING, 1) + oprot.writeString(self.message) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __str__(self): + return repr(self) + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class UnknownDBException(TException): + """ + Attributes: + - message + """ + + thrift_spec = ( + None, # 0 + (1, TType.STRING, 'message', None, None, ), # 1 + ) + + def __init__(self, message=None,): + self.message = message + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.STRING: + self.message = iprot.readString(); + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('UnknownDBException') + if self.message is not None: + oprot.writeFieldBegin('message', TType.STRING, 1) + oprot.writeString(self.message) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __str__(self): + return repr(self) + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class AlreadyExistsException(TException): + """ + Attributes: + - message + """ + + thrift_spec = ( + None, # 0 + (1, TType.STRING, 'message', None, None, ), # 1 + ) + + def __init__(self, message=None,): + self.message = message + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.STRING: + self.message = iprot.readString(); + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('AlreadyExistsException') + if self.message is not None: + oprot.writeFieldBegin('message', TType.STRING, 1) + oprot.writeString(self.message) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __str__(self): + return repr(self) + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class InvalidPartitionException(TException): + """ + Attributes: + - message + """ + + thrift_spec = ( + None, # 0 + (1, TType.STRING, 'message', None, None, ), # 1 + ) + + def __init__(self, message=None,): + self.message = message + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.STRING: + self.message = iprot.readString(); + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('InvalidPartitionException') + if self.message is not None: + oprot.writeFieldBegin('message', TType.STRING, 1) + oprot.writeString(self.message) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __str__(self): + return repr(self) + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class UnknownPartitionException(TException): + """ + Attributes: + - message + """ + + thrift_spec = ( + None, # 0 + (1, TType.STRING, 'message', None, None, ), # 1 + ) + + def __init__(self, message=None,): + self.message = message + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.STRING: + self.message = iprot.readString(); + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('UnknownPartitionException') + if self.message is not None: + oprot.writeFieldBegin('message', TType.STRING, 1) + oprot.writeString(self.message) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __str__(self): + return repr(self) + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class InvalidObjectException(TException): + """ + Attributes: + - message + """ + + thrift_spec = ( + None, # 0 + (1, TType.STRING, 'message', None, None, ), # 1 + ) + + def __init__(self, message=None,): + self.message = message + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.STRING: + self.message = iprot.readString(); + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('InvalidObjectException') + if self.message is not None: + oprot.writeFieldBegin('message', TType.STRING, 1) + oprot.writeString(self.message) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __str__(self): + return repr(self) + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class NoSuchObjectException(TException): + """ + Attributes: + - message + """ + + thrift_spec = ( + None, # 0 + (1, TType.STRING, 'message', None, None, ), # 1 + ) + + def __init__(self, message=None,): + self.message = message + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.STRING: + self.message = iprot.readString(); + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('NoSuchObjectException') + if self.message is not None: + oprot.writeFieldBegin('message', TType.STRING, 1) + oprot.writeString(self.message) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __str__(self): + return repr(self) + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class IndexAlreadyExistsException(TException): + """ + Attributes: + - message + """ + + thrift_spec = ( + None, # 0 + (1, TType.STRING, 'message', None, None, ), # 1 + ) + + def __init__(self, message=None,): + self.message = message + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.STRING: + self.message = iprot.readString(); + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('IndexAlreadyExistsException') + if self.message is not None: + oprot.writeFieldBegin('message', TType.STRING, 1) + oprot.writeString(self.message) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __str__(self): + return repr(self) + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class InvalidOperationException(TException): + """ + Attributes: + - message + """ + + thrift_spec = ( + None, # 0 + (1, TType.STRING, 'message', None, None, ), # 1 + ) + + def __init__(self, message=None,): + self.message = message + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.STRING: + self.message = iprot.readString(); + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('InvalidOperationException') + if self.message is not None: + oprot.writeFieldBegin('message', TType.STRING, 1) + oprot.writeString(self.message) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __str__(self): return repr(self) def __repr__(self): @@ -5452,3 +7283,255 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) + +class NoSuchTxnException(TException): + """ + Attributes: + - message + """ + + thrift_spec = ( + None, # 0 + (1, TType.STRING, 'message', None, None, ), # 1 + ) + + def __init__(self, message=None,): + self.message = message + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.STRING: + self.message = iprot.readString(); + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('NoSuchTxnException') + if self.message is not None: + oprot.writeFieldBegin('message', TType.STRING, 1) + oprot.writeString(self.message) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __str__(self): + return repr(self) + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class TxnAbortedException(TException): + """ + Attributes: + - message + """ + + thrift_spec = ( + None, # 0 + (1, TType.STRING, 'message', None, None, ), # 1 + ) + + def __init__(self, message=None,): + self.message = message + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.STRING: + self.message = iprot.readString(); + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('TxnAbortedException') + if self.message is not None: + oprot.writeFieldBegin('message', TType.STRING, 1) + oprot.writeString(self.message) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __str__(self): + return repr(self) + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class TxnOpenException(TException): + """ + Attributes: + - message + """ + + thrift_spec = ( + None, # 0 + (1, TType.STRING, 'message', None, None, ), # 1 + ) + + def __init__(self, message=None,): + self.message = message + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.STRING: + self.message = iprot.readString(); + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('TxnOpenException') + if self.message is not None: + oprot.writeFieldBegin('message', TType.STRING, 1) + oprot.writeString(self.message) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __str__(self): + return repr(self) + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class NoSuchLockException(TException): + """ + Attributes: + - message + """ + + thrift_spec = ( + None, # 0 + (1, TType.STRING, 'message', None, None, ), # 1 + ) + + def __init__(self, message=None,): + self.message = message + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.STRING: + self.message = iprot.readString(); + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('NoSuchLockException') + if self.message is not None: + oprot.writeFieldBegin('message', TType.STRING, 1) + oprot.writeString(self.message) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __str__(self): + return repr(self) + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) diff --git metastore/src/gen/thrift/gen-rb/hive_metastore_types.rb metastore/src/gen/thrift/gen-rb/hive_metastore_types.rb index fba024a..8fa27e6 100644 --- metastore/src/gen/thrift/gen-rb/hive_metastore_types.rb +++ metastore/src/gen/thrift/gen-rb/hive_metastore_types.rb @@ -32,6 +32,46 @@ module PartitionEventType VALID_VALUES = Set.new([LOAD_DONE]).freeze end +module TxnState + COMMITTED = 1 + ABORTED = 2 + OPEN = 3 + VALUE_MAP = {1 => "COMMITTED", 2 => "ABORTED", 3 => "OPEN"} + VALID_VALUES = Set.new([COMMITTED, ABORTED, OPEN]).freeze +end + +module LockLevel + DB = 1 + TABLE = 2 + PARTITION = 3 + VALUE_MAP = {1 => "DB", 2 => "TABLE", 3 => "PARTITION"} + VALID_VALUES = Set.new([DB, TABLE, PARTITION]).freeze +end + +module LockState + ACQUIRED = 1 + WAITING = 2 + ABORT = 3 + NOT_ACQUIRED = 4 + VALUE_MAP = {1 => "ACQUIRED", 2 => "WAITING", 3 => "ABORT", 4 => "NOT_ACQUIRED"} + VALID_VALUES = Set.new([ACQUIRED, WAITING, ABORT, NOT_ACQUIRED]).freeze +end + +module LockType + SHARED_READ = 1 + SHARED_WRITE = 2 + EXCLUSIVE = 3 + VALUE_MAP = {1 => "SHARED_READ", 2 => "SHARED_WRITE", 3 => "EXCLUSIVE"} + VALID_VALUES = Set.new([SHARED_READ, SHARED_WRITE, EXCLUSIVE]).freeze +end + +module CompactionType + MINOR = 1 + MAJOR = 2 + VALUE_MAP = {1 => "MINOR", 2 => "MAJOR"} + VALID_VALUES = Set.new([MINOR, MAJOR]).freeze +end + module FunctionType JAVA = 1 VALUE_MAP = {1 => "JAVA"} @@ -1085,6 +1125,465 @@ class Function ::Thrift::Struct.generate_accessors self end +class TxnInfo + include ::Thrift::Struct, ::Thrift::Struct_Union + ID = 1 + STATE = 2 + USER = 3 + HOSTNAME = 4 + + FIELDS = { + ID => {:type => ::Thrift::Types::I64, :name => 'id'}, + STATE => {:type => ::Thrift::Types::I32, :name => 'state', :enum_class => ::TxnState}, + USER => {:type => ::Thrift::Types::STRING, :name => 'user'}, + HOSTNAME => {:type => ::Thrift::Types::STRING, :name => 'hostname'} + } + + def struct_fields; FIELDS; end + + def validate + raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field id is unset!') unless @id + raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field state is unset!') unless @state + raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field user is unset!') unless @user + raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field hostname is unset!') unless @hostname + unless @state.nil? || ::TxnState::VALID_VALUES.include?(@state) + raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Invalid value of field state!') + end + end + + ::Thrift::Struct.generate_accessors self +end + +class GetOpenTxnsInfoResponse + include ::Thrift::Struct, ::Thrift::Struct_Union + TXN_HIGH_WATER_MARK = 1 + OPEN_TXNS = 2 + + FIELDS = { + TXN_HIGH_WATER_MARK => {:type => ::Thrift::Types::I64, :name => 'txn_high_water_mark'}, + OPEN_TXNS => {:type => ::Thrift::Types::LIST, :name => 'open_txns', :element => {:type => ::Thrift::Types::STRUCT, :class => ::TxnInfo}} + } + + def struct_fields; FIELDS; end + + def validate + raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field txn_high_water_mark is unset!') unless @txn_high_water_mark + raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field open_txns is unset!') unless @open_txns + end + + ::Thrift::Struct.generate_accessors self +end + +class GetOpenTxnsResponse + include ::Thrift::Struct, ::Thrift::Struct_Union + TXN_HIGH_WATER_MARK = 1 + OPEN_TXNS = 2 + + FIELDS = { + TXN_HIGH_WATER_MARK => {:type => ::Thrift::Types::I64, :name => 'txn_high_water_mark'}, + OPEN_TXNS => {:type => ::Thrift::Types::SET, :name => 'open_txns', :element => {:type => ::Thrift::Types::I64}} + } + + def struct_fields; FIELDS; end + + def validate + raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field txn_high_water_mark is unset!') unless @txn_high_water_mark + raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field open_txns is unset!') unless @open_txns + end + + ::Thrift::Struct.generate_accessors self +end + +class OpenTxnRequest + include ::Thrift::Struct, ::Thrift::Struct_Union + NUM_TXNS = 1 + USER = 2 + HOSTNAME = 3 + + FIELDS = { + NUM_TXNS => {:type => ::Thrift::Types::I32, :name => 'num_txns'}, + USER => {:type => ::Thrift::Types::STRING, :name => 'user'}, + HOSTNAME => {:type => ::Thrift::Types::STRING, :name => 'hostname'} + } + + def struct_fields; FIELDS; end + + def validate + raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field num_txns is unset!') unless @num_txns + raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field user is unset!') unless @user + raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field hostname is unset!') unless @hostname + end + + ::Thrift::Struct.generate_accessors self +end + +class OpenTxnsResponse + include ::Thrift::Struct, ::Thrift::Struct_Union + TXN_IDS = 1 + + FIELDS = { + TXN_IDS => {:type => ::Thrift::Types::LIST, :name => 'txn_ids', :element => {:type => ::Thrift::Types::I64}} + } + + def struct_fields; FIELDS; end + + def validate + raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field txn_ids is unset!') unless @txn_ids + end + + ::Thrift::Struct.generate_accessors self +end + +class AbortTxnRequest + include ::Thrift::Struct, ::Thrift::Struct_Union + TXNID = 1 + + FIELDS = { + TXNID => {:type => ::Thrift::Types::I64, :name => 'txnid'} + } + + def struct_fields; FIELDS; end + + def validate + raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field txnid is unset!') unless @txnid + end + + ::Thrift::Struct.generate_accessors self +end + +class CommitTxnRequest + include ::Thrift::Struct, ::Thrift::Struct_Union + TXNID = 1 + + FIELDS = { + TXNID => {:type => ::Thrift::Types::I64, :name => 'txnid'} + } + + def struct_fields; FIELDS; end + + def validate + raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field txnid is unset!') unless @txnid + end + + ::Thrift::Struct.generate_accessors self +end + +class LockComponent + include ::Thrift::Struct, ::Thrift::Struct_Union + TYPE = 1 + LEVEL = 2 + DBNAME = 3 + TABLENAME = 4 + PARTITIONNAME = 5 + + FIELDS = { + TYPE => {:type => ::Thrift::Types::I32, :name => 'type', :enum_class => ::LockType}, + LEVEL => {:type => ::Thrift::Types::I32, :name => 'level', :enum_class => ::LockLevel}, + DBNAME => {:type => ::Thrift::Types::STRING, :name => 'dbname'}, + TABLENAME => {:type => ::Thrift::Types::STRING, :name => 'tablename', :optional => true}, + PARTITIONNAME => {:type => ::Thrift::Types::STRING, :name => 'partitionname', :optional => true} + } + + def struct_fields; FIELDS; end + + def validate + raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field type is unset!') unless @type + raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field level is unset!') unless @level + raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field dbname is unset!') unless @dbname + unless @type.nil? || ::LockType::VALID_VALUES.include?(@type) + raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Invalid value of field type!') + end + unless @level.nil? || ::LockLevel::VALID_VALUES.include?(@level) + raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Invalid value of field level!') + end + end + + ::Thrift::Struct.generate_accessors self +end + +class LockRequest + include ::Thrift::Struct, ::Thrift::Struct_Union + COMPONENT = 1 + TXNID = 2 + USER = 3 + HOSTNAME = 4 + + FIELDS = { + COMPONENT => {:type => ::Thrift::Types::LIST, :name => 'component', :element => {:type => ::Thrift::Types::STRUCT, :class => ::LockComponent}}, + TXNID => {:type => ::Thrift::Types::I64, :name => 'txnid', :optional => true}, + USER => {:type => ::Thrift::Types::STRING, :name => 'user'}, + HOSTNAME => {:type => ::Thrift::Types::STRING, :name => 'hostname'} + } + + def struct_fields; FIELDS; end + + def validate + raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field component is unset!') unless @component + raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field user is unset!') unless @user + raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field hostname is unset!') unless @hostname + end + + ::Thrift::Struct.generate_accessors self +end + +class LockResponse + include ::Thrift::Struct, ::Thrift::Struct_Union + LOCKID = 1 + STATE = 2 + + FIELDS = { + LOCKID => {:type => ::Thrift::Types::I64, :name => 'lockid'}, + STATE => {:type => ::Thrift::Types::I32, :name => 'state', :enum_class => ::LockState} + } + + def struct_fields; FIELDS; end + + def validate + raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field lockid is unset!') unless @lockid + raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field state is unset!') unless @state + unless @state.nil? || ::LockState::VALID_VALUES.include?(@state) + raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Invalid value of field state!') + end + end + + ::Thrift::Struct.generate_accessors self +end + +class CheckLockRequest + include ::Thrift::Struct, ::Thrift::Struct_Union + LOCKID = 1 + + FIELDS = { + LOCKID => {:type => ::Thrift::Types::I64, :name => 'lockid'} + } + + def struct_fields; FIELDS; end + + def validate + raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field lockid is unset!') unless @lockid + end + + ::Thrift::Struct.generate_accessors self +end + +class UnlockRequest + include ::Thrift::Struct, ::Thrift::Struct_Union + LOCKID = 1 + + FIELDS = { + LOCKID => {:type => ::Thrift::Types::I64, :name => 'lockid'} + } + + def struct_fields; FIELDS; end + + def validate + raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field lockid is unset!') unless @lockid + end + + ::Thrift::Struct.generate_accessors self +end + +class ShowLocksRequest + include ::Thrift::Struct, ::Thrift::Struct_Union + + FIELDS = { + + } + + def struct_fields; FIELDS; end + + def validate + end + + ::Thrift::Struct.generate_accessors self +end + +class ShowLocksResponseElement + include ::Thrift::Struct, ::Thrift::Struct_Union + LOCKID = 1 + DBNAME = 2 + TABLENAME = 3 + PARTNAME = 4 + STATE = 5 + TYPE = 6 + TXNID = 7 + LASTHEARTBEAT = 8 + ACQUIREDAT = 9 + USER = 10 + HOSTNAME = 11 + + FIELDS = { + LOCKID => {:type => ::Thrift::Types::I64, :name => 'lockid'}, + DBNAME => {:type => ::Thrift::Types::STRING, :name => 'dbname'}, + TABLENAME => {:type => ::Thrift::Types::STRING, :name => 'tablename', :optional => true}, + PARTNAME => {:type => ::Thrift::Types::STRING, :name => 'partname', :optional => true}, + STATE => {:type => ::Thrift::Types::I32, :name => 'state', :enum_class => ::LockState}, + TYPE => {:type => ::Thrift::Types::I32, :name => 'type', :enum_class => ::LockType}, + TXNID => {:type => ::Thrift::Types::I64, :name => 'txnid', :optional => true}, + LASTHEARTBEAT => {:type => ::Thrift::Types::I64, :name => 'lastheartbeat'}, + ACQUIREDAT => {:type => ::Thrift::Types::I64, :name => 'acquiredat', :optional => true}, + USER => {:type => ::Thrift::Types::STRING, :name => 'user'}, + HOSTNAME => {:type => ::Thrift::Types::STRING, :name => 'hostname'} + } + + def struct_fields; FIELDS; end + + def validate + raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field lockid is unset!') unless @lockid + raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field dbname is unset!') unless @dbname + raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field state is unset!') unless @state + raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field type is unset!') unless @type + raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field lastheartbeat is unset!') unless @lastheartbeat + raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field user is unset!') unless @user + raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field hostname is unset!') unless @hostname + unless @state.nil? || ::LockState::VALID_VALUES.include?(@state) + raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Invalid value of field state!') + end + unless @type.nil? || ::LockType::VALID_VALUES.include?(@type) + raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Invalid value of field type!') + end + end + + ::Thrift::Struct.generate_accessors self +end + +class ShowLocksResponse + include ::Thrift::Struct, ::Thrift::Struct_Union + LOCKS = 1 + + FIELDS = { + LOCKS => {:type => ::Thrift::Types::LIST, :name => 'locks', :element => {:type => ::Thrift::Types::STRUCT, :class => ::ShowLocksResponseElement}} + } + + def struct_fields; FIELDS; end + + def validate + end + + ::Thrift::Struct.generate_accessors self +end + +class HeartbeatRequest + include ::Thrift::Struct, ::Thrift::Struct_Union + LOCKID = 1 + TXNID = 2 + + FIELDS = { + LOCKID => {:type => ::Thrift::Types::I64, :name => 'lockid', :optional => true}, + TXNID => {:type => ::Thrift::Types::I64, :name => 'txnid', :optional => true} + } + + def struct_fields; FIELDS; end + + def validate + end + + ::Thrift::Struct.generate_accessors self +end + +class CompactionRequest + include ::Thrift::Struct, ::Thrift::Struct_Union + DBNAME = 1 + TABLENAME = 2 + PARTITIONNAME = 3 + TYPE = 4 + RUNAS = 5 + + FIELDS = { + DBNAME => {:type => ::Thrift::Types::STRING, :name => 'dbname'}, + TABLENAME => {:type => ::Thrift::Types::STRING, :name => 'tablename'}, + PARTITIONNAME => {:type => ::Thrift::Types::STRING, :name => 'partitionname', :optional => true}, + TYPE => {:type => ::Thrift::Types::I32, :name => 'type', :enum_class => ::CompactionType}, + RUNAS => {:type => ::Thrift::Types::STRING, :name => 'runas', :optional => true} + } + + def struct_fields; FIELDS; end + + def validate + raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field dbname is unset!') unless @dbname + raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field tablename is unset!') unless @tablename + raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field type is unset!') unless @type + unless @type.nil? || ::CompactionType::VALID_VALUES.include?(@type) + raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Invalid value of field type!') + end + end + + ::Thrift::Struct.generate_accessors self +end + +class ShowCompactRequest + include ::Thrift::Struct, ::Thrift::Struct_Union + + FIELDS = { + + } + + def struct_fields; FIELDS; end + + def validate + end + + ::Thrift::Struct.generate_accessors self +end + +class ShowCompactResponseElement + include ::Thrift::Struct, ::Thrift::Struct_Union + DBNAME = 1 + TABLENAME = 2 + PARTITIONNAME = 3 + TYPE = 4 + STATE = 5 + WORKERID = 6 + START = 7 + RUNAS = 8 + + FIELDS = { + DBNAME => {:type => ::Thrift::Types::STRING, :name => 'dbname'}, + TABLENAME => {:type => ::Thrift::Types::STRING, :name => 'tablename'}, + PARTITIONNAME => {:type => ::Thrift::Types::STRING, :name => 'partitionname'}, + TYPE => {:type => ::Thrift::Types::I32, :name => 'type', :enum_class => ::CompactionType}, + STATE => {:type => ::Thrift::Types::STRING, :name => 'state'}, + WORKERID => {:type => ::Thrift::Types::STRING, :name => 'workerid'}, + START => {:type => ::Thrift::Types::I64, :name => 'start'}, + RUNAS => {:type => ::Thrift::Types::STRING, :name => 'runAs'} + } + + def struct_fields; FIELDS; end + + def validate + raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field dbname is unset!') unless @dbname + raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field tablename is unset!') unless @tablename + raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field partitionname is unset!') unless @partitionname + raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field type is unset!') unless @type + raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field state is unset!') unless @state + raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field workerid is unset!') unless @workerid + raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field start is unset!') unless @start + raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field runAs is unset!') unless @runAs + unless @type.nil? || ::CompactionType::VALID_VALUES.include?(@type) + raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Invalid value of field type!') + end + end + + ::Thrift::Struct.generate_accessors self +end + +class ShowCompactResponse + include ::Thrift::Struct, ::Thrift::Struct_Union + COMPACTS = 1 + + FIELDS = { + COMPACTS => {:type => ::Thrift::Types::LIST, :name => 'compacts', :element => {:type => ::Thrift::Types::STRUCT, :class => ::ShowCompactResponseElement}} + } + + def struct_fields; FIELDS; end + + def validate + raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field compacts is unset!') unless @compacts + end + + ::Thrift::Struct.generate_accessors self +end + class MetaException < ::Thrift::Exception include ::Thrift::Struct, ::Thrift::Struct_Union def initialize(message=nil) @@ -1337,3 +1836,87 @@ class InvalidInputException < ::Thrift::Exception ::Thrift::Struct.generate_accessors self end +class NoSuchTxnException < ::Thrift::Exception + include ::Thrift::Struct, ::Thrift::Struct_Union + def initialize(message=nil) + super() + self.message = message + end + + MESSAGE = 1 + + FIELDS = { + MESSAGE => {:type => ::Thrift::Types::STRING, :name => 'message'} + } + + def struct_fields; FIELDS; end + + def validate + end + + ::Thrift::Struct.generate_accessors self +end + +class TxnAbortedException < ::Thrift::Exception + include ::Thrift::Struct, ::Thrift::Struct_Union + def initialize(message=nil) + super() + self.message = message + end + + MESSAGE = 1 + + FIELDS = { + MESSAGE => {:type => ::Thrift::Types::STRING, :name => 'message'} + } + + def struct_fields; FIELDS; end + + def validate + end + + ::Thrift::Struct.generate_accessors self +end + +class TxnOpenException < ::Thrift::Exception + include ::Thrift::Struct, ::Thrift::Struct_Union + def initialize(message=nil) + super() + self.message = message + end + + MESSAGE = 1 + + FIELDS = { + MESSAGE => {:type => ::Thrift::Types::STRING, :name => 'message'} + } + + def struct_fields; FIELDS; end + + def validate + end + + ::Thrift::Struct.generate_accessors self +end + +class NoSuchLockException < ::Thrift::Exception + include ::Thrift::Struct, ::Thrift::Struct_Union + def initialize(message=nil) + super() + self.message = message + end + + MESSAGE = 1 + + FIELDS = { + MESSAGE => {:type => ::Thrift::Types::STRING, :name => 'message'} + } + + def struct_fields; FIELDS; end + + def validate + end + + ::Thrift::Struct.generate_accessors self +end + diff --git metastore/src/gen/thrift/gen-rb/thrift_hive_metastore.rb metastore/src/gen/thrift/gen-rb/thrift_hive_metastore.rb index f7fa7f3..ff46007 100644 --- metastore/src/gen/thrift/gen-rb/thrift_hive_metastore.rb +++ metastore/src/gen/thrift/gen-rb/thrift_hive_metastore.rb @@ -1575,6 +1575,194 @@ module ThriftHiveMetastore return end + def get_open_txns() + send_get_open_txns() + return recv_get_open_txns() + end + + def send_get_open_txns() + send_message('get_open_txns', Get_open_txns_args) + end + + def recv_get_open_txns() + result = receive_message(Get_open_txns_result) + return result.success unless result.success.nil? + raise ::Thrift::ApplicationException.new(::Thrift::ApplicationException::MISSING_RESULT, 'get_open_txns failed: unknown result') + end + + def get_open_txns_info() + send_get_open_txns_info() + return recv_get_open_txns_info() + end + + def send_get_open_txns_info() + send_message('get_open_txns_info', Get_open_txns_info_args) + end + + def recv_get_open_txns_info() + result = receive_message(Get_open_txns_info_result) + return result.success unless result.success.nil? + raise ::Thrift::ApplicationException.new(::Thrift::ApplicationException::MISSING_RESULT, 'get_open_txns_info failed: unknown result') + end + + def open_txns(rqst) + send_open_txns(rqst) + return recv_open_txns() + end + + def send_open_txns(rqst) + send_message('open_txns', Open_txns_args, :rqst => rqst) + end + + def recv_open_txns() + result = receive_message(Open_txns_result) + return result.success unless result.success.nil? + raise ::Thrift::ApplicationException.new(::Thrift::ApplicationException::MISSING_RESULT, 'open_txns failed: unknown result') + end + + def abort_txn(rqst) + send_abort_txn(rqst) + recv_abort_txn() + end + + def send_abort_txn(rqst) + send_message('abort_txn', Abort_txn_args, :rqst => rqst) + end + + def recv_abort_txn() + result = receive_message(Abort_txn_result) + raise result.o1 unless result.o1.nil? + return + end + + def commit_txn(rqst) + send_commit_txn(rqst) + recv_commit_txn() + end + + def send_commit_txn(rqst) + send_message('commit_txn', Commit_txn_args, :rqst => rqst) + end + + def recv_commit_txn() + result = receive_message(Commit_txn_result) + raise result.o1 unless result.o1.nil? + raise result.o2 unless result.o2.nil? + return + end + + def lock(rqst) + send_lock(rqst) + return recv_lock() + end + + def send_lock(rqst) + send_message('lock', Lock_args, :rqst => rqst) + end + + def recv_lock() + result = receive_message(Lock_result) + return result.success unless result.success.nil? + raise result.o1 unless result.o1.nil? + raise result.o2 unless result.o2.nil? + raise ::Thrift::ApplicationException.new(::Thrift::ApplicationException::MISSING_RESULT, 'lock failed: unknown result') + end + + def check_lock(rqst) + send_check_lock(rqst) + return recv_check_lock() + end + + def send_check_lock(rqst) + send_message('check_lock', Check_lock_args, :rqst => rqst) + end + + def recv_check_lock() + result = receive_message(Check_lock_result) + return result.success unless result.success.nil? + raise result.o1 unless result.o1.nil? + raise result.o2 unless result.o2.nil? + raise result.o3 unless result.o3.nil? + raise ::Thrift::ApplicationException.new(::Thrift::ApplicationException::MISSING_RESULT, 'check_lock failed: unknown result') + end + + def unlock(rqst) + send_unlock(rqst) + recv_unlock() + end + + def send_unlock(rqst) + send_message('unlock', Unlock_args, :rqst => rqst) + end + + def recv_unlock() + result = receive_message(Unlock_result) + raise result.o1 unless result.o1.nil? + raise result.o2 unless result.o2.nil? + return + end + + def show_locks(rqst) + send_show_locks(rqst) + return recv_show_locks() + end + + def send_show_locks(rqst) + send_message('show_locks', Show_locks_args, :rqst => rqst) + end + + def recv_show_locks() + result = receive_message(Show_locks_result) + return result.success unless result.success.nil? + raise ::Thrift::ApplicationException.new(::Thrift::ApplicationException::MISSING_RESULT, 'show_locks failed: unknown result') + end + + def heartbeat(ids) + send_heartbeat(ids) + recv_heartbeat() + end + + def send_heartbeat(ids) + send_message('heartbeat', Heartbeat_args, :ids => ids) + end + + def recv_heartbeat() + result = receive_message(Heartbeat_result) + raise result.o1 unless result.o1.nil? + raise result.o2 unless result.o2.nil? + raise result.o3 unless result.o3.nil? + return + end + + def compact(rqst) + send_compact(rqst) + recv_compact() + end + + def send_compact(rqst) + send_message('compact', Compact_args, :rqst => rqst) + end + + def recv_compact() + result = receive_message(Compact_result) + return + end + + def show_compact(rqst) + send_show_compact(rqst) + return recv_show_compact() + end + + def send_show_compact(rqst) + send_message('show_compact', Show_compact_args, :rqst => rqst) + end + + def recv_show_compact() + result = receive_message(Show_compact_result) + return result.success unless result.success.nil? + raise ::Thrift::ApplicationException.new(::Thrift::ApplicationException::MISSING_RESULT, 'show_compact failed: unknown result') + end + end class Processor < ::FacebookService::Processor @@ -2812,6 +3000,128 @@ module ThriftHiveMetastore write_result(result, oprot, 'cancel_delegation_token', seqid) end + def process_get_open_txns(seqid, iprot, oprot) + args = read_args(iprot, Get_open_txns_args) + result = Get_open_txns_result.new() + result.success = @handler.get_open_txns() + write_result(result, oprot, 'get_open_txns', seqid) + end + + def process_get_open_txns_info(seqid, iprot, oprot) + args = read_args(iprot, Get_open_txns_info_args) + result = Get_open_txns_info_result.new() + result.success = @handler.get_open_txns_info() + write_result(result, oprot, 'get_open_txns_info', seqid) + end + + def process_open_txns(seqid, iprot, oprot) + args = read_args(iprot, Open_txns_args) + result = Open_txns_result.new() + result.success = @handler.open_txns(args.rqst) + write_result(result, oprot, 'open_txns', seqid) + end + + def process_abort_txn(seqid, iprot, oprot) + args = read_args(iprot, Abort_txn_args) + result = Abort_txn_result.new() + begin + @handler.abort_txn(args.rqst) + rescue ::NoSuchTxnException => o1 + result.o1 = o1 + end + write_result(result, oprot, 'abort_txn', seqid) + end + + def process_commit_txn(seqid, iprot, oprot) + args = read_args(iprot, Commit_txn_args) + result = Commit_txn_result.new() + begin + @handler.commit_txn(args.rqst) + rescue ::NoSuchTxnException => o1 + result.o1 = o1 + rescue ::TxnAbortedException => o2 + result.o2 = o2 + end + write_result(result, oprot, 'commit_txn', seqid) + end + + def process_lock(seqid, iprot, oprot) + args = read_args(iprot, Lock_args) + result = Lock_result.new() + begin + result.success = @handler.lock(args.rqst) + rescue ::NoSuchTxnException => o1 + result.o1 = o1 + rescue ::TxnAbortedException => o2 + result.o2 = o2 + end + write_result(result, oprot, 'lock', seqid) + end + + def process_check_lock(seqid, iprot, oprot) + args = read_args(iprot, Check_lock_args) + result = Check_lock_result.new() + begin + result.success = @handler.check_lock(args.rqst) + rescue ::NoSuchTxnException => o1 + result.o1 = o1 + rescue ::TxnAbortedException => o2 + result.o2 = o2 + rescue ::NoSuchLockException => o3 + result.o3 = o3 + end + write_result(result, oprot, 'check_lock', seqid) + end + + def process_unlock(seqid, iprot, oprot) + args = read_args(iprot, Unlock_args) + result = Unlock_result.new() + begin + @handler.unlock(args.rqst) + rescue ::NoSuchLockException => o1 + result.o1 = o1 + rescue ::TxnOpenException => o2 + result.o2 = o2 + end + write_result(result, oprot, 'unlock', seqid) + end + + def process_show_locks(seqid, iprot, oprot) + args = read_args(iprot, Show_locks_args) + result = Show_locks_result.new() + result.success = @handler.show_locks(args.rqst) + write_result(result, oprot, 'show_locks', seqid) + end + + def process_heartbeat(seqid, iprot, oprot) + args = read_args(iprot, Heartbeat_args) + result = Heartbeat_result.new() + begin + @handler.heartbeat(args.ids) + rescue ::NoSuchLockException => o1 + result.o1 = o1 + rescue ::NoSuchTxnException => o2 + result.o2 = o2 + rescue ::TxnAbortedException => o3 + result.o3 = o3 + end + write_result(result, oprot, 'heartbeat', seqid) + end + + def process_compact(seqid, iprot, oprot) + args = read_args(iprot, Compact_args) + result = Compact_result.new() + @handler.compact(args.rqst) + write_result(result, oprot, 'compact', seqid) + end + + def process_show_compact(seqid, iprot, oprot) + args = read_args(iprot, Show_compact_args) + result = Show_compact_result.new() + result.success = @handler.show_compact(args.rqst) + write_result(result, oprot, 'show_compact', seqid) + end + end # HELPER FUNCTIONS AND STRUCTURES @@ -6429,5 +6739,404 @@ module ThriftHiveMetastore ::Thrift::Struct.generate_accessors self end + class Get_open_txns_args + include ::Thrift::Struct, ::Thrift::Struct_Union + + FIELDS = { + + } + + def struct_fields; FIELDS; end + + def validate + end + + ::Thrift::Struct.generate_accessors self + end + + class Get_open_txns_result + include ::Thrift::Struct, ::Thrift::Struct_Union + SUCCESS = 0 + + FIELDS = { + SUCCESS => {:type => ::Thrift::Types::STRUCT, :name => 'success', :class => ::GetOpenTxnsResponse} + } + + def struct_fields; FIELDS; end + + def validate + end + + ::Thrift::Struct.generate_accessors self + end + + class Get_open_txns_info_args + include ::Thrift::Struct, ::Thrift::Struct_Union + + FIELDS = { + + } + + def struct_fields; FIELDS; end + + def validate + end + + ::Thrift::Struct.generate_accessors self + end + + class Get_open_txns_info_result + include ::Thrift::Struct, ::Thrift::Struct_Union + SUCCESS = 0 + + FIELDS = { + SUCCESS => {:type => ::Thrift::Types::STRUCT, :name => 'success', :class => ::GetOpenTxnsInfoResponse} + } + + def struct_fields; FIELDS; end + + def validate + end + + ::Thrift::Struct.generate_accessors self + end + + class Open_txns_args + include ::Thrift::Struct, ::Thrift::Struct_Union + RQST = 1 + + FIELDS = { + RQST => {:type => ::Thrift::Types::STRUCT, :name => 'rqst', :class => ::OpenTxnRequest} + } + + def struct_fields; FIELDS; end + + def validate + end + + ::Thrift::Struct.generate_accessors self + end + + class Open_txns_result + include ::Thrift::Struct, ::Thrift::Struct_Union + SUCCESS = 0 + + FIELDS = { + SUCCESS => {:type => ::Thrift::Types::STRUCT, :name => 'success', :class => ::OpenTxnsResponse} + } + + def struct_fields; FIELDS; end + + def validate + end + + ::Thrift::Struct.generate_accessors self + end + + class Abort_txn_args + include ::Thrift::Struct, ::Thrift::Struct_Union + RQST = 1 + + FIELDS = { + RQST => {:type => ::Thrift::Types::STRUCT, :name => 'rqst', :class => ::AbortTxnRequest} + } + + def struct_fields; FIELDS; end + + def validate + end + + ::Thrift::Struct.generate_accessors self + end + + class Abort_txn_result + include ::Thrift::Struct, ::Thrift::Struct_Union + O1 = 1 + + FIELDS = { + O1 => {:type => ::Thrift::Types::STRUCT, :name => 'o1', :class => ::NoSuchTxnException} + } + + def struct_fields; FIELDS; end + + def validate + end + + ::Thrift::Struct.generate_accessors self + end + + class Commit_txn_args + include ::Thrift::Struct, ::Thrift::Struct_Union + RQST = 1 + + FIELDS = { + RQST => {:type => ::Thrift::Types::STRUCT, :name => 'rqst', :class => ::CommitTxnRequest} + } + + def struct_fields; FIELDS; end + + def validate + end + + ::Thrift::Struct.generate_accessors self + end + + class Commit_txn_result + include ::Thrift::Struct, ::Thrift::Struct_Union + O1 = 1 + O2 = 2 + + FIELDS = { + O1 => {:type => ::Thrift::Types::STRUCT, :name => 'o1', :class => ::NoSuchTxnException}, + O2 => {:type => ::Thrift::Types::STRUCT, :name => 'o2', :class => ::TxnAbortedException} + } + + def struct_fields; FIELDS; end + + def validate + end + + ::Thrift::Struct.generate_accessors self + end + + class Lock_args + include ::Thrift::Struct, ::Thrift::Struct_Union + RQST = 1 + + FIELDS = { + RQST => {:type => ::Thrift::Types::STRUCT, :name => 'rqst', :class => ::LockRequest} + } + + def struct_fields; FIELDS; end + + def validate + end + + ::Thrift::Struct.generate_accessors self + end + + class Lock_result + include ::Thrift::Struct, ::Thrift::Struct_Union + SUCCESS = 0 + O1 = 1 + O2 = 2 + + FIELDS = { + SUCCESS => {:type => ::Thrift::Types::STRUCT, :name => 'success', :class => ::LockResponse}, + O1 => {:type => ::Thrift::Types::STRUCT, :name => 'o1', :class => ::NoSuchTxnException}, + O2 => {:type => ::Thrift::Types::STRUCT, :name => 'o2', :class => ::TxnAbortedException} + } + + def struct_fields; FIELDS; end + + def validate + end + + ::Thrift::Struct.generate_accessors self + end + + class Check_lock_args + include ::Thrift::Struct, ::Thrift::Struct_Union + RQST = 1 + + FIELDS = { + RQST => {:type => ::Thrift::Types::STRUCT, :name => 'rqst', :class => ::CheckLockRequest} + } + + def struct_fields; FIELDS; end + + def validate + end + + ::Thrift::Struct.generate_accessors self + end + + class Check_lock_result + include ::Thrift::Struct, ::Thrift::Struct_Union + SUCCESS = 0 + O1 = 1 + O2 = 2 + O3 = 3 + + FIELDS = { + SUCCESS => {:type => ::Thrift::Types::STRUCT, :name => 'success', :class => ::LockResponse}, + O1 => {:type => ::Thrift::Types::STRUCT, :name => 'o1', :class => ::NoSuchTxnException}, + O2 => {:type => ::Thrift::Types::STRUCT, :name => 'o2', :class => ::TxnAbortedException}, + O3 => {:type => ::Thrift::Types::STRUCT, :name => 'o3', :class => ::NoSuchLockException} + } + + def struct_fields; FIELDS; end + + def validate + end + + ::Thrift::Struct.generate_accessors self + end + + class Unlock_args + include ::Thrift::Struct, ::Thrift::Struct_Union + RQST = 1 + + FIELDS = { + RQST => {:type => ::Thrift::Types::STRUCT, :name => 'rqst', :class => ::UnlockRequest} + } + + def struct_fields; FIELDS; end + + def validate + end + + ::Thrift::Struct.generate_accessors self + end + + class Unlock_result + include ::Thrift::Struct, ::Thrift::Struct_Union + O1 = 1 + O2 = 2 + + FIELDS = { + O1 => {:type => ::Thrift::Types::STRUCT, :name => 'o1', :class => ::NoSuchLockException}, + O2 => {:type => ::Thrift::Types::STRUCT, :name => 'o2', :class => ::TxnOpenException} + } + + def struct_fields; FIELDS; end + + def validate + end + + ::Thrift::Struct.generate_accessors self + end + + class Show_locks_args + include ::Thrift::Struct, ::Thrift::Struct_Union + RQST = 1 + + FIELDS = { + RQST => {:type => ::Thrift::Types::STRUCT, :name => 'rqst', :class => ::ShowLocksRequest} + } + + def struct_fields; FIELDS; end + + def validate + end + + ::Thrift::Struct.generate_accessors self + end + + class Show_locks_result + include ::Thrift::Struct, ::Thrift::Struct_Union + SUCCESS = 0 + + FIELDS = { + SUCCESS => {:type => ::Thrift::Types::STRUCT, :name => 'success', :class => ::ShowLocksResponse} + } + + def struct_fields; FIELDS; end + + def validate + end + + ::Thrift::Struct.generate_accessors self + end + + class Heartbeat_args + include ::Thrift::Struct, ::Thrift::Struct_Union + IDS = 1 + + FIELDS = { + IDS => {:type => ::Thrift::Types::STRUCT, :name => 'ids', :class => ::HeartbeatRequest} + } + + def struct_fields; FIELDS; end + + def validate + end + + ::Thrift::Struct.generate_accessors self + end + + class Heartbeat_result + include ::Thrift::Struct, ::Thrift::Struct_Union + O1 = 1 + O2 = 2 + O3 = 3 + + FIELDS = { + O1 => {:type => ::Thrift::Types::STRUCT, :name => 'o1', :class => ::NoSuchLockException}, + O2 => {:type => ::Thrift::Types::STRUCT, :name => 'o2', :class => ::NoSuchTxnException}, + O3 => {:type => ::Thrift::Types::STRUCT, :name => 'o3', :class => ::TxnAbortedException} + } + + def struct_fields; FIELDS; end + + def validate + end + + ::Thrift::Struct.generate_accessors self + end + + class Compact_args + include ::Thrift::Struct, ::Thrift::Struct_Union + RQST = 1 + + FIELDS = { + RQST => {:type => ::Thrift::Types::STRUCT, :name => 'rqst', :class => ::CompactionRequest} + } + + def struct_fields; FIELDS; end + + def validate + end + + ::Thrift::Struct.generate_accessors self + end + + class Compact_result + include ::Thrift::Struct, ::Thrift::Struct_Union + + FIELDS = { + + } + + def struct_fields; FIELDS; end + + def validate + end + + ::Thrift::Struct.generate_accessors self + end + + class Show_compact_args + include ::Thrift::Struct, ::Thrift::Struct_Union + RQST = 1 + + FIELDS = { + RQST => {:type => ::Thrift::Types::STRUCT, :name => 'rqst', :class => ::ShowCompactRequest} + } + + def struct_fields; FIELDS; end + + def validate + end + + ::Thrift::Struct.generate_accessors self + end + + class Show_compact_result + include ::Thrift::Struct, ::Thrift::Struct_Union + SUCCESS = 0 + + FIELDS = { + SUCCESS => {:type => ::Thrift::Types::STRUCT, :name => 'success', :class => ::ShowCompactResponse} + } + + def struct_fields; FIELDS; end + + def validate + end + + ::Thrift::Struct.generate_accessors self + end + end diff --git metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java index d9c9213..ba51b96 100644 --- metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java +++ metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java @@ -18,6 +18,8 @@ package org.apache.hadoop.hive.metastore; +import com.facebook.fb303.FacebookBase; +import com.facebook.fb303.fb_status; import static org.apache.commons.lang.StringUtils.join; import static org.apache.hadoop.hive.metastore.MetaStoreUtils.DEFAULT_DATABASE_COMMENT; import static org.apache.hadoop.hive.metastore.MetaStoreUtils.DEFAULT_DATABASE_NAME; @@ -60,12 +62,16 @@ import org.apache.hadoop.hive.common.metrics.Metrics; import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.conf.HiveConf.ConfVars; +import org.apache.hadoop.hive.metastore.api.AbortTxnRequest; import org.apache.hadoop.hive.metastore.api.AddPartitionsRequest; import org.apache.hadoop.hive.metastore.api.AddPartitionsResult; import org.apache.hadoop.hive.metastore.api.AlreadyExistsException; +import org.apache.hadoop.hive.metastore.api.CheckLockRequest; import org.apache.hadoop.hive.metastore.api.ColumnStatistics; import org.apache.hadoop.hive.metastore.api.ColumnStatisticsDesc; import org.apache.hadoop.hive.metastore.api.ColumnStatisticsObj; +import org.apache.hadoop.hive.metastore.api.CommitTxnRequest; +import org.apache.hadoop.hive.metastore.api.CompactionRequest; import org.apache.hadoop.hive.metastore.api.ConfigValSecurityException; import org.apache.hadoop.hive.metastore.api.Database; import org.apache.hadoop.hive.metastore.api.DropPartitionsExpr; @@ -74,6 +80,9 @@ import org.apache.hadoop.hive.metastore.api.EnvironmentContext; import org.apache.hadoop.hive.metastore.api.FieldSchema; import org.apache.hadoop.hive.metastore.api.Function; +import org.apache.hadoop.hive.metastore.api.GetOpenTxnsInfoResponse; +import org.apache.hadoop.hive.metastore.api.GetOpenTxnsResponse; +import org.apache.hadoop.hive.metastore.api.HeartbeatRequest; import org.apache.hadoop.hive.metastore.api.HiveObjectPrivilege; import org.apache.hadoop.hive.metastore.api.HiveObjectRef; import org.apache.hadoop.hive.metastore.api.HiveObjectType; @@ -83,8 +92,14 @@ import org.apache.hadoop.hive.metastore.api.InvalidObjectException; import org.apache.hadoop.hive.metastore.api.InvalidOperationException; import org.apache.hadoop.hive.metastore.api.InvalidPartitionException; +import org.apache.hadoop.hive.metastore.api.LockRequest; +import org.apache.hadoop.hive.metastore.api.LockResponse; import org.apache.hadoop.hive.metastore.api.MetaException; +import org.apache.hadoop.hive.metastore.api.NoSuchLockException; import org.apache.hadoop.hive.metastore.api.NoSuchObjectException; +import org.apache.hadoop.hive.metastore.api.NoSuchTxnException; +import org.apache.hadoop.hive.metastore.api.OpenTxnRequest; +import org.apache.hadoop.hive.metastore.api.OpenTxnsResponse; import org.apache.hadoop.hive.metastore.api.Partition; import org.apache.hadoop.hive.metastore.api.PartitionEventType; import org.apache.hadoop.hive.metastore.api.PartitionsByExprRequest; @@ -97,16 +112,24 @@ import org.apache.hadoop.hive.metastore.api.PrivilegeGrantInfo; import org.apache.hadoop.hive.metastore.api.RequestPartsSpec; import org.apache.hadoop.hive.metastore.api.Role; +import org.apache.hadoop.hive.metastore.api.ShowCompactRequest; +import org.apache.hadoop.hive.metastore.api.ShowCompactResponse; +import org.apache.hadoop.hive.metastore.api.ShowLocksRequest; +import org.apache.hadoop.hive.metastore.api.ShowLocksResponse; import org.apache.hadoop.hive.metastore.api.SkewedInfo; import org.apache.hadoop.hive.metastore.api.Table; import org.apache.hadoop.hive.metastore.api.TableStatsRequest; import org.apache.hadoop.hive.metastore.api.TableStatsResult; -import org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore; +import org.apache.hadoop.hive.metastore.api.TxnAbortedException; +import org.apache.hadoop.hive.metastore.api.TxnOpenException; import org.apache.hadoop.hive.metastore.api.Type; import org.apache.hadoop.hive.metastore.api.UnknownDBException; import org.apache.hadoop.hive.metastore.api.UnknownPartitionException; import org.apache.hadoop.hive.metastore.api.UnknownTableException; +import org.apache.hadoop.hive.metastore.api.UnlockRequest; import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants; +import org.apache.hadoop.hive.metastore.txn.TxnHandler; +import org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore; import org.apache.hadoop.hive.metastore.events.AddPartitionEvent; import org.apache.hadoop.hive.metastore.events.AlterPartitionEvent; import org.apache.hadoop.hive.metastore.events.AlterTableEvent; @@ -149,14 +172,8 @@ import org.apache.thrift.protocol.TBinaryProtocol; import org.apache.thrift.server.TServer; import org.apache.thrift.server.TThreadPoolServer; -import org.apache.thrift.transport.TFramedTransport; -import org.apache.thrift.transport.TServerSocket; -import org.apache.thrift.transport.TServerTransport; -import org.apache.thrift.transport.TTransport; -import org.apache.thrift.transport.TTransportFactory; +import org.apache.thrift.transport.*; -import com.facebook.fb303.FacebookBase; -import com.facebook.fb303.fb_status; import com.google.common.base.Splitter; import com.google.common.collect.Lists; @@ -218,6 +235,13 @@ protected synchronized RawStore initialValue() { } }; + private final ThreadLocal threadLocalTxn = new ThreadLocal() { + @Override + protected synchronized TxnHandler initialValue() { + return null; + } + }; + // Thread local configuration is needed as many threads could make changes // to the conf using the connection hook private final ThreadLocal threadLocalConf = @@ -443,6 +467,15 @@ public RawStore getMS() throws MetaException { return ms; } + private TxnHandler getTxnHandler() { + TxnHandler txn = threadLocalTxn.get(); + if (txn == null) { + txn = new TxnHandler(hiveConf); + threadLocalTxn.set(txn); + } + return txn; + } + private RawStore newRawStore() throws MetaException { LOG.info(addPrefix("Opening raw store with implemenation class:" + rawStoreClassName)); @@ -4706,6 +4739,121 @@ public Function get_function(String dbName, String funcName) return func; } + + // Transaction and locking methods + @Override + public GetOpenTxnsResponse get_open_txns() throws TException { + try { + return getTxnHandler().getOpenTxns(); + } catch (MetaException e) { + throw new TException(e); + } + } + + // Transaction and locking methods + @Override + public GetOpenTxnsInfoResponse get_open_txns_info() throws TException { + try { + return getTxnHandler().getOpenTxnsInfo(); + } catch (MetaException e) { + throw new TException(e); + } + } + + @Override + public OpenTxnsResponse open_txns(OpenTxnRequest rqst) throws TException { + try { + return getTxnHandler().openTxns(rqst); + } catch (MetaException e) { + throw new TException(e); + } + } + + @Override + public void abort_txn(AbortTxnRequest rqst) throws NoSuchTxnException, TException { + try { + getTxnHandler().abortTxn(rqst); + } catch (MetaException e) { + throw new TException(e); + } + } + + @Override + public void commit_txn(CommitTxnRequest rqst) + throws NoSuchTxnException, TxnAbortedException, TException { + try { + getTxnHandler().commitTxn(rqst); + } catch (MetaException e) { + throw new TException(e); + } + } + + @Override + public LockResponse lock(LockRequest rqst) + throws NoSuchTxnException, TxnAbortedException, TException { + try { + return getTxnHandler().lock(rqst); + } catch (MetaException e) { + throw new TException(e); + } + } + + @Override + public LockResponse check_lock(CheckLockRequest rqst) + throws NoSuchTxnException, TxnAbortedException, NoSuchLockException, TException { + try { + return getTxnHandler().checkLock(rqst); + } catch (MetaException e) { + throw new TException(e); + } + } + + @Override + public void unlock(UnlockRequest rqst) + throws NoSuchLockException, TxnOpenException, TException { + try { + getTxnHandler().unlock(rqst); + } catch (MetaException e) { + throw new TException(e); + } + } + + @Override + public ShowLocksResponse show_locks(ShowLocksRequest rqst) throws TException { + try { + return getTxnHandler().showLocks(rqst); + } catch (MetaException e) { + throw new TException(e); + } + } + + @Override + public void heartbeat(HeartbeatRequest ids) + throws NoSuchLockException, NoSuchTxnException, TxnAbortedException, TException { + try { + getTxnHandler().heartbeat(ids); + } catch (MetaException e) { + throw new TException(e); + } + } + + @Override + public void compact(CompactionRequest rqst) throws TException { + try { + getTxnHandler().compact(rqst); + } catch (MetaException e) { + throw new TException(e); + } + } + + @Override + public ShowCompactResponse show_compact(ShowCompactRequest rqst) throws TException { + try { + return getTxnHandler().showCompact(rqst); + } catch (MetaException e) { + throw new TException(e); + } + } } public static IHMSHandler newHMSHandler(String name, HiveConf hiveConf) throws MetaException { diff --git metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java index 51464cf..8d5b225 100644 --- metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java +++ metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java @@ -26,8 +26,8 @@ import java.lang.reflect.InvocationTargetException; import java.lang.reflect.Method; import java.lang.reflect.Proxy; -import java.net.URI; -import java.net.URISyntaxException; +import java.net.InetAddress; +import java.net.UnknownHostException; import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Arrays; @@ -46,11 +46,16 @@ import org.apache.hadoop.hive.common.ObjectPair; import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.conf.HiveConf.ConfVars; +import org.apache.hadoop.hive.metastore.api.AbortTxnRequest; import org.apache.hadoop.hive.metastore.api.AddPartitionsRequest; import org.apache.hadoop.hive.metastore.api.AddPartitionsResult; import org.apache.hadoop.hive.metastore.api.AlreadyExistsException; +import org.apache.hadoop.hive.metastore.api.CheckLockRequest; import org.apache.hadoop.hive.metastore.api.ColumnStatistics; import org.apache.hadoop.hive.metastore.api.ColumnStatisticsObj; +import org.apache.hadoop.hive.metastore.api.CommitTxnRequest; +import org.apache.hadoop.hive.metastore.api.CompactionRequest; +import org.apache.hadoop.hive.metastore.api.CompactionType; import org.apache.hadoop.hive.metastore.api.ConfigValSecurityException; import org.apache.hadoop.hive.metastore.api.Database; import org.apache.hadoop.hive.metastore.api.DropPartitionsExpr; @@ -59,6 +64,9 @@ import org.apache.hadoop.hive.metastore.api.EnvironmentContext; import org.apache.hadoop.hive.metastore.api.FieldSchema; import org.apache.hadoop.hive.metastore.api.Function; +import org.apache.hadoop.hive.metastore.api.GetOpenTxnsInfoResponse; +import org.apache.hadoop.hive.metastore.api.GetOpenTxnsResponse; +import org.apache.hadoop.hive.metastore.api.HeartbeatRequest; import org.apache.hadoop.hive.metastore.api.HiveObjectPrivilege; import org.apache.hadoop.hive.metastore.api.HiveObjectRef; import org.apache.hadoop.hive.metastore.api.Index; @@ -66,8 +74,14 @@ import org.apache.hadoop.hive.metastore.api.InvalidObjectException; import org.apache.hadoop.hive.metastore.api.InvalidOperationException; import org.apache.hadoop.hive.metastore.api.InvalidPartitionException; +import org.apache.hadoop.hive.metastore.api.LockRequest; +import org.apache.hadoop.hive.metastore.api.LockResponse; import org.apache.hadoop.hive.metastore.api.MetaException; +import org.apache.hadoop.hive.metastore.api.NoSuchLockException; import org.apache.hadoop.hive.metastore.api.NoSuchObjectException; +import org.apache.hadoop.hive.metastore.api.NoSuchTxnException; +import org.apache.hadoop.hive.metastore.api.OpenTxnRequest; +import org.apache.hadoop.hive.metastore.api.OpenTxnsResponse; import org.apache.hadoop.hive.metastore.api.Partition; import org.apache.hadoop.hive.metastore.api.PartitionEventType; import org.apache.hadoop.hive.metastore.api.PartitionsByExprRequest; @@ -78,13 +92,20 @@ import org.apache.hadoop.hive.metastore.api.PrivilegeBag; import org.apache.hadoop.hive.metastore.api.RequestPartsSpec; import org.apache.hadoop.hive.metastore.api.Role; +import org.apache.hadoop.hive.metastore.api.ShowCompactRequest; +import org.apache.hadoop.hive.metastore.api.ShowCompactResponse; +import org.apache.hadoop.hive.metastore.api.ShowLocksRequest; +import org.apache.hadoop.hive.metastore.api.ShowLocksResponse; import org.apache.hadoop.hive.metastore.api.Table; import org.apache.hadoop.hive.metastore.api.TableStatsRequest; import org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore; +import org.apache.hadoop.hive.metastore.api.TxnAbortedException; +import org.apache.hadoop.hive.metastore.api.TxnOpenException; import org.apache.hadoop.hive.metastore.api.Type; import org.apache.hadoop.hive.metastore.api.UnknownDBException; import org.apache.hadoop.hive.metastore.api.UnknownPartitionException; import org.apache.hadoop.hive.metastore.api.UnknownTableException; +import org.apache.hadoop.hive.metastore.api.UnlockRequest; import org.apache.hadoop.hive.shims.HadoopShims; import org.apache.hadoop.hive.shims.ShimLoader; import org.apache.hadoop.hive.thrift.HadoopThriftAuthBridge; @@ -98,6 +119,12 @@ import org.apache.thrift.transport.TTransport; import org.apache.thrift.transport.TTransportException; +import java.net.URI; +import java.net.URISyntaxException; + +import static org.apache.hadoop.hive.metastore.MetaStoreUtils.DEFAULT_DATABASE_NAME; +import static org.apache.hadoop.hive.metastore.MetaStoreUtils.isIndexTable; + /** * Hive Metastore Client. */ @@ -665,7 +692,6 @@ public boolean dropPartition(String db_name, String tbl_name, List part_ * @param name * @param dbname * @throws NoSuchObjectException - * @throws ExistingDependentsException * @throws MetaException * @throws TException * @see org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore.Iface#drop_table(java.lang.String, @@ -689,7 +715,6 @@ public void dropTable(String tableName, boolean deleteData) * @param deleteData * delete the underlying data or just delete the table in metadata * @throws NoSuchObjectException - * @throws ExistingDependentsException * @throws MetaException * @throws TException * @see org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore.Iface#drop_table(java.lang.String, @@ -920,7 +945,8 @@ public Partition getPartitionWithAuthInfo(String db_name, String tbl_name, List part_vals, String user_name, List group_names) throws MetaException, UnknownTableException, NoSuchObjectException, TException { - return deepCopy(client.get_partition_with_auth(db_name, tbl_name, part_vals, user_name, group_names)); + return deepCopy(client.get_partition_with_auth(db_name, tbl_name, part_vals, user_name, + group_names)); } /** @@ -1444,6 +1470,167 @@ public void cancelDelegationToken(String tokenStrForm) throws MetaException, TEx client.cancel_delegation_token(tokenStrForm); } + public static class ValidTxnListImpl implements ValidTxnList { + + private GetOpenTxnsResponse txns; + + public ValidTxnListImpl() { + } + + public ValidTxnListImpl(GetOpenTxnsResponse t) { + txns = t; + } + + @Override + public boolean isTxnCommitted(long txnid) { + if (txns.getTxn_high_water_mark() < txnid) return false; + return !txns.getOpen_txns().contains(txnid); + } + + @Override + public RangeResponse isTxnRangeCommitted(long minTxnId, long maxTxnId) { + if (txns.getTxn_high_water_mark() < minTxnId) return RangeResponse.NONE; + + RangeResponse rc = RangeResponse.ALL; + boolean foundCommitted = false; + for (long id = minTxnId; id <= maxTxnId; id++) { + if (isTxnCommitted(id)) foundCommitted = true; + else rc = RangeResponse.SOME; + } + if (!foundCommitted) rc = RangeResponse.NONE; + return rc; + } + + @Override + public GetOpenTxnsResponse getOpenTxns() { + return txns; + } + + @Override + public String toString() { + StringBuffer buf = new StringBuffer(); + buf.append(getOpenTxns().getTxn_high_water_mark()); + Set openTxns = getOpenTxns().getOpen_txns(); + if (openTxns != null && openTxns.size() > 0) { + for (long txn : openTxns) { + buf.append(':'); + buf.append(txn); + } + } else { + buf.append(':'); + } + return buf.toString(); + } + + @Override + public void fromString(String src) { + // Make sure we have a non-null value in txns so that any future calls to this don't NPE. + txns = new GetOpenTxnsResponse(); + if (src == null) { + txns.setTxn_high_water_mark(Long.MAX_VALUE); + txns.setOpen_txns(new HashSet()); + return; + } + + String[] tString = src.split(":"); + txns.setTxn_high_water_mark(Long.valueOf(tString[0])); + Set openTxns = new HashSet(); + for (int i = 1; i < tString.length; i++) openTxns.add(Long.valueOf(tString[i])); + txns.setOpen_txns(openTxns); + } + } + + @Override + public ValidTxnList getValidTxns() throws TException { + GetOpenTxnsResponse txns = client.get_open_txns(); + return new ValidTxnListImpl(txns); + } + + @Override + public long openTxn(String user) throws TException { + OpenTxnsResponse txns = openTxns(user, 1); + return txns.getTxn_ids().get(0); + } + + @Override + public OpenTxnsResponse openTxns(String user, int numTxns) throws TException { + String hostname = null; + try { + hostname = InetAddress.getLocalHost().getHostName(); + } catch (UnknownHostException e) { + LOG.error("Unable to resolve my host name " + e.getMessage()); + throw new RuntimeException(e); + } + return client.open_txns(new OpenTxnRequest(numTxns, user, hostname)); + } + + @Override + public void rollbackTxn(long txnid) throws NoSuchTxnException, TException { + client.abort_txn(new AbortTxnRequest(txnid)); + } + + @Override + public void commitTxn(long txnid) + throws NoSuchTxnException, TxnAbortedException, TException { + client.commit_txn(new CommitTxnRequest(txnid)); + } + + @Override + public GetOpenTxnsInfoResponse showTxns() throws TException { + return client.get_open_txns_info(); + } + + @Override + public LockResponse lock(LockRequest request) + throws NoSuchTxnException, TxnAbortedException, TException { + return client.lock(request); + } + + @Override + public LockResponse checkLock(long lockid) + throws NoSuchTxnException, TxnAbortedException, NoSuchLockException, + TException { + return client.check_lock(new CheckLockRequest(lockid)); + } + + @Override + public void unlock(long lockid) + throws NoSuchLockException, TxnOpenException, TException { + client.unlock(new UnlockRequest(lockid)); + } + + @Override + public ShowLocksResponse showLocks() throws TException { + return client.show_locks(new ShowLocksRequest()); + } + + @Override + public void heartbeat(long txnid, long lockid) + throws NoSuchLockException, NoSuchTxnException, TxnAbortedException, + TException { + HeartbeatRequest hb = new HeartbeatRequest(); + hb.setLockid(lockid); + hb.setTxnid(txnid); + client.heartbeat(hb); + } + + @Override + public void compact(String dbname, String tableName, String partitionName, CompactionType type) + throws TException { + CompactionRequest cr = new CompactionRequest(); + if (dbname == null) cr.setDbname(DEFAULT_DATABASE_NAME); + else cr.setDbname(dbname); + cr.setTablename(tableName); + if (partitionName != null) cr.setPartitionname(partitionName); + cr.setType(type); + client.compact(cr); + } + + @Override + public ShowCompactResponse showCompactions() throws TException { + return client.show_compact(new ShowCompactRequest()); + } + /** * Creates a synchronized wrapper for any {@link IMetaStoreClient}. * This may be used by multi-threaded applications until we have @@ -1483,7 +1670,8 @@ public Object invoke(Object proxy, Method method, Object [] args) @Override public void markPartitionForEvent(String db_name, String tbl_name, Map partKVs, PartitionEventType eventType) - throws MetaException, TException, NoSuchObjectException, UnknownDBException, UnknownTableException, + throws MetaException, TException, NoSuchObjectException, UnknownDBException, + UnknownTableException, InvalidPartitionException, UnknownPartitionException { assert db_name != null; assert tbl_name != null; diff --git metastore/src/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java metastore/src/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java index 3515ae5..612a1a7 100644 --- metastore/src/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java +++ metastore/src/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java @@ -18,6 +18,20 @@ package org.apache.hadoop.hive.metastore; +import org.apache.hadoop.hive.metastore.api.CompactionType; +import org.apache.hadoop.hive.metastore.api.GetOpenTxnsInfoResponse; +import org.apache.hadoop.hive.metastore.api.GetOpenTxnsResponse; +import org.apache.hadoop.hive.metastore.api.LockRequest; +import org.apache.hadoop.hive.metastore.api.LockResponse; +import org.apache.hadoop.hive.metastore.api.NoSuchLockException; +import org.apache.hadoop.hive.metastore.api.NoSuchTxnException; +import org.apache.hadoop.hive.metastore.api.OpenTxnsResponse; +import org.apache.hadoop.hive.metastore.api.ShowCompactResponse; +import org.apache.hadoop.hive.metastore.api.ShowLocksResponse; +import org.apache.hadoop.hive.metastore.api.TxnAbortedException; +import org.apache.hadoop.hive.metastore.api.TxnOpenException; +import org.apache.thrift.TException; + import java.util.List; import java.util.Map; @@ -159,7 +173,6 @@ * The table wasn't found. * @throws TException * A thrift communication error occurred - * @throws ExistingDependentsException */ public void dropTable(String dbname, String tableName, boolean deleteData, boolean ignoreUknownTab) throws MetaException, TException, @@ -361,7 +374,9 @@ public Partition getPartition(String tblName, String dbName, List partVals) throws NoSuchObjectException, MetaException, TException; /** - * @param partition + * @param partitionSpecs + * @param sourceDb + * @param sourceTable * @param destdb * @param destTableName * @return partition object @@ -1020,6 +1035,262 @@ public Function getFunction(String dbName, String funcName) public List getFunctions(String dbName, String pattern) throws MetaException, TException; + // Transaction and locking methods + public interface ValidTxnList { + + /** + * Key used to store valid txn list in a {@link org.apache.hadoop.conf.Configuration} object. + */ + public static final String VALID_TXNS_KEY = "hive.txn.valid.txns"; + + /** + * The response to a range query. NONE means no values in this range match, + * SOME mean that some do, and ALL means that every value does. + */ + public enum RangeResponse {NONE, SOME, ALL}; + + /** + * Indicates whether a given transaction has been committed and should be + * viewed as valid for read. + * @param txnid id for the transaction + * @return true if committed, false otherwise + */ + public boolean isTxnCommitted(long txnid); + + /** + * Find out if a range of transaction ids have been committed. + * @param minTxnId minimum txnid to look for, inclusive + * @param maxTxnId maximum txnid to look for, inclusive + * @return Indicate whether none, some, or all of these transactions have been committed. + */ + public RangeResponse isTxnRangeCommitted(long minTxnId, long maxTxnId); + + /** + * Get at the underlying OpenTxn structure. This is useful if the user + * wishes to get a list of all open transactions for more efficient + * filtering. + * @return open transactions + */ + public GetOpenTxnsResponse getOpenTxns(); + + /** + * Write this validTxnList into a string. Obviously all implementations will already + * implement this, but it is being called out specifically here to make clear that the + * implementation needs to override the default implementation. This should produce a string + * that can be used by {@link #fromString(String)} to populate a validTxnsList. + */ + public String toString(); + + /** + * Populate this validTxnList from the string. It is assumed that the string was created via + * {@link #toString()}. + * @param src source string. + */ + public void fromString(String src); + } + + /** + * Get a structure that details valid transactions. + * @return list of valid transactions + * @throws TException + */ + public ValidTxnList getValidTxns() throws TException; + + /** + * Initiate a transaction. + * @param user User who is opening this transaction. This is the Hive user, + * not necessarily the OS user. It is assumed that this user has already been + * authenticated and authorized at this point. + * @return transaction identifier + * @throws TException + */ + public long openTxn(String user) throws TException; + + /** + * Initiate a batch of transactions. It is not guaranteed that the + * requested number of transactions will be instantiated. The system has a + * maximum number instantiated per request, controlled by hive.txn.max + * .batch.open in hive-site.xml. If the user requests more than this + * value, only the configured max will be returned. + * + *

Increasing the number of transactions requested in the batch will + * allow applications that stream data into Hive to place more commits in a + * single file, thus reducing load on the namenode and making reads of the + * data more efficient. However, opening more transactions in a batch will + * also result in readers needing to keep a larger list of open + * transactions to ignore, potentially slowing their reads. Users will + * need to test in their system to understand the optimal number of + * transactions to request in a batch. + *

+ * @param user User who is opening this transaction. This is the Hive user, + * not necessarily the OS user. It is assumed that this user has already been + * authenticated and authorized at this point. + * @param numTxns number of requested transactions to open + * @return list of opened txn ids. As noted above, this may be less than + * requested, so the user should check how many were returned rather than + * optimistically assuming that the result matches the request. + * @throws TException + */ + public OpenTxnsResponse openTxns(String user, int numTxns) throws TException; + + /** + * Rollback a transaction. This will also unlock any locks associated with + * this transaction. + * @param txnid id of transaction to be rolled back. + * @throws NoSuchTxnException if the requested transaction does not exist. + * Note that this can result from the transaction having timed out and been + * deleted. + * @throws TException + */ + public void rollbackTxn(long txnid) throws NoSuchTxnException, TException; + + /** + * Commit a transaction. This will also unlock any locks associated with + * this transaction. + * @param txnid id of transaction to be committed. + * @throws NoSuchTxnException if the requested transaction does not exist. + * This can result fro the transaction having timed out and been deleted by + * the compactor. + * @throws TxnAbortedException if the requested transaction has been + * aborted. This can result from the transaction timing out. + * @throws TException + */ + public void commitTxn(long txnid) + throws NoSuchTxnException, TxnAbortedException, TException; + + /** + * Show the list of currently open transactions. This is for use by "show transactions" in the + * grammar, not for applications that want to find a list of current transactions to work with. + * Those wishing the latter should call {@link #getValidTxns()}. + * @return List of currently opened transactions, included aborted ones. + * @throws TException + */ + public GetOpenTxnsInfoResponse showTxns() throws TException; + + /** + * Request a set of locks. All locks needed for a particular query, DML, + * or DDL operation should be batched together and requested in one lock + * call. This avoids deadlocks. It also avoids blocking other users who + * only require some of the locks required by this user. + * + *

If the operation requires a transaction (INSERT, UPDATE, + * or DELETE) that transaction id must be provided as part this lock + * request. All locks associated with a transaction will be released when + * that transaction is committed or rolled back.

+ * * + *

Once a lock is acquired, {@link #heartbeat(long, long)} must be called + * on a regular basis to avoid the lock being timed out by the system.

+ * @param request The lock request. {@link LockRequestBuilder} can be used + * construct this request. + * @return a lock response, which will provide two things, + * the id of the lock (to be used in all further calls regarding this lock) + * as well as a state of the lock. If the state is ACQUIRED then the user + * can proceed. If it is WAITING the user should wait and call + * {@link #checkLock(long)} before proceeding. All components of the lock + * will have the same state. + * @throws NoSuchTxnException if the requested transaction does not exist. + * This can result fro the transaction having timed out and been deleted by + * the compactor. + * @throws TxnAbortedException if the requested transaction has been + * aborted. This can result from the transaction timing out. + * @throws TException + */ + public LockResponse lock(LockRequest request) + throws NoSuchTxnException, TxnAbortedException, TException; + + /** + * Check the status of a set of locks requested via a + * {@link #lock(org.apache.hadoop.hive.metastore.api.LockRequest)} call. + * Once a lock is acquired, {@link #heartbeat(long, long)} must be called + * on a regular basis to avoid the lock being timed out by the system. + * @param lockid lock id returned by lock(). + * @return a lock response, which will provide two things, + * the id of the lock (to be used in all further calls regarding this lock) + * as well as a state of the lock. If the state is ACQUIRED then the user + * can proceed. If it is WAITING the user should wait and call + * this method again before proceeding. All components of the lock + * will have the same state. + * @throws NoSuchTxnException if the requested transaction does not exist. + * This can result fro the transaction having timed out and been deleted by + * the compactor. + * @throws TxnAbortedException if the requested transaction has been + * aborted. This can result from the transaction timing out. + * @throws NoSuchLockException if the requested lockid does not exist. + * This can result from the lock timing out and being unlocked by the system. + * @throws TException + */ + public LockResponse checkLock(long lockid) + throws NoSuchTxnException, TxnAbortedException, NoSuchLockException, + TException; + + /** + * Unlock a set of locks. This can only be called when the locks are not + * assocaited with a transaction. + * @param lockid lock id returned by + * {@link #lock(org.apache.hadoop.hive.metastore.api.LockRequest)} + * @throws NoSuchLockException if the requested lockid does not exist. + * This can result from the lock timing out and being unlocked by the system. + * @throws TxnOpenException if the locks are are associated with a + * transaction. + * @throws TException + */ + public void unlock(long lockid) + throws NoSuchLockException, TxnOpenException, TException; + + /** + * Show all currently held and waiting locks. + * @return List of currently held and waiting locks. + * @throws TException + */ + public ShowLocksResponse showLocks() throws TException; + + /** + * Send a heartbeat to indicate that the client holding these locks (if + * any) and that opened this transaction (if one exists) is still alive. + * The default timeout for transactions and locks is 300 seconds, + * though it is configurable. To determine how often to heartbeat you will + * need to ask your system administrator how the metastore thrift service + * has been configured. + * @param txnid the id of the open transaction. If no transaction is open + * (it is a DDL or query) then this can be set to 0. + * @param lockid the id of the locks obtained. If no locks have been + * obtained then this can be set to 0. + * @throws NoSuchTxnException if the requested transaction does not exist. + * This can result fro the transaction having timed out and been deleted by + * the compactor. + * @throws TxnAbortedException if the requested transaction has been + * aborted. This can result from the transaction timing out. + * @throws NoSuchLockException if the requested lockid does not exist. + * This can result from the lock timing out and being unlocked by the system. + * @throws TException + */ + public void heartbeat(long txnid, long lockid) + throws NoSuchLockException, NoSuchTxnException, TxnAbortedException, + TException; + + /** + * Send a request to compact a table or partition. This will not block until the compaction is + * complete. It will instead put a request on the queue for that table or partition to be + * compacted. No checking is done on the dbname, tableName, or partitionName to make sure they + * refer to valid objects. It is assumed this has already been done by the caller. + * @param dbname Name of the database the table is in. If null, this will be assumed to be + * 'default'. + * @param tableName Name of the table to be compacted. This cannot be null. If partitionName + * is null, this must be a non-partitioned table. + * @param partitionName Name of the partition to be compacted + * @param type Whether this is a major or minor compaction. + * @throws TException + */ + public void compact(String dbname, String tableName, String partitionName, CompactionType type) + throws TException; + + /** + * Get a list of all current compactions. + * @return List of all current compactions. This includes compactions waiting to happen, + * in progress, and finished but waiting to clean the existing files. + * @throws TException + */ + public ShowCompactResponse showCompactions() throws TException; public class IncompatibleMetastoreException extends MetaException { public IncompatibleMetastoreException(String message) { diff --git metastore/src/java/org/apache/hadoop/hive/metastore/LockComponentBuilder.java metastore/src/java/org/apache/hadoop/hive/metastore/LockComponentBuilder.java new file mode 100644 index 0000000..acd4653 --- /dev/null +++ metastore/src/java/org/apache/hadoop/hive/metastore/LockComponentBuilder.java @@ -0,0 +1,107 @@ +/** + * 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. + */ +package org.apache.hadoop.hive.metastore; + +import org.apache.hadoop.hive.metastore.api.LockComponent; +import org.apache.hadoop.hive.metastore.api.LockLevel; +import org.apache.hadoop.hive.metastore.api.LockType; + +/** + * A builder for {@link LockComponent}s + */ +public class LockComponentBuilder { + private LockComponent component; + private boolean tableNameSet; + private boolean partNameSet; + + public LockComponentBuilder() { + component = new LockComponent(); + tableNameSet = partNameSet = false; + } + + /** + * Set the lock to be exclusive. + * @return reference to this builder + */ + public LockComponentBuilder setExclusive() { + component.setType(LockType.EXCLUSIVE); + return this; + } + + /** + * Set the lock to be semi-shared. + * @return reference to this builder + */ + public LockComponentBuilder setSemiShared() { + component.setType(LockType.SHARED_WRITE); + return this; + } + + /** + * Set the lock to be shared. + * @return reference to this builder + */ + public LockComponentBuilder setShared() { + component.setType(LockType.SHARED_READ); + return this; + } + + /** + * Set the database name. + * @param dbName database name + * @return reference to this builder + */ + public LockComponentBuilder setDbName(String dbName) { + component.setDbname(dbName); + return this; + } + + /** + * Set the table name. + * @param tableName table name + * @return reference to this builder + */ + public LockComponentBuilder setTableName(String tableName) { + component.setTablename(tableName); + tableNameSet = true; + return this; + } + + /** + * Set the partition name. + * @param partitionName partition name + * @return reference to this builder + */ + public LockComponentBuilder setPartitionName(String partitionName) { + component.setPartitionname(partitionName); + partNameSet = true; + return this; + } + + /** + * Get the constructed lock component. + * @return lock component. + */ + public LockComponent build() { + LockLevel level = LockLevel.DB; + if (tableNameSet) level = LockLevel.TABLE; + if (partNameSet) level = LockLevel.PARTITION; + component.setLevel(level); + return component; + } +} diff --git metastore/src/java/org/apache/hadoop/hive/metastore/LockRequestBuilder.java metastore/src/java/org/apache/hadoop/hive/metastore/LockRequestBuilder.java new file mode 100644 index 0000000..2fa7e07 --- /dev/null +++ metastore/src/java/org/apache/hadoop/hive/metastore/LockRequestBuilder.java @@ -0,0 +1,159 @@ +/** + * 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. + */ +package org.apache.hadoop.hive.metastore; + +import org.apache.hadoop.hive.metastore.api.LockComponent; +import org.apache.hadoop.hive.metastore.api.LockRequest; +import org.apache.hadoop.hive.metastore.api.LockType; + +import java.net.InetAddress; +import java.net.UnknownHostException; +import java.util.HashMap; +import java.util.Map; + +/** + * Builder class to make constructing {@link LockRequest} easier. + */ +public class LockRequestBuilder { + + private LockRequest req; + private LockTrie trie; + private boolean userSet; + + public LockRequestBuilder() { + req = new LockRequest(); + trie = new LockTrie(); + userSet = false; + } + + /** + * Get the constructed LockRequest. + * @return lock request + */ + public LockRequest build() { + if (!userSet) { + throw new RuntimeException("Cannot build a lock without giving a user"); + } + trie.addLocksToRequest(req); + try { + req.setHostname(InetAddress.getLocalHost().getHostName()); + } catch (UnknownHostException e) { + throw new RuntimeException("Unable to determine our local host!"); + } + return req; + } + + /** + * Set the transaction id. + * @param txnid transaction id + * @return reference to this builder + */ + public LockRequestBuilder setTransactionId(long txnid) { + req.setTxnid(txnid); + return this; + } + + public LockRequestBuilder setUser(String user) { + if (user == null) user = "unknown"; + req.setUser(user); + userSet = true; + return this; + } + + /** + * Add a lock component to the lock request + * @param component to add + * @return reference to this builder + */ + public LockRequestBuilder addLockComponent(LockComponent component) { + trie.add(component); + return this; + } + + // For reasons that are completely incomprehensible to me the semantic + // analyzers often ask for multiple locks on the same entity (for example + // a shared_read and an exlcusive lock). The db locking system gets confused + // by this and dead locks on it. To resolve that, we'll make sure in the + // request that multiple locks are coalesced and promoted to the higher + // level of locking. To do this we put all locks components in trie based + // on dbname, tablename, partition name and handle the promotion as new + // requests come in. This structure depends on the fact that null is a + // valid key in a HashMap. So a database lock will map to (dbname, null, + // null). + private static class LockTrie { + Map trie; + + LockTrie() { + trie = new HashMap(); + } + + public void add(LockComponent comp) { + TableTrie tabs = trie.get(comp.getDbname()); + if (tabs == null) { + tabs = new TableTrie(); + trie.put(comp.getDbname(), tabs); + } + setTable(comp, tabs); + } + + public void addLocksToRequest(LockRequest request) { + for (TableTrie tab : trie.values()) { + for (PartTrie part : tab.values()) { + for (LockComponent lock : part.values()) { + request.addToComponent(lock); + } + } + } + } + + private void setTable(LockComponent comp, TableTrie tabs) { + PartTrie parts = tabs.get(comp.getTablename()); + if (parts == null) { + parts = new PartTrie(); + tabs.put(comp.getTablename(), parts); + } + setPart(comp, parts); + } + + private void setPart(LockComponent comp, PartTrie parts) { + LockComponent existing = parts.get(comp.getPartitionname()); + if (existing == null) { + // No existing lock for this partition. + parts.put(comp.getPartitionname(), comp); + } else if (existing.getType() != LockType.EXCLUSIVE && + (comp.getType() == LockType.EXCLUSIVE || + comp.getType() == LockType.SHARED_WRITE)) { + // We only need to promote if comp.type is > existing.type. For + // efficiency we check if existing is exclusive (in which case we + // need never promote) or if comp is exclusive or shared_write (in + // which case we can promote even though they may both be shared + // write). If comp is shared_read there's never a need to promote. + parts.put(comp.getPartitionname(), comp); + } + } + + private static class TableTrie extends HashMap { + } + + private static class PartTrie extends HashMap { + } + + + + } +} diff --git metastore/src/java/org/apache/hadoop/hive/metastore/txn/CompactionInfo.java metastore/src/java/org/apache/hadoop/hive/metastore/txn/CompactionInfo.java new file mode 100644 index 0000000..a818300 --- /dev/null +++ metastore/src/java/org/apache/hadoop/hive/metastore/txn/CompactionInfo.java @@ -0,0 +1,60 @@ +/** + * 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. + */ +package org.apache.hadoop.hive.metastore.txn; + +import org.apache.hadoop.hive.metastore.api.CompactionType; + +/** + * Information on a possible or running compaction. + */ +public class CompactionInfo { + public long id; + public String dbname; + public String tableName; + public String partName; + public CompactionType type; + public String runAs; + public boolean tooManyAborts = false; + + private String fullPartitionName = null; + private String fullTableName = null; + + public String getFullPartitionName() { + if (fullPartitionName == null) { + StringBuffer buf = new StringBuffer(dbname); + buf.append('.'); + buf.append(tableName); + if (partName != null) { + buf.append('.'); + buf.append(partName); + } + fullPartitionName = buf.toString(); + } + return fullPartitionName; + } + + public String getFullTableName() { + if (fullTableName == null) { + StringBuffer buf = new StringBuffer(dbname); + buf.append('.'); + buf.append(tableName); + fullTableName = buf.toString(); + } + return fullTableName; + } +} diff --git metastore/src/java/org/apache/hadoop/hive/metastore/txn/CompactionTxnHandler.java metastore/src/java/org/apache/hadoop/hive/metastore/txn/CompactionTxnHandler.java new file mode 100644 index 0000000..be399db --- /dev/null +++ metastore/src/java/org/apache/hadoop/hive/metastore/txn/CompactionTxnHandler.java @@ -0,0 +1,456 @@ +/** + * 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. + */ +package org.apache.hadoop.hive.metastore.txn; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; + +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.metastore.api.*; + +import java.sql.*; +import java.util.*; + +/** + * Extends the transaction handler with methods needed only by the compactor threads. These + * methods are not available through the thrift interface. + */ +public class CompactionTxnHandler extends TxnHandler { + static final private String CLASS_NAME = CompactionTxnHandler.class.getName(); + static final private Log LOG = LogFactory.getLog(CLASS_NAME); + + public CompactionTxnHandler(HiveConf conf) { + super(conf); + } + + /** + * This will look through the completed_txn_components table and look for partitions or tables + * that may be ready for compaction. Also, look through txns and txn_components tables for + * aborted transactions that we should add to the list. + * @param maxAborted Maximum number of aborted queries to allow before marking this as a + * potential compaction. + * @return list of CompactionInfo structs. These will not have id, type, + * or runAs set since these are only potential compactions not actual ones. + */ + public List findPotentialCompactions(int maxAborted) throws MetaException { + Connection dbConn = getDbConn(); + List response = new ArrayList(); + try { + Statement stmt = dbConn.createStatement(); + // Check for completed transactions + String s = "select ctc_database, ctc_table, ctc_partition from COMPLETED_TXN_COMPONENTS"; + LOG.debug("Going to execute query <" + s + ">"); + ResultSet rs = stmt.executeQuery(s); + while (rs.next()) { + CompactionInfo info = new CompactionInfo(); + info.dbname = rs.getString(1); + info.tableName = rs.getString(2); + info.partName = rs.getString(3); + response.add(info); + } + + s = "select tc_database, tc_table, tc_partition " + + "from TXNS, TXN_COMPONENTS " + + "where txn_id = tc_txnid and txn_state = '" + TXN_ABORTED + "' " + + "group by tc_database, tc_table, tc_partition " + + "having count(*) > " + maxAborted; + + LOG.debug("Going to execute query <" + s + ">"); + rs = stmt.executeQuery(s); + while (rs.next()) { + CompactionInfo info = new CompactionInfo(); + info.dbname = rs.getString(1); + info.tableName = rs.getString(2); + info.partName = rs.getString(3); + info.tooManyAborts = true; + response.add(info); + } + + LOG.debug("Going to rollback"); + dbConn.rollback(); + } catch (SQLException e) { + LOG.error("Unable to connect to transaction database " + e.getMessage()); + } finally { + closeDbConn(dbConn); + } + return response; + } + + /** + * Sets the user to run as. This is for the case + * where the request was generated by the user and so the worker must set this value later. + * @param cq_id id of this entry in the queue + * @param user user to run the jobs as + */ + public void setRunAs(long cq_id, String user) throws MetaException { + Connection dbConn = getDbConn(); + try { + Statement stmt = dbConn.createStatement(); + String s = "update COMPACTION_QUEUE set cq_run_as = '" + user + "' where cq_id = " + cq_id; + LOG.debug("Going to execute update <" + s + ">"); + if (stmt.executeUpdate(s) != 1) { + LOG.error("Unable to update compaction record"); + LOG.debug("Going to rollback"); + dbConn.rollback(); + } + LOG.debug("Going to commit"); + dbConn.commit(); + } catch (SQLException e) { + LOG.error("Unable to update compaction queue, " + e.getMessage()); + try { + LOG.debug("Going to rollback"); + dbConn.rollback(); + } catch (SQLException e1) { + } + } finally { + closeDbConn(dbConn); + } + } + + /** + * This will grab the next compaction request off of + * the queue, and assign it to the worker. + * @param workerId id of the worker calling this, will be recorded in the db + * @return an info element for this compaction request, or null if there is no work to do now. + */ + public CompactionInfo findNextToCompact(String workerId) throws MetaException { + Connection dbConn = getDbConn(); + CompactionInfo info = new CompactionInfo(); + + try { + Statement stmt = dbConn.createStatement(); + String s = "select cq_id, cq_database, cq_table, cq_partition, " + + "cq_type from COMPACTION_QUEUE where cq_state = '" + INITIATED_STATE + "' for update"; + LOG.debug("Going to execute query <" + s + ">"); + ResultSet rs = stmt.executeQuery(s); + if (!rs.next()) { + LOG.debug("No compactions found ready to compact"); + dbConn.rollback(); + return null; + } + info.id = rs.getLong(1); + info.dbname = rs.getString(2); + info.tableName = rs.getString(3); + info.partName = rs.getString(4); + switch (rs.getString(5).charAt(0)) { + case MAJOR_TYPE: info.type = CompactionType.MAJOR; break; + case MINOR_TYPE: info.type = CompactionType.MINOR; break; + default: throw new MetaException("Unexpected compaction type " + rs.getString(5)); + } + + // Now, update this record as being worked on by this worker. + long now = System.currentTimeMillis(); + s = "update COMPACTION_QUEUE set cq_worker_id = '" + workerId + "', " + + "cq_start = " + now + ", cq_state = '" + WORKING_STATE + "' where cq_id = " + info.id; + LOG.debug("Going to execute update <" + s + ">"); + if (stmt.executeUpdate(s) != 1) { + LOG.error("Unable to update compaction record"); + LOG.debug("Going to rollback"); + dbConn.rollback(); + } + LOG.debug("Going to commit"); + dbConn.commit(); + return info; + } catch (SQLException e) { + LOG.error("Unable to select next element for compaction, " + e.getMessage()); + try { + LOG.debug("Going to rollback"); + dbConn.rollback(); + } catch (SQLException e1) { + } + return null; + } finally { + closeDbConn(dbConn); + } + } + + /** + * This will mark an entry in the queue as compacted + * and put it in the ready to clean state. + * @param info info on the compaciton entry to mark as compacted. + */ + public void markCompacted(CompactionInfo info) throws MetaException { + Connection dbConn = getDbConn(); + try { + Statement stmt = dbConn.createStatement(); + String s = "update COMPACTION_QUEUE set cq_state = '" + READY_FOR_CLEANING + "', " + + "cq_worker_id = null where cq_id = " + info.id; + LOG.debug("Going to execute update <" + s + ">"); + if (stmt.executeUpdate(s) != 1) { + LOG.error("Unable to update compaction record"); + LOG.debug("Going to rollback"); + dbConn.rollback(); + } + LOG.debug("Going to commit"); + dbConn.commit(); + } catch (SQLException e) { + try { + LOG.error("Unable to update compaction queue " + e.getMessage()); + LOG.debug("Going to rollback"); + dbConn.rollback(); + } catch (SQLException e1) { + } + } finally { + closeDbConn(dbConn); + } + } + + /** + * Find entries in the queue that are ready to + * be cleaned. + * @return information on the entry in the queue. + */ + public List findReadyToClean() throws MetaException { + Connection dbConn = getDbConn(); + List rc = new ArrayList(); + + try { + Statement stmt = dbConn.createStatement(); + String s = "select cq_id, cq_database, cq_table, cq_partition, " + + "cq_type, cq_run_as from COMPACTION_QUEUE where cq_state = '" + READY_FOR_CLEANING + "'" + + " for update"; + LOG.debug("Going to execute query <" + s + ">"); + ResultSet rs = stmt.executeQuery(s); + while (rs.next()) { + CompactionInfo info = new CompactionInfo(); + info.id = rs.getLong(1); + info.dbname = rs.getString(2); + info.tableName = rs.getString(3); + info.partName = rs.getString(4); + switch (rs.getString(5).charAt(0)) { + case MAJOR_TYPE: info.type = CompactionType.MAJOR; break; + case MINOR_TYPE: info.type = CompactionType.MINOR; break; + default: throw new MetaException("Unexpected compaction type " + rs.getString(5)); + } + info.runAs = rs.getString(6); + rc.add(info); + } + LOG.debug("Going to rollback"); + dbConn.rollback(); + return rc; + } catch (SQLException e) { + LOG.error("Unable to select next element for cleaning, " + e.getMessage()); + try { + LOG.debug("Going to rollback"); + dbConn.rollback(); + } catch (SQLException e1) { + } + return null; + } finally { + closeDbConn(dbConn); + } + } + + /** + * This will remove an entry from the queue after + * it has been compacted. + * @param info info on the compaction entry to remove + */ + public void markCleaned(CompactionInfo info) throws MetaException { + Connection dbConn = getDbConn(); + try { + Statement stmt = dbConn.createStatement(); + String s = "delete from COMPACTION_QUEUE where cq_id = " + info.id; + LOG.debug("Going to execute update <" + s + ">"); + if (stmt.executeUpdate(s) != 1) { + LOG.error("Unable to delete compaction record"); + LOG.debug("Going to rollback"); + dbConn.rollback(); + } + + // Remove entries from completed_txn_components as well, so we don't start looking there + // again. + s = "delete from COMPLETED_TXN_COMPONENTS where ctc_database = '" + info.dbname + "' and " + + "ctc_table = '" + info.tableName + "'"; + if (info.partName != null) { + s += " and ctc_partition = '" + info.partName + "'"; + } + LOG.debug("Going to execute update <" + s + ">"); + if (stmt.executeUpdate(s) < 1) { + LOG.error("Expected to remove at least one row from completed_txn_components when " + + "marking compaction entry as clean!"); + } + + + s = "select txn_id from TXNS, TXN_COMPONENTS where txn_id = tc_txnid and txn_state = '" + + TXN_ABORTED + "' and tc_database = '" + info.dbname + "' and tc_table = '" + + info.tableName + "'"; + if (info.partName != null) s += " and tc_partition = '" + info.partName + "'"; + LOG.debug("Going to execute update <" + s + ">"); + ResultSet rs = stmt.executeQuery(s); + Set txnids = new HashSet(); + while (rs.next()) txnids.add(rs.getLong(1)); + if (txnids.size() > 0) { + + // Remove entries from txn_components, as there may be aborted txn components + StringBuffer buf = new StringBuffer(); + buf.append("delete from TXN_COMPONENTS where tc_txnid in ("); + boolean first = true; + for (long id : txnids) { + if (first) first = false; + else buf.append(", "); + buf.append(id); + } + + buf.append(") and tc_database = '"); + buf.append(info.dbname); + buf.append("' and tc_table = '"); + buf.append(info.tableName); + buf.append("'"); + if (info.partName != null) { + buf.append(" and tc_partition = '"); + buf.append(info.partName); + buf.append("'"); + } + LOG.debug("Going to execute update <" + buf.toString() + ">"); + int rc = stmt.executeUpdate(buf.toString()); + LOG.debug("Removed " + rc + " records from txn_components"); + + // Don't bother cleaning from the txns table. A separate call will do that. We don't + // know here which txns still have components from other tables or partitions in the + // table, so we don't know which ones we can and cannot clean. + } + + LOG.debug("Going to commit"); + dbConn.commit(); + } catch (SQLException e) { + try { + LOG.error("Unable to delete from compaction queue " + e.getMessage()); + LOG.debug("Going to rollback"); + dbConn.rollback(); + } catch (SQLException e1) { + } + } finally { + closeDbConn(dbConn); + } + } + + /** + * Clean up aborted transactions from txns that have no components in txn_components. + */ + public void cleanEmptyAbortedTxns() throws MetaException { + Connection dbConn = getDbConn(); + try { + Statement stmt = dbConn.createStatement(); + String s = "select txn_id from TXNS where " + + "txn_id not in (select tc_txnid from TXN_COMPONENTS) and " + + "txn_state = '" + TXN_ABORTED + "'"; + LOG.debug("Going to execute query <" + s + ">"); + ResultSet rs = stmt.executeQuery(s); + Set txnids = new HashSet(); + while (rs.next()) txnids.add(rs.getLong(1)); + if (txnids.size() > 0) { + StringBuffer buf = new StringBuffer("delete from TXNS where txn_id in ("); + boolean first = true; + for (long tid : txnids) { + if (first) first = false; + else buf.append(", "); + buf.append(tid); + } + buf.append(")"); + LOG.debug("Going to execute update <" + buf.toString() + ">"); + int rc = stmt.executeUpdate(buf.toString()); + LOG.debug("Removed " + rc + " records from txns"); + LOG.debug("Going to commit"); + dbConn.commit(); + } + } catch (SQLException e) { + LOG.error("Unable to delete from txns table " + e.getMessage()); + LOG.debug("Going to rollback"); + try { + dbConn.rollback(); + } catch (SQLException e1) { + } + } finally { + closeDbConn(dbConn); + } + } + + /** + * This will take all entries assigned to workers + * on a host return them to INITIATED state. The initiator should use this at start up to + * clean entries from any workers that were in the middle of compacting when the metastore + * shutdown. It does not reset entries from worker threads on other hosts as those may still + * be working. + * @param hostname Name of this host. It is assumed this prefixes the thread's worker id, + * so that like hostname% will match the worker id. + */ + public void revokeFromLocalWorkers(String hostname) throws MetaException { + Connection dbConn = getDbConn(); + try { + Statement stmt = dbConn.createStatement(); + String s = "update COMPACTION_QUEUE set cq_worker_id = null, cq_start = null, cq_state = '" + + INITIATED_STATE+ "' where cq_state = '" + WORKING_STATE + "' and cq_worker_id like '" + + hostname + "%'"; + LOG.debug("Going to execute update <" + s + ">"); + // It isn't an error if the following returns no rows, as the local workers could have died + // with nothing assigned to them. + stmt.executeUpdate(s); + LOG.debug("Going to commit"); + dbConn.commit(); + } catch (SQLException e) { + try { + LOG.error("Unable to change dead worker's records back to initiated state " + + e.getMessage()); + LOG.debug("Going to rollback"); + dbConn.rollback(); + } catch (SQLException e1) { + } + } finally { + closeDbConn(dbConn); + } + } + + /** + * This call will return all compaction queue + * entries assigned to a worker but over the timeout back to the initiated state. + * This should be called by the initiator on start up and occasionally when running to clean up + * after dead threads. At start up {@link #revokeFromLocalWorkers(String)} should be called + * first. + * @param timeout number of milliseconds since start time that should elapse before a worker is + * declared dead. + */ + public void revokeTimedoutWorkers(long timeout) throws MetaException { + Connection dbConn = getDbConn(); + long latestValidStart = System.currentTimeMillis() - timeout; + try { + Statement stmt = dbConn.createStatement(); + String s = "update COMPACTION_QUEUE set cq_worker_id = null, cq_start = null, cq_state = '" + + INITIATED_STATE+ "' where cq_state = '" + WORKING_STATE + "' and cq_start < " + + latestValidStart; + LOG.debug("Going to execute update <" + s + ">"); + // It isn't an error if the following returns no rows, as the local workers could have died + // with nothing assigned to them. + stmt.executeUpdate(s); + LOG.debug("Going to commit"); + dbConn.commit(); + } catch (SQLException e) { + try { + LOG.error("Unable to change dead worker's records back to initiated state " + + e.getMessage()); + LOG.debug("Going to rollback"); + dbConn.rollback(); + } catch (SQLException e1) { + } + } finally { + closeDbConn(dbConn); + } + } +} + + diff --git metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnDbUtil.java metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnDbUtil.java new file mode 100644 index 0000000..bbb0d28 --- /dev/null +++ metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnDbUtil.java @@ -0,0 +1,205 @@ +/** + * 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. + */ +package org.apache.hadoop.hive.metastore.txn; + +import org.apache.hadoop.hive.conf.HiveConf; + +import java.sql.Connection; +import java.sql.Driver; +import java.sql.ResultSet; +import java.sql.Statement; +import java.util.Properties; + +/** + * Utility methods for creating and destroying txn database/schema. Placed + * here in a separate class so it can be shared across unit tests. + */ +public class TxnDbUtil { + private final static String jdbcString = "jdbc:derby:;databaseName=metastore_db;create=true"; + private final static String jdbcDriver = "org.apache.derby.jdbc.EmbeddedDriver"; + private final static String txnMgr = "org.apache.hadoop.hive.ql.lockmgr.DbTxnManager"; + + /** + * Set up the configuration so it will use the DbTxnManager, concurrency will be set to true, + * and the JDBC configs will be set for putting the transaction and lock info in the embedded + * metastore. + * @param conf HiveConf to add these values to. + */ + public static void setConfValues(HiveConf conf) { + conf.setVar(HiveConf.ConfVars.HIVE_TXN_JDBC_DRIVER, jdbcDriver); + conf.setVar(HiveConf.ConfVars.HIVE_TXN_JDBC_CONNECT_STRING, jdbcString); + conf.setVar(HiveConf.ConfVars.HIVE_TXN_MANAGER, txnMgr); + conf.setBoolVar(HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY, true); + } + + public static void prepDb() throws Exception { + // This is a bogus hack because it copies the contents of the SQL file + // intended for creating derby databases, and thus will inexorably get + // out of date with it. I'm open to any suggestions on how to make this + // read the file in a build friendly way. + Driver driver = (Driver)Class.forName(jdbcDriver).newInstance(); + Connection conn = driver.connect(jdbcString, new Properties()); + Statement s = conn.createStatement(); + s.execute("CREATE TABLE TXNS (" + + " TXN_ID bigint PRIMARY KEY," + + " TXN_STATE char(1) NOT NULL," + + " TXN_STARTED bigint NOT NULL," + + " TXN_LAST_HEARTBEAT bigint NOT NULL," + + " TXN_USER varchar(128) NOT NULL," + + " TXN_HOST varchar(128) NOT NULL)"); + + s.execute("CREATE TABLE TXN_COMPONENTS (" + + " TC_TXNID bigint REFERENCES TXNS (TXN_ID)," + + " TC_DATABASE varchar(128) NOT NULL," + + " TC_TABLE varchar(128)," + + " TC_PARTITION varchar(767))"); + s.execute("CREATE TABLE COMPLETED_TXN_COMPONENTS (" + + " CTC_TXNID bigint," + + " CTC_DATABASE varchar(128) NOT NULL," + + " CTC_TABLE varchar(128)," + + " CTC_PARTITION varchar(767))"); + s.execute("CREATE TABLE NEXT_TXN_ID (" + + " NTXN_NEXT bigint NOT NULL)"); + s.execute("INSERT INTO NEXT_TXN_ID VALUES(1)"); + s.execute("CREATE TABLE HIVE_LOCKS (" + + " HL_LOCK_EXT_ID bigint NOT NULL," + + " HL_LOCK_INT_ID bigint NOT NULL," + + " HL_TXNID bigint," + + " HL_DB varchar(128) NOT NULL," + + " HL_TABLE varchar(128)," + + " HL_PARTITION varchar(767)," + + " HL_LOCK_STATE char(1) NOT NULL," + + " HL_LOCK_TYPE char(1) NOT NULL," + + " HL_LAST_HEARTBEAT bigint NOT NULL," + + " HL_ACQUIRED_AT bigint," + + " HL_USER varchar(128) NOT NULL," + + " HL_HOST varchar(128) NOT NULL," + + " PRIMARY KEY(HL_LOCK_EXT_ID, HL_LOCK_INT_ID))"); + s.execute("CREATE INDEX HL_TXNID_INDEX ON HIVE_LOCKS (HL_TXNID)"); + + s.execute("CREATE TABLE NEXT_LOCK_ID (" + + " NL_NEXT bigint NOT NULL)"); + s.execute("INSERT INTO NEXT_LOCK_ID VALUES(1)"); + + s.execute("CREATE TABLE COMPACTION_QUEUE (" + + " CQ_ID bigint PRIMARY KEY," + + " CQ_DATABASE varchar(128) NOT NULL," + + " CQ_TABLE varchar(128) NOT NULL," + + " CQ_PARTITION varchar(767)," + + " CQ_STATE char(1) NOT NULL," + + " CQ_TYPE char(1) NOT NULL," + + " CQ_WORKER_ID varchar(128)," + + " CQ_START bigint," + + " CQ_RUN_AS varchar(128))"); + + s.execute("CREATE TABLE NEXT_COMPACTION_QUEUE_ID (NCQ_NEXT bigint NOT NULL)"); + s.execute("INSERT INTO NEXT_COMPACTION_QUEUE_ID VALUES(1)"); + + conn.commit(); + conn.close(); + } + + public static void cleanDb() throws Exception { + Driver driver = (Driver)Class.forName(jdbcDriver).newInstance(); + Connection conn = driver.connect(jdbcString, new Properties()); + Statement s = conn.createStatement(); + // We want to try these, whether they succeed or fail. + try { + s.execute("DROP INDEX HL_TXNID_INDEX"); + } catch (Exception e) { + System.err.println("Unable to drop index HL_TXNID_INDEX " + + e.getMessage()); + } + try { + s.execute("DROP TABLE TXN_COMPONENTS"); + } catch (Exception e) { + System.err.println("Unable to drop table TXN_COMPONENTS " + + e.getMessage()); + } + try { + s.execute("DROP TABLE COMPLETED_TXN_COMPONENTS"); + } catch (Exception e) { + System.err.println("Unable to drop table COMPLETED_TXN_COMPONENTS " + + e.getMessage()); + } + try { + s.execute("DROP TABLE TXNS"); + } catch (Exception e) { + System.err.println("Unable to drop table TXNS " + + e.getMessage()); + } + try { + s.execute("DROP TABLE NEXT_TXN_ID"); + } catch (Exception e) { + System.err.println("Unable to drop table NEXT_TXN_ID " + + e.getMessage()); + } + try { + s.execute("DROP TABLE HIVE_LOCKS"); + } catch (Exception e) { + System.err.println("Unable to drop table HIVE_LOCKS " + + e.getMessage()); + } + try { + s.execute("DROP TABLE NEXT_LOCK_ID"); + } catch (Exception e) { + } + try { + s.execute("DROP TABLE COMPACTION_QUEUE"); + } catch (Exception e) { + } + try { + s.execute("DROP TABLE NEXT_COMPACTION_QUEUE_ID"); + } catch (Exception e) { + } + conn.commit(); + conn.close(); + } + + /** + * A tool to count the number of partitions, tables, + * and databases locked by a particular lockId. + * @param lockId lock id to look for lock components + * @return number of components, or 0 if there is no lock + */ + public static int countLockComponents(long lockId) throws Exception { + Driver driver = (Driver)Class.forName(jdbcDriver).newInstance(); + Connection conn = driver.connect(jdbcString, new Properties()); + Statement s = conn.createStatement(); + ResultSet rs = s.executeQuery("select count(*) from hive_locks where " + + "hl_lock_ext_id = " + lockId); + if (!rs.next()) return 0; + int rc = rs.getInt(1); + conn.rollback(); + conn.close(); + return rc; + } + + public static int findNumCurrentLocks() throws Exception { + Driver driver = (Driver)Class.forName(jdbcDriver).newInstance(); + Connection conn = driver.connect(jdbcString, new Properties()); + Statement s = conn.createStatement(); + ResultSet rs = s.executeQuery("select count(*) from hive_locks"); + if (!rs.next()) return 0; + int rc = rs.getInt(1); + conn.rollback(); + conn.close(); + return rc; + } + +} diff --git metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java new file mode 100644 index 0000000..4441c2f --- /dev/null +++ metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java @@ -0,0 +1,1384 @@ +/** + * 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. + */ +package org.apache.hadoop.hive.metastore.txn; + +import com.jolbox.bonecp.BoneCP; +import com.jolbox.bonecp.BoneCPConfig; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; + +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.metastore.api.*; +import org.apache.hadoop.util.StringUtils; + +import java.sql.*; +import java.util.*; + +/** + * A handler to answer transaction related calls that come into the metastore + * server. + */ +public class TxnHandler { + // Compactor states + static final public String INITIATED_RESPONSE = "initiated"; + static final public String WORKING_RESPONSE = "working"; + static final public String CLEANING_RESPONSE = "ready for cleaning"; + + static final protected char INITIATED_STATE = 'i'; + static final protected char WORKING_STATE = 'w'; + static final protected char READY_FOR_CLEANING = 'r'; + + // Compactor types + static final protected char MAJOR_TYPE = 'a'; + static final protected char MINOR_TYPE = 'i'; + + // Transaction states + static final protected char TXN_ABORTED = 'a'; + static final protected char TXN_OPEN = 'o'; + + // Lock states + static final private char LOCK_ACQUIRED = 'a'; + static final private char LOCK_WAITING = 'w'; + + // Lock types + static final private char LOCK_EXCLUSIVE = 'e'; + static final private char LOCK_SHARED = 'r'; + static final private char LOCK_SEMI_SHARED = 'w'; + + static final private String CLASS_NAME = TxnHandler.class.getName(); + static final private Log LOG = LogFactory.getLog(CLASS_NAME); + + static private BoneCP connPool; + + // Transaction timeout, in milliseconds. + private long timeout; + protected HiveConf conf; + + public TxnHandler(HiveConf conf) { + this.conf = conf; + + checkQFileTestHack(); + + // Set up the JDBC connection pool + String connString = + HiveConf.getVar(conf, HiveConf.ConfVars.HIVE_TXN_JDBC_CONNECT_STRING); + try { + setupJdbcConnectionPool(connString); + } catch (SQLException e) { + String msg = "Unable to instantiate JDBC connection pooling, " + e.getMessage(); + LOG.error(msg); + throw new RuntimeException(e); + } + + timeout = HiveConf.getIntVar(conf, HiveConf.ConfVars.HIVE_TXN_TIMEOUT) * 1000; + buildJumpTable(); + } + + public GetOpenTxnsInfoResponse getOpenTxnsInfo() throws MetaException { + // We need to figure out the current transaction number and the list of + // open transactions. To avoid needing a transaction on the underlying + // database we'll look at the current transaction number first. If it + // subsequently shows up in the open list that's ok. + Connection dbConn = getDbConn(); + try { + Statement stmt = dbConn.createStatement(); + LOG.debug("Going to execute query "); + rs = stmt.executeQuery("select txn_id, txn_state, txn_user, txn_host from TXNS"); + while (rs.next()) { + char c = rs.getString(2).charAt(0); + TxnState state; + switch (c) { + case TXN_ABORTED: + state = TxnState.ABORTED; + break; + + case TXN_OPEN: + state = TxnState.OPEN; + break; + + default: + throw new MetaException("Unexpected transaction state " + c + + " found in txns table"); + } + txnInfo.add(new TxnInfo(rs.getLong(1), state, rs.getString(3), rs.getString(4))); + } + stmt.close(); + LOG.debug("Going to rollback"); + dbConn.rollback(); + return new GetOpenTxnsInfoResponse(hwm, txnInfo); + } catch (SQLException e) { + try { + LOG.debug("Going to rollback"); + dbConn.rollback(); + } catch (SQLException e1) { + } + throw new MetaException("Unable to select from transaction database, " + + StringUtils.stringifyException(e)); + } finally { + closeDbConn(dbConn); + } + } + + public GetOpenTxnsResponse getOpenTxns() throws MetaException{ + // We need to figure out the current transaction number and the list of + // open transactions. To avoid needing a transaction on the underlying + // database we'll look at the current transaction number first. If it + // subsequently shows up in the open list that's ok. + Connection dbConn = getDbConn(); + try { + Statement stmt = dbConn.createStatement(); + LOG.debug("Going to execute query "); + rs = stmt.executeQuery("select txn_id from TXNS"); + while (rs.next()) { + openList.add(rs.getLong(1)); + } + stmt.close(); + LOG.debug("Going to rollback"); + dbConn.rollback(); + return new GetOpenTxnsResponse(hwm, openList); + } catch (SQLException e) { + try { + LOG.debug("Going to rollback"); + dbConn.rollback(); + } catch (SQLException e1) { + } + throw new MetaException("Unable to select from transaction database, " + + StringUtils.stringifyException(e)); + } finally { + closeDbConn(dbConn); + } + } + + public OpenTxnsResponse openTxns(OpenTxnRequest rqst) throws MetaException { + int numTxns = rqst.getNum_txns(); + Connection dbConn = getDbConn(); + try { + // Make sure the user has not requested an insane amount of txns. + int maxTxns = HiveConf.getIntVar(conf, + HiveConf.ConfVars.HIVE_TXN_MAX_OPEN_BATCH); + if (numTxns > maxTxns) numTxns = maxTxns; + + Statement stmt = dbConn.createStatement(); + LOG.debug("Going to execute query "); + ResultSet rs = stmt.executeQuery("select nl_next from NEXT_LOCK_ID " + + "for update"); + if (!rs.next()) { + LOG.debug("Going to rollback"); + dbConn.rollback(); + throw new MetaException("Transaction tables not properly " + + "initialized, no record found in next_lock_id"); + } + long extLockId = rs.getLong(1); + String s = "update NEXT_LOCK_ID set nl_next = " + (extLockId + 1); + LOG.debug("Going to execute update <" + s + ">"); + stmt.executeUpdate(s); + long intLockId = 0; + for (LockComponent lc : rqst.getComponent()) { + intLockId++; + String dbName = lc.getDbname(); + String tblName = lc.getTablename(); + String partName = lc.getPartitionname(); + LockType lockType = lc.getType(); + char lockChar = 'z'; + switch (lockType) { + case EXCLUSIVE: lockChar = LOCK_EXCLUSIVE; break; + case SHARED_READ: lockChar = LOCK_SHARED; break; + case SHARED_WRITE: lockChar = LOCK_SEMI_SHARED; break; + } + long now = System.currentTimeMillis(); + s = "insert into HIVE_LOCKS " + + " (hl_lock_ext_id, hl_lock_int_id, hl_txnid, hl_db, hl_table, " + + "hl_partition, hl_lock_state, hl_lock_type, hl_last_heartbeat, hl_user, hl_host)" + + " values (" + extLockId + ", " + + + intLockId + "," + (txnid >= 0 ? txnid : "null") + ", '" + + dbName + "', " + (tblName == null ? "null" : "'" + tblName + "'" ) + + ", " + (partName == null ? "null" : "'" + partName + "'") + + ", '" + LOCK_WAITING + "', " + "'" + lockChar + "', " + now + ", '" + + rqst.getUser() + "', '" + rqst.getHostname() + "')"; + LOG.debug("Going to execute update <" + s + ">"); + stmt.executeUpdate(s); + } + LockResponse rsp = checkLock(dbConn, extLockId, txnid, wait); + if (!wait && rsp.getState() != LockState.ACQUIRED) { + LOG.debug("Lock not acquired, going to rollback"); + dbConn.rollback(); + rsp = new LockResponse(); + rsp.setState(LockState.NOT_ACQUIRED); + } + return rsp; + } catch (SQLException e) { + try { + LOG.debug("Going to rollback"); + dbConn.rollback(); + } catch (SQLException e1) { + } + throw new MetaException("Unable to connect to transaction database " + + StringUtils.stringifyException(e)); + } catch (NoSuchLockException e) { + // This should never happen, as we just added the lock id + throw new MetaException("Couldn't find a lock we just created!"); + } + } + + private LockResponse checkLock(Connection dbConn, + long extLockId, + long txnid, + boolean alwaysCommit) + throws NoSuchLockException, NoSuchTxnException, TxnAbortedException, MetaException { + List locksBeingChecked = getLockInfoFromLockId(dbConn, extLockId); + LockResponse response = new LockResponse(); + response.setLockid(extLockId); + + long now = System.currentTimeMillis(); + try { + LOG.debug("Setting savepoint"); + Savepoint save = dbConn.setSavepoint(); + Statement stmt = dbConn.createStatement(); + StringBuffer query = new StringBuffer("select hl_lock_ext_id, " + + "hl_lock_int_id, hl_db, hl_table, hl_partition, hl_lock_state, " + + "hl_lock_type from HIVE_LOCKS where hl_db in ("); + + Set strings = new HashSet(locksBeingChecked.size()); + for (LockInfo info : locksBeingChecked) { + strings.add(info.db); + } + boolean first = true; + for (String s : strings) { + if (first) first = false; + else query.append(", "); + query.append('\''); + query.append(s); + query.append('\''); + } + query.append(")"); + + // If any of the table requests are null, then I need to pull all the + // table locks for this db. + boolean sawNull = false; + strings.clear(); + for (LockInfo info : locksBeingChecked) { + if (info.table == null) { + sawNull = true; + break; + } else { + strings.add(info.table); + } + } + if (!sawNull) { + query.append(" and (hl_table is null or hl_table in("); + first = true; + for (String s : strings) { + if (first) first = false; + else query.append(", "); + query.append('\''); + query.append(s); + query.append('\''); + } + query.append("))"); + + // If any of the partition requests are null, then I need to pull all + // partition locks for this table. + sawNull = false; + strings.clear(); + for (LockInfo info : locksBeingChecked) { + if (info.partition == null) { + sawNull = true; + break; + } else { + strings.add(info.partition); + } + } + if (!sawNull) { + query.append(" and (hl_partition is null or hl_partition in("); + first = true; + for (String s : strings) { + if (first) first = false; + else query.append(", "); + query.append('\''); + query.append(s); + query.append('\''); + } + query.append("))"); + } + } + query.append(" for update"); + + LOG.debug("Going to execute query <" + query.toString() + ">"); + ResultSet rs = stmt.executeQuery(query.toString()); + SortedSet lockSet = new TreeSet(new LockInfoComparator()); + while (rs.next()) { + lockSet.add(new LockInfo(rs)); + } + // Turn the tree set into an array so we can move back and forth easily + // in it. + LockInfo[] locks = (LockInfo[])lockSet.toArray(new LockInfo[1]); + + for (LockInfo info : locksBeingChecked) { + // Find the lock record we're checking + int index = -1; + for (int i = 0; i < locks.length; i++) { + if (locks[i].equals(info)) { + index = i; + break; + } + } + + // If we didn't find the lock, then it must not be in the table + if (index == -1) { + LOG.debug("Going to rollback"); + dbConn.rollback(); + throw new MetaException("How did we get here, " + + "we heartbeated our lock before we started!"); + } + + + // If we've found it and it's already been marked acquired, + // then just look at the other locks. + if (locks[index].state == LockState.ACQUIRED) { + continue; + } + + // Look at everything in front of this lock to see if it should block + // it or not. + for (int i = index - 1; i >= 0; i--) { + // Check if we're operating on the same database, if not, move on + if (!locks[index].db.equals(locks[i].db)) { + continue; + } + + // If table is null on either of these, then they are claiming to + // lock the whole database and we need to check it. Otherwise, + // check if they are operating on the same table, if not, move on. + if (locks[index].table != null && locks[i].table != null + && !locks[index].table.equals(locks[i].table)) { + continue; + } + + // If partition is null on either of these, then they are claiming to + // lock the whole table and we need to check it. Otherwise, + // check if they are operating on the same partition, if not, move on. + if (locks[index].partition != null && locks[i].partition != null + && !locks[index].partition.equals(locks[i].partition)) { + continue; + } + + // We've found something that matches what we're trying to lock, + // so figure out if we can lock it too. + switch (jumpTable.get(locks[index].type).get(locks[i].type).get + (locks[i].state)) { + case ACQUIRE: + acquire(dbConn, stmt, extLockId, info.intLockId); + break; + case WAIT: + wait(dbConn, save); + if (alwaysCommit) { + // In the case where lockNoWait has been called we don't want to commit because + // it's going to roll everything back. In every other case we want to commit here. + LOG.debug("Going to commit"); + dbConn.commit(); + } + response.setState(LockState.WAITING); + return response; + case KEEP_LOOKING: + continue; + } + } + + // If we've arrived here it means there's nothing in the way of the + // lock, so acquire the lock. + acquire(dbConn, stmt, extLockId, info.intLockId); + } + + // We acquired all of the locks, so commit and return acquired. + LOG.debug("Going to commit"); + dbConn.commit(); + response.setState(LockState.ACQUIRED); + return response; + } catch (SQLException e) { + try { + LOG.debug("Going to rollback"); + dbConn.rollback(); + } catch (SQLException e1) { + } + throw new MetaException("Unable to connect to transaction database " + + StringUtils.stringifyException(e)); + } + } + + private void wait(Connection dbConn, Savepoint save) throws SQLException { + // Need to rollback because we did a select for update but we didn't + // actually update anything. Also, we may have locked some locks as + // acquired that we now want to not acquire. It's ok to rollback because + // once we see one wait, we're done, we won't look for more. + // Only rollback to savepoint because we want to commit our heartbeat + // changes. + LOG.debug("Going to rollback to savepoint"); + dbConn.rollback(save); + } + + private void acquire(Connection dbConn, Statement stmt, long extLockId, long intLockId) + throws SQLException, NoSuchLockException { + long now = System.currentTimeMillis(); + String s = "update HIVE_LOCKS set hl_lock_state = '" + LOCK_ACQUIRED + "', " + + "hl_last_heartbeat = " + now + ", hl_acquired_at = " + now + " where hl_lock_ext_id = " + + extLockId + " and hl_lock_int_id = " + intLockId; + LOG.debug("Going to execute update <" + s + ">"); + int rc = stmt.executeUpdate(s); + if (rc < 1) { + LOG.debug("Going to rollback"); + dbConn.rollback(); + throw new NoSuchLockException("No such lock: (" + extLockId + "," + + + intLockId + ")"); + } + // We update the database, but we don't commit because there may be other + // locks together with this, and we only want to acquire one if we can + // acquire all. + } + + // Heartbeats on the lock table. This does not call commit as it assumes + // someone downstream will. However, it does lock rows in the lock table. + private void heartbeatLock(Connection dbConn, long extLockId) + throws NoSuchLockException, MetaException { + // If the lock id is 0, then there are no locks in this heartbeat + if (extLockId == 0) return; + try { + Statement stmt = dbConn.createStatement(); + long now = System.currentTimeMillis(); + + String s = "update HIVE_LOCKS set hl_last_heartbeat = " + + now + " where hl_lock_ext_id = " + extLockId; + LOG.debug("Going to execute update <" + s + ">"); + int rc = stmt.executeUpdate(s); + if (rc < 1) { + LOG.debug("Going to rollback"); + dbConn.rollback(); + throw new NoSuchLockException("No such lock: " + extLockId); + } + } catch (SQLException e) { + try { + LOG.debug("Going to rollback"); + dbConn.rollback(); + } catch (SQLException e1) { + } + throw new MetaException("Unable to connect to transaction database" + + StringUtils.stringifyException(e)); + } + } + + // Heartbeats on the txn table. This does not call commit as it assumes + // someone downstream will. However, it does lock rows in the txn table. + private void heartbeatTxn(Connection dbConn, long txnid) + throws NoSuchTxnException, TxnAbortedException, MetaException { + // If the txnid is 0, then there are no transactions in this heartbeat + if (txnid == 0) return; + try { + Statement stmt = dbConn.createStatement(); + long now = System.currentTimeMillis(); + // We need to check whether this transaction is valid and open + String s = "select txn_state from TXNS where txn_id = " + + txnid + "for update"; + LOG.debug("Going to execute query <" + s + ">"); + ResultSet rs = stmt.executeQuery(s); + if (!rs.next()) { + LOG.debug("Going to rollback"); + dbConn.rollback(); + throw new NoSuchTxnException("No such transaction: " + txnid); + } + if (rs.getString(1).charAt(0) == TXN_ABORTED) { + LOG.debug("Going to rollback"); + dbConn.rollback(); + throw new TxnAbortedException("Transaction " + txnid + + " already aborted"); + } + s = "update TXNS set txn_last_heartbeat = " + now + + " where txn_id = " + txnid; + LOG.debug("Going to execute update <" + s + ">"); + stmt.executeUpdate(s); + + } catch (SQLException e) { + try { + LOG.debug("Going to rollback"); + dbConn.rollback(); + } catch (SQLException e1) { + } + throw new MetaException("Unable to connect to transaction database " + + StringUtils.stringifyException(e)); + } + } + + // NEVER call this function without first calling heartbeat(long, long) + private long getTxnIdFromLockId(Connection dbConn, long extLockId) + throws NoSuchLockException, MetaException { + try { + Statement stmt = dbConn.createStatement(); + String s = "select hl_txnid from HIVE_LOCKS where hl_lock_ext_id = " + + extLockId; + LOG.debug("Going to execute query <" + s + ">"); + ResultSet rs = stmt.executeQuery(s); + if (!rs.next()) { + throw new MetaException("This should never happen! We already " + + "checked the lock existed but now we can't find it!"); + } + long txnid = rs.getLong(1); + LOG.debug("Return txnid " + (rs.wasNull() ? -1 : txnid)); + return (rs.wasNull() ? -1 : txnid); + } catch (SQLException e) { + throw new MetaException("Unable to connect to transaction database " + + StringUtils.stringifyException(e)); + } + } + + // NEVER call this function without first calling heartbeat(long, long) + private List getLockInfoFromLockId(Connection dbConn, long extLockId) + throws NoSuchLockException, MetaException { + try { + Statement stmt = dbConn.createStatement(); + String s = "select hl_lock_ext_id, hl_lock_int_id, hl_db, hl_table, " + + "hl_partition, hl_lock_state, hl_lock_type from HIVE_LOCKS where " + + "hl_lock_ext_id = " + extLockId; + LOG.debug("Going to execute query <" + s + ">"); + ResultSet rs = stmt.executeQuery(s); + boolean sawAtLeastOne = false; + List ourLockInfo = new ArrayList(); + while (rs.next()) { + ourLockInfo.add(new LockInfo(rs)); + sawAtLeastOne = true; + } + if (!sawAtLeastOne) { + throw new MetaException("This should never happen! We already " + + "checked the lock existed but now we can't find it!"); + } + return ourLockInfo; + } catch (SQLException e) { + throw new MetaException("Unable to connect to transaction database " + + StringUtils.stringifyException(e)); + } + } + + // Clean time out locks from the database. This does a commit, + // and thus should be done before any calls to heartbeat that will leave + // open transactions. + private void timeOutLocks(Connection dbConn) throws MetaException { + try { + long now = System.currentTimeMillis(); + Statement stmt = dbConn.createStatement(); + // Remove any timed out locks from the table. + String s = "delete from HIVE_LOCKS where hl_last_heartbeat < " + + (now - timeout); + LOG.debug("Going to execute update <" + s + ">"); + stmt.executeUpdate(s); + LOG.debug("Going to commit"); + dbConn.commit(); + } catch (SQLException e) { + try { + LOG.debug("Going to rollback"); + dbConn.rollback(); + } catch (SQLException e1) { + } + throw new MetaException("Unable to connect to transaction database " + + StringUtils.stringifyException(e)); + } + } + + private synchronized void setupJdbcConnectionPool(String driverUrl) throws SQLException { + if (connPool != null) return; + + BoneCPConfig config = new BoneCPConfig(); + config.setJdbcUrl(driverUrl); + config.setMaxConnectionsPerPartition(10); + config.setPartitionCount(1); + connPool = new BoneCP(config); + } + + private synchronized void buildJumpTable() { + if (jumpTable != null) return; + + jumpTable = + new HashMap>>(3); + + // SR: Lock we are trying to acquire is shared read + Map> m = + new HashMap>(3); + jumpTable.put(LockType.SHARED_READ, m); + + // SR.SR: Lock we are examining is shared read + Map m2 = new HashMap(2); + m.put(LockType.SHARED_READ, m2); + + // SR.SR.acquired Lock we are examining is acquired; We can acquire + // because two shared reads can acquire together and there must be + // nothing in front of this one to prevent acquisition. + m2.put(LockState.ACQUIRED, LockAction.ACQUIRE); + + // SR.SR.wait Lock we are examining is waiting. In this case we keep + // looking, as it's possible that something in front is blocking it or + // that the other locker hasn't checked yet and he could lock as well. + m2.put(LockState.WAITING, LockAction.KEEP_LOOKING); + + // SR.SW: Lock we are examining is shared write + m2 = new HashMap(2); + m.put(LockType.SHARED_WRITE, m2); + + // SR.SW.acquired Lock we are examining is acquired; We can acquire + // because a read can share with a write, and there must be + // nothing in front of this one to prevent acquisition. + m2.put(LockState.ACQUIRED, LockAction.ACQUIRE); + + // SR.SW.wait Lock we are examining is waiting. In this case we keep + // looking, as it's possible that something in front is blocking it or + // that the other locker hasn't checked yet and he could lock as well or + // that something is blocking it that would not block a read. + m2.put(LockState.WAITING, LockAction.KEEP_LOOKING); + + // SR.E: Lock we are examining is exclusive + m2 = new HashMap(2); + m.put(LockType.EXCLUSIVE, m2); + + // No matter whether it has acquired or not, we cannot pass an exclusive. + m2.put(LockState.ACQUIRED, LockAction.WAIT); + m2.put(LockState.WAITING, LockAction.WAIT); + + // SW: Lock we are trying to acquire is shared write + m = new HashMap>(3); + jumpTable.put(LockType.SHARED_WRITE, m); + + // SW.SR: Lock we are examining is shared read + m2 = new HashMap(2); + m.put(LockType.SHARED_READ, m2); + + // SW.SR.acquired Lock we are examining is acquired; We need to keep + // looking, because there may or may not be another shared write in front + // that would block us. + m2.put(LockState.ACQUIRED, LockAction.KEEP_LOOKING); + + // SW.SR.wait Lock we are examining is waiting. In this case we keep + // looking, as it's possible that something in front is blocking it or + // that the other locker hasn't checked yet and he could lock as well. + m2.put(LockState.WAITING, LockAction.KEEP_LOOKING); + + // SW.SW: Lock we are examining is shared write + m2 = new HashMap(2); + m.put(LockType.SHARED_WRITE, m2); + + // Regardless of acquired or waiting, one shared write cannot pass another. + m2.put(LockState.ACQUIRED, LockAction.WAIT); + m2.put(LockState.WAITING, LockAction.WAIT); + + // SW.E: Lock we are examining is exclusive + m2 = new HashMap(2); + m.put(LockType.EXCLUSIVE, m2); + + // No matter whether it has acquired or not, we cannot pass an exclusive. + m2.put(LockState.ACQUIRED, LockAction.WAIT); + m2.put(LockState.WAITING, LockAction.WAIT); + + // E: Lock we are trying to acquire is exclusive + m = new HashMap>(3); + jumpTable.put(LockType.EXCLUSIVE, m); + + // E.SR: Lock we are examining is shared read + m2 = new HashMap(2); + m.put(LockType.SHARED_READ, m2); + + // Exclusives can never pass + m2.put(LockState.ACQUIRED, LockAction.WAIT); + m2.put(LockState.WAITING, LockAction.WAIT); + + // E.SW: Lock we are examining is shared write + m2 = new HashMap(2); + m.put(LockType.SHARED_WRITE, m2); + + // Exclusives can never pass + m2.put(LockState.ACQUIRED, LockAction.WAIT); + m2.put(LockState.WAITING, LockAction.WAIT); + + // E.E: Lock we are examining is exclusive + m2 = new HashMap(2); + m.put(LockType.EXCLUSIVE, m2); + + // No matter whether it has acquired or not, we cannot pass an exclusive. + m2.put(LockState.ACQUIRED, LockAction.WAIT); + m2.put(LockState.WAITING, LockAction.WAIT); + } +} diff --git metastore/src/test/org/apache/hadoop/hive/metastore/TestLockRequestBuilder.java metastore/src/test/org/apache/hadoop/hive/metastore/TestLockRequestBuilder.java new file mode 100644 index 0000000..0964826 --- /dev/null +++ metastore/src/test/org/apache/hadoop/hive/metastore/TestLockRequestBuilder.java @@ -0,0 +1,584 @@ +/** + * 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. + */ +package org.apache.hadoop.hive.metastore; + +import junit.framework.Assert; +import org.apache.hadoop.hive.metastore.api.LockComponent; +import org.apache.hadoop.hive.metastore.api.LockLevel; +import org.apache.hadoop.hive.metastore.api.LockRequest; +import org.apache.hadoop.hive.metastore.api.LockType; +import org.junit.Test; + +import java.net.InetAddress; +import java.util.List; + +/** + * Tests for LockRequestBuilder. + */ +public class TestLockRequestBuilder { + + // Test failure if user not set + @Test + public void noUser() { + LockRequestBuilder bldr = new LockRequestBuilder(); + LockComponent comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, "mydb"); + bldr.addLockComponent(comp); + comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, "mydb"); + comp.setTablename("mytable"); + bldr.addLockComponent(comp); + boolean caughtException = false; + try { + LockRequest req = bldr.build(); + } catch (RuntimeException e) { + Assert.assertEquals("Cannot build a lock without giving a user", e.getMessage()); + caughtException = true; + } + Assert.assertTrue(caughtException); + } + + // Test that database and table don't coalesce. + @Test + public void testDbTable() throws Exception { + LockRequestBuilder bldr = new LockRequestBuilder(); + LockComponent comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, "mydb"); + bldr.addLockComponent(comp).setUser("fred"); + comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, "mydb"); + comp.setTablename("mytable"); + bldr.addLockComponent(comp); + LockRequest req = bldr.build(); + List locks = req.getComponent(); + Assert.assertEquals(2, locks.size()); + Assert.assertEquals("fred", req.getUser()); + Assert.assertEquals(InetAddress.getLocalHost().getHostName(), req.getHostname()); + } + + // Test that database and table don't coalesce. + @Test + public void testTablePartition() { + LockRequestBuilder bldr = new LockRequestBuilder(); + LockComponent comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, "mydb"); + comp.setTablename("mytable"); + bldr.addLockComponent(comp).setUser(null); + comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, "mydb"); + comp.setTablename("mytable"); + comp.setPartitionname("mypartition"); + bldr.addLockComponent(comp); + LockRequest req = bldr.build(); + List locks = req.getComponent(); + Assert.assertEquals(2, locks.size()); + Assert.assertEquals("unknown", req.getUser()); + } + + // Test that 2 separate databases don't coalesce. + @Test + public void testTwoSeparateDbs() { + LockRequestBuilder bldr = new LockRequestBuilder(); + LockComponent comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, "mydb"); + bldr.addLockComponent(comp).setUser("fred"); + comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, "yourdb"); + bldr.addLockComponent(comp); + LockRequest req = bldr.build(); + List locks = req.getComponent(); + Assert.assertEquals(2, locks.size()); + } + + // Test that 2 exclusive db locks coalesce to one + @Test + public void testExExDb() { + LockRequestBuilder bldr = new LockRequestBuilder(); + LockComponent comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, "mydb"); + bldr.addLockComponent(comp); + comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, "mydb"); + bldr.addLockComponent(comp).setUser("fred"); + LockRequest req = bldr.build(); + List locks = req.getComponent(); + Assert.assertEquals(1, locks.size()); + } + + // Test that existing exclusive db with new shared_write coalesces to + // exclusive + @Test + public void testExSWDb() { + LockRequestBuilder bldr = new LockRequestBuilder(); + LockComponent comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, "mydb"); + bldr.addLockComponent(comp).setUser("fred"); + comp = new LockComponent(LockType.SHARED_WRITE, LockLevel.DB, "mydb"); + bldr.addLockComponent(comp); + LockRequest req = bldr.build(); + List locks = req.getComponent(); + Assert.assertEquals(1, locks.size()); + Assert.assertEquals(LockType.EXCLUSIVE, locks.get(0).getType()); + } + + // Test that existing exclusive db with new shared_read coalesces to + // exclusive + @Test + public void testExSRDb() { + LockRequestBuilder bldr = new LockRequestBuilder(); + LockComponent comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, "mydb"); + bldr.addLockComponent(comp).setUser("fred"); + comp = new LockComponent(LockType.SHARED_READ, LockLevel.DB, "mydb"); + bldr.addLockComponent(comp); + LockRequest req = bldr.build(); + List locks = req.getComponent(); + Assert.assertEquals(1, locks.size()); + Assert.assertEquals(LockType.EXCLUSIVE, locks.get(0).getType()); + } + + // Test that existing shared_write db with new exclusive coalesces to + // exclusive + @Test + public void testSWExDb() { + LockRequestBuilder bldr = new LockRequestBuilder(); + LockComponent comp = new LockComponent(LockType.SHARED_WRITE, LockLevel.DB, "mydb"); + bldr.addLockComponent(comp); + comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, "mydb"); + bldr.addLockComponent(comp).setUser("fred"); + LockRequest req = bldr.build(); + List locks = req.getComponent(); + Assert.assertEquals(1, locks.size()); + Assert.assertEquals(LockType.EXCLUSIVE, locks.get(0).getType()); + } + + // Test that existing shared_write db with new shared_write coalesces to + // shared_write + @Test + public void testSWSWDb() { + LockRequestBuilder bldr = new LockRequestBuilder(); + LockComponent comp = new LockComponent(LockType.SHARED_WRITE, LockLevel.DB, "mydb"); + bldr.addLockComponent(comp); + comp = new LockComponent(LockType.SHARED_WRITE, LockLevel.DB, "mydb"); + bldr.addLockComponent(comp).setUser("fred"); + LockRequest req = bldr.build(); + List locks = req.getComponent(); + Assert.assertEquals(1, locks.size()); + Assert.assertEquals(LockType.SHARED_WRITE, locks.get(0).getType()); + } + + // Test that existing shared_write db with new shared_read coalesces to + // shared_write + @Test + public void testSWSRDb() { + LockRequestBuilder bldr = new LockRequestBuilder(); + LockComponent comp = new LockComponent(LockType.SHARED_WRITE, LockLevel.DB, "mydb"); + bldr.addLockComponent(comp); + comp = new LockComponent(LockType.SHARED_READ, LockLevel.DB, "mydb"); + bldr.addLockComponent(comp).setUser("fred"); + LockRequest req = bldr.build(); + List locks = req.getComponent(); + Assert.assertEquals(1, locks.size()); + Assert.assertEquals(LockType.SHARED_WRITE, locks.get(0).getType()); + } + + // Test that existing shared_read db with new exclusive coalesces to + // exclusive + @Test + public void testSRExDb() { + LockRequestBuilder bldr = new LockRequestBuilder(); + LockComponent comp = new LockComponent(LockType.SHARED_READ, LockLevel.DB, "mydb"); + bldr.addLockComponent(comp); + comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, "mydb"); + bldr.addLockComponent(comp).setUser("fred"); + LockRequest req = bldr.build(); + List locks = req.getComponent(); + Assert.assertEquals(1, locks.size()); + Assert.assertEquals(LockType.EXCLUSIVE, locks.get(0).getType()); + } + + // Test that existing shared_read db with new shared_write coalesces to + // shared_write + @Test + public void testSRSWDb() { + LockRequestBuilder bldr = new LockRequestBuilder(); + LockComponent comp = new LockComponent(LockType.SHARED_READ, LockLevel.DB, "mydb"); + bldr.addLockComponent(comp); + comp = new LockComponent(LockType.SHARED_WRITE, LockLevel.DB, "mydb"); + bldr.addLockComponent(comp).setUser("fred"); + LockRequest req = bldr.build(); + List locks = req.getComponent(); + Assert.assertEquals(1, locks.size()); + Assert.assertEquals(LockType.SHARED_WRITE, locks.get(0).getType()); + } + + // Test that existing shared_read db with new shared_read coalesces to + // shared_read + @Test + public void testSRSRDb() { + LockRequestBuilder bldr = new LockRequestBuilder(); + LockComponent comp = new LockComponent(LockType.SHARED_READ, LockLevel.DB, "mydb"); + bldr.addLockComponent(comp); + comp = new LockComponent(LockType.SHARED_READ, LockLevel.DB, "mydb"); + bldr.addLockComponent(comp).setUser("fred"); + LockRequest req = bldr.build(); + List locks = req.getComponent(); + Assert.assertEquals(1, locks.size()); + Assert.assertEquals(LockType.SHARED_READ, locks.get(0).getType()); + } + + // Test that 2 separate tables don't coalesce. + @Test + public void testTwoSeparateTables() { + LockRequestBuilder bldr = new LockRequestBuilder(); + LockComponent comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, "mydb"); + comp.setTablename("mytable"); + bldr.addLockComponent(comp); + comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, "mydb"); + comp.setTablename("yourtable"); + bldr.addLockComponent(comp).setUser("fred"); + LockRequest req = bldr.build(); + List locks = req.getComponent(); + Assert.assertEquals(2, locks.size()); + } + + // Test that 2 exclusive table locks coalesce to one + @Test + public void testExExTable() { + LockRequestBuilder bldr = new LockRequestBuilder(); + LockComponent comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, "mydb"); + comp.setTablename("mytable"); + bldr.addLockComponent(comp); + comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, "mydb"); + comp.setTablename("mytable"); + bldr.addLockComponent(comp).setUser("fred"); + LockRequest req = bldr.build(); + List locks = req.getComponent(); + Assert.assertEquals(1, locks.size()); + } + + // Test that existing exclusive table with new shared_write coalesces to + // exclusive + @Test + public void testExSWTable() { + LockRequestBuilder bldr = new LockRequestBuilder(); + LockComponent comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, "mydb"); + comp.setTablename("mytable"); + bldr.addLockComponent(comp); + comp = new LockComponent(LockType.SHARED_WRITE, LockLevel.DB, "mydb"); + comp.setTablename("mytable"); + bldr.addLockComponent(comp).setUser("fred"); + LockRequest req = bldr.build(); + List locks = req.getComponent(); + Assert.assertEquals(1, locks.size()); + Assert.assertEquals(LockType.EXCLUSIVE, locks.get(0).getType()); + } + + // Test that existing exclusive table with new shared_read coalesces to + // exclusive + @Test + public void testExSRTable() { + LockRequestBuilder bldr = new LockRequestBuilder(); + LockComponent comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, "mydb"); + comp.setTablename("mytable"); + bldr.addLockComponent(comp); + comp = new LockComponent(LockType.SHARED_READ, LockLevel.DB, "mydb"); + comp.setTablename("mytable"); + bldr.addLockComponent(comp).setUser("fred"); + LockRequest req = bldr.build(); + List locks = req.getComponent(); + Assert.assertEquals(1, locks.size()); + Assert.assertEquals(LockType.EXCLUSIVE, locks.get(0).getType()); + } + + // Test that existing shared_write table with new exclusive coalesces to + // exclusive + @Test + public void testSWExTable() { + LockRequestBuilder bldr = new LockRequestBuilder(); + LockComponent comp = new LockComponent(LockType.SHARED_WRITE, LockLevel.DB, "mydb"); + comp.setTablename("mytable"); + bldr.addLockComponent(comp); + comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, "mydb"); + comp.setTablename("mytable"); + bldr.addLockComponent(comp).setUser("fred"); + LockRequest req = bldr.build(); + List locks = req.getComponent(); + Assert.assertEquals(1, locks.size()); + Assert.assertEquals(LockType.EXCLUSIVE, locks.get(0).getType()); + } + + // Test that existing shared_write table with new shared_write coalesces to + // shared_write + @Test + public void testSWSWTable() { + LockRequestBuilder bldr = new LockRequestBuilder(); + LockComponent comp = new LockComponent(LockType.SHARED_WRITE, LockLevel.DB, "mydb"); + comp.setTablename("mytable"); + bldr.addLockComponent(comp); + comp = new LockComponent(LockType.SHARED_WRITE, LockLevel.DB, "mydb"); + comp.setTablename("mytable"); + bldr.addLockComponent(comp).setUser("fred"); + LockRequest req = bldr.build(); + List locks = req.getComponent(); + Assert.assertEquals(1, locks.size()); + Assert.assertEquals(LockType.SHARED_WRITE, locks.get(0).getType()); + } + + // Test that existing shared_write table with new shared_read coalesces to + // shared_write + @Test + public void testSWSRTable() { + LockRequestBuilder bldr = new LockRequestBuilder(); + LockComponent comp = new LockComponent(LockType.SHARED_WRITE, LockLevel.DB, "mydb"); + comp.setTablename("mytable"); + bldr.addLockComponent(comp); + comp = new LockComponent(LockType.SHARED_READ, LockLevel.DB, "mydb"); + comp.setTablename("mytable"); + bldr.addLockComponent(comp).setUser("fred"); + LockRequest req = bldr.build(); + List locks = req.getComponent(); + Assert.assertEquals(1, locks.size()); + Assert.assertEquals(LockType.SHARED_WRITE, locks.get(0).getType()); + } + + // Test that existing shared_read table with new exclusive coalesces to + // exclusive + @Test + public void testSRExTable() { + LockRequestBuilder bldr = new LockRequestBuilder(); + LockComponent comp = new LockComponent(LockType.SHARED_READ, LockLevel.DB, "mydb"); + comp.setTablename("mytable"); + bldr.addLockComponent(comp); + comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, "mydb"); + comp.setTablename("mytable"); + bldr.addLockComponent(comp).setUser("fred"); + LockRequest req = bldr.build(); + List locks = req.getComponent(); + Assert.assertEquals(1, locks.size()); + Assert.assertEquals(LockType.EXCLUSIVE, locks.get(0).getType()); + } + + // Test that existing shared_read table with new shared_write coalesces to + // shared_write + @Test + public void testSRSWTable() { + LockRequestBuilder bldr = new LockRequestBuilder(); + LockComponent comp = new LockComponent(LockType.SHARED_READ, LockLevel.DB, "mydb"); + comp.setTablename("mytable"); + bldr.addLockComponent(comp); + comp = new LockComponent(LockType.SHARED_WRITE, LockLevel.DB, "mydb"); + comp.setTablename("mytable"); + bldr.addLockComponent(comp).setUser("fred"); + LockRequest req = bldr.build(); + List locks = req.getComponent(); + Assert.assertEquals(1, locks.size()); + Assert.assertEquals(LockType.SHARED_WRITE, locks.get(0).getType()); + } + + // Test that existing shared_read table with new shared_read coalesces to + // shared_read + @Test + public void testSRSRTable() { + LockRequestBuilder bldr = new LockRequestBuilder(); + LockComponent comp = new LockComponent(LockType.SHARED_READ, LockLevel.DB, "mydb"); + comp.setTablename("mytable"); + bldr.addLockComponent(comp); + comp = new LockComponent(LockType.SHARED_READ, LockLevel.DB, "mydb"); + comp.setTablename("mytable"); + bldr.addLockComponent(comp).setUser("fred"); + LockRequest req = bldr.build(); + List locks = req.getComponent(); + Assert.assertEquals(1, locks.size()); + Assert.assertEquals(LockType.SHARED_READ, locks.get(0).getType()); + } + + // Test that 2 separate partitions don't coalesce. + @Test + public void testTwoSeparatePartitions() { + LockRequestBuilder bldr = new LockRequestBuilder(); + LockComponent comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, "mydb"); + comp.setTablename("mytable"); + comp.setPartitionname("mypart"); + bldr.addLockComponent(comp); + comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, "mydb"); + comp.setTablename("mytable"); + comp.setPartitionname("yourpart"); + bldr.addLockComponent(comp).setUser("fred"); + LockRequest req = bldr.build(); + List locks = req.getComponent(); + Assert.assertEquals(2, locks.size()); + } + + // Test that 2 exclusive partition locks coalesce to one + @Test + public void testExExPart() { + LockRequestBuilder bldr = new LockRequestBuilder(); + LockComponent comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, "mydb"); + comp.setTablename("mytable"); + comp.setPartitionname("mypart"); + bldr.addLockComponent(comp); + comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, "mydb"); + comp.setTablename("mytable"); + comp.setPartitionname("mypart"); + bldr.addLockComponent(comp).setUser("fred"); + LockRequest req = bldr.build(); + List locks = req.getComponent(); + Assert.assertEquals(1, locks.size()); + } + + // Test that existing exclusive partition with new shared_write coalesces to + // exclusive + @Test + public void testExSWPart() { + LockRequestBuilder bldr = new LockRequestBuilder(); + LockComponent comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, "mydb"); + comp.setTablename("mytable"); + comp.setPartitionname("mypart"); + bldr.addLockComponent(comp); + comp = new LockComponent(LockType.SHARED_WRITE, LockLevel.DB, "mydb"); + comp.setTablename("mytable"); + comp.setPartitionname("mypart"); + bldr.addLockComponent(comp).setUser("fred"); + LockRequest req = bldr.build(); + List locks = req.getComponent(); + Assert.assertEquals(1, locks.size()); + Assert.assertEquals(LockType.EXCLUSIVE, locks.get(0).getType()); + } + + // Test that existing exclusive partition with new shared_read coalesces to + // exclusive + @Test + public void testExSRPart() { + LockRequestBuilder bldr = new LockRequestBuilder(); + LockComponent comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, "mydb"); + comp.setTablename("mytable"); + comp.setPartitionname("mypart"); + bldr.addLockComponent(comp); + comp = new LockComponent(LockType.SHARED_READ, LockLevel.DB, "mydb"); + comp.setTablename("mytable"); + comp.setPartitionname("mypart"); + bldr.addLockComponent(comp).setUser("fred"); + LockRequest req = bldr.build(); + List locks = req.getComponent(); + Assert.assertEquals(1, locks.size()); + Assert.assertEquals(LockType.EXCLUSIVE, locks.get(0).getType()); + } + + // Test that existing shared_write partition with new exclusive coalesces to + // exclusive + @Test + public void testSWExPart() { + LockRequestBuilder bldr = new LockRequestBuilder(); + LockComponent comp = new LockComponent(LockType.SHARED_WRITE, LockLevel.DB, "mydb"); + comp.setTablename("mytable"); + comp.setPartitionname("mypart"); + bldr.addLockComponent(comp); + comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, "mydb"); + comp.setTablename("mytable"); + comp.setPartitionname("mypart"); + bldr.addLockComponent(comp).setUser("fred"); + LockRequest req = bldr.build(); + List locks = req.getComponent(); + Assert.assertEquals(1, locks.size()); + Assert.assertEquals(LockType.EXCLUSIVE, locks.get(0).getType()); + } + + // Test that existing shared_write partition with new shared_write coalesces to + // shared_write + @Test + public void testSWSWPart() { + LockRequestBuilder bldr = new LockRequestBuilder(); + LockComponent comp = new LockComponent(LockType.SHARED_WRITE, LockLevel.DB, "mydb"); + comp.setTablename("mytable"); + comp.setPartitionname("mypart"); + bldr.addLockComponent(comp); + comp = new LockComponent(LockType.SHARED_WRITE, LockLevel.DB, "mydb"); + comp.setTablename("mytable"); + comp.setPartitionname("mypart"); + bldr.addLockComponent(comp).setUser("fred"); + LockRequest req = bldr.build(); + List locks = req.getComponent(); + Assert.assertEquals(1, locks.size()); + Assert.assertEquals(LockType.SHARED_WRITE, locks.get(0).getType()); + } + + // Test that existing shared_write partition with new shared_read coalesces to + // shared_write + @Test + public void testSWSRPart() { + LockRequestBuilder bldr = new LockRequestBuilder(); + LockComponent comp = new LockComponent(LockType.SHARED_WRITE, LockLevel.DB, "mydb"); + comp.setTablename("mytable"); + comp.setPartitionname("mypart"); + bldr.addLockComponent(comp); + comp = new LockComponent(LockType.SHARED_READ, LockLevel.DB, "mydb"); + comp.setTablename("mytable"); + comp.setPartitionname("mypart"); + bldr.addLockComponent(comp).setUser("fred"); + LockRequest req = bldr.build(); + List locks = req.getComponent(); + Assert.assertEquals(1, locks.size()); + Assert.assertEquals(LockType.SHARED_WRITE, locks.get(0).getType()); + } + + // Test that existing shared_read partition with new exclusive coalesces to + // exclusive + @Test + public void testSRExPart() { + LockRequestBuilder bldr = new LockRequestBuilder(); + LockComponent comp = new LockComponent(LockType.SHARED_READ, LockLevel.DB, "mydb"); + comp.setTablename("mytable"); + comp.setPartitionname("mypart"); + bldr.addLockComponent(comp); + comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, "mydb"); + comp.setTablename("mytable"); + comp.setPartitionname("mypart"); + bldr.addLockComponent(comp).setUser("fred"); + LockRequest req = bldr.build(); + List locks = req.getComponent(); + Assert.assertEquals(1, locks.size()); + Assert.assertEquals(LockType.EXCLUSIVE, locks.get(0).getType()); + } + + // Test that existing shared_read partition with new shared_write coalesces to + // shared_write + @Test + public void testSRSWPart() { + LockRequestBuilder bldr = new LockRequestBuilder(); + LockComponent comp = new LockComponent(LockType.SHARED_READ, LockLevel.DB, "mydb"); + comp.setTablename("mytable"); + comp.setPartitionname("mypart"); + bldr.addLockComponent(comp); + comp = new LockComponent(LockType.SHARED_WRITE, LockLevel.DB, "mydb"); + comp.setTablename("mytable"); + comp.setPartitionname("mypart"); + bldr.addLockComponent(comp).setUser("fred"); + LockRequest req = bldr.build(); + List locks = req.getComponent(); + Assert.assertEquals(1, locks.size()); + Assert.assertEquals(LockType.SHARED_WRITE, locks.get(0).getType()); + } + + // Test that existing shared_read partition with new shared_read coalesces to + // shared_read + @Test + public void testSRSRPart() { + LockRequestBuilder bldr = new LockRequestBuilder(); + LockComponent comp = new LockComponent(LockType.SHARED_READ, LockLevel.DB, "mydb"); + comp.setTablename("mytable"); + comp.setPartitionname("mypart"); + bldr.addLockComponent(comp); + comp = new LockComponent(LockType.SHARED_READ, LockLevel.DB, "mydb"); + comp.setTablename("mytable"); + comp.setPartitionname("mypart"); + bldr.addLockComponent(comp).setUser("fred"); + LockRequest req = bldr.build(); + List locks = req.getComponent(); + Assert.assertEquals(1, locks.size()); + Assert.assertEquals(LockType.SHARED_READ, locks.get(0).getType()); + } +} diff --git metastore/src/test/org/apache/hadoop/hive/metastore/txn/TestCompactionTxnHandler.java metastore/src/test/org/apache/hadoop/hive/metastore/txn/TestCompactionTxnHandler.java new file mode 100644 index 0000000..a15a210 --- /dev/null +++ metastore/src/test/org/apache/hadoop/hive/metastore/txn/TestCompactionTxnHandler.java @@ -0,0 +1,438 @@ +/** + * 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. + */ +package org.apache.hadoop.hive.metastore.txn; + +import junit.framework.Assert; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.metastore.IMetaStoreClient; +import org.apache.hadoop.hive.metastore.api.*; +import org.apache.log4j.Level; +import org.apache.log4j.LogManager; +import org.junit.After; +import org.junit.Before; +import org.junit.Ignore; +import org.junit.Test; + +import java.util.ArrayList; +import java.util.List; + +import static junit.framework.Assert.*; + +/** + * Tests for TxnHandler. + */ +public class TestCompactionTxnHandler { + + private HiveConf conf = new HiveConf(); + private CompactionTxnHandler txnHandler; + + public TestCompactionTxnHandler() throws Exception { + TxnDbUtil.setConfValues(conf); + LogManager.getLogger(TxnHandler.class.getName()).setLevel(Level.DEBUG); + tearDown(); + } + + @Test + public void testFindNextToCompact() throws Exception { + CompactionRequest rqst = new CompactionRequest("foo", "bar", CompactionType.MINOR); + rqst.setPartitionname("ds=today"); + txnHandler.compact(rqst); + long now = System.currentTimeMillis(); + CompactionInfo ci = txnHandler.findNextToCompact("fred"); + assertNotNull(ci); + assertEquals("foo", ci.dbname); + assertEquals("bar", ci.tableName); + assertEquals("ds=today", ci.partName); + assertEquals(CompactionType.MINOR, ci.type); + assertNull(ci.runAs); + assertNull(txnHandler.findNextToCompact("fred")); + + txnHandler.setRunAs(ci.id, "bob"); + + ShowCompactResponse rsp = txnHandler.showCompact(new ShowCompactRequest()); + List compacts = rsp.getCompacts(); + assertEquals(1, compacts.size()); + ShowCompactResponseElement c = compacts.get(0); + assertEquals("foo", c.getDbname()); + assertEquals("bar", c.getTablename()); + assertEquals("ds=today", c.getPartitionname()); + assertEquals(CompactionType.MINOR, c.getType()); + assertEquals("working", c.getState()); + assertTrue(c.getStart() - 5000 < now && c.getStart() + 5000 > now); + assertEquals("fred", c.getWorkerid()); + assertEquals("bob", c.getRunAs()); + } + + @Test + public void testFindNextToCompact2() throws Exception { + CompactionRequest rqst = new CompactionRequest("foo", "bar", CompactionType.MINOR); + rqst.setPartitionname("ds=today"); + txnHandler.compact(rqst); + + rqst = new CompactionRequest("foo", "bar", CompactionType.MINOR); + rqst.setPartitionname("ds=yesterday"); + txnHandler.compact(rqst); + + long now = System.currentTimeMillis(); + boolean expectToday = false; + CompactionInfo ci = txnHandler.findNextToCompact("fred"); + assertNotNull(ci); + assertEquals("foo", ci.dbname); + assertEquals("bar", ci.tableName); + if ("ds=today".equals(ci.partName)) expectToday = false; + else if ("ds=yesterday".equals(ci.partName)) expectToday = true; + else fail("partition name should have been today or yesterday but was " + ci.partName); + assertEquals(CompactionType.MINOR, ci.type); + + ci = txnHandler.findNextToCompact("fred"); + assertNotNull(ci); + assertEquals("foo", ci.dbname); + assertEquals("bar", ci.tableName); + if (expectToday) assertEquals("ds=today", ci.partName); + else assertEquals("ds=yesterday", ci.partName); + assertEquals(CompactionType.MINOR, ci.type); + + assertNull(txnHandler.findNextToCompact("fred")); + + ShowCompactResponse rsp = txnHandler.showCompact(new ShowCompactRequest()); + List compacts = rsp.getCompacts(); + assertEquals(2, compacts.size()); + for (ShowCompactResponseElement e : compacts) { + assertEquals("working", e.getState()); + assertTrue(e.getStart() - 5000 < now && e.getStart() + 5000 > now); + assertEquals("fred", e.getWorkerid()); + } + } + + @Test + public void testFindNextToCompactNothingToCompact() throws Exception { + assertNull(txnHandler.findNextToCompact("fred")); + } + + @Test + public void testMarkCompacted() throws Exception { + CompactionRequest rqst = new CompactionRequest("foo", "bar", CompactionType.MINOR); + rqst.setPartitionname("ds=today"); + txnHandler.compact(rqst); + CompactionInfo ci = txnHandler.findNextToCompact("fred"); + assertNotNull(ci); + + txnHandler.markCompacted(ci); + assertNull(txnHandler.findNextToCompact("fred")); + + + + ShowCompactResponse rsp = txnHandler.showCompact(new ShowCompactRequest()); + List compacts = rsp.getCompacts(); + assertEquals(1, compacts.size()); + ShowCompactResponseElement c = compacts.get(0); + assertEquals("foo", c.getDbname()); + assertEquals("bar", c.getTablename()); + assertEquals("ds=today", c.getPartitionname()); + assertEquals(CompactionType.MINOR, c.getType()); + assertEquals("ready for cleaning", c.getState()); + assertNull(c.getWorkerid()); + } + + @Test + public void testFindNextToClean() throws Exception { + CompactionRequest rqst = new CompactionRequest("foo", "bar", CompactionType.MINOR); + rqst.setPartitionname("ds=today"); + txnHandler.compact(rqst); + assertEquals(0, txnHandler.findReadyToClean().size()); + CompactionInfo ci = txnHandler.findNextToCompact("fred"); + assertNotNull(ci); + + assertEquals(0, txnHandler.findReadyToClean().size()); + txnHandler.markCompacted(ci); + assertNull(txnHandler.findNextToCompact("fred")); + + List toClean = txnHandler.findReadyToClean(); + assertEquals(1, toClean.size()); + assertNull(txnHandler.findNextToCompact("fred")); + + ShowCompactResponse rsp = txnHandler.showCompact(new ShowCompactRequest()); + List compacts = rsp.getCompacts(); + assertEquals(1, compacts.size()); + ShowCompactResponseElement c = compacts.get(0); + assertEquals("foo", c.getDbname()); + assertEquals("bar", c.getTablename()); + assertEquals("ds=today", c.getPartitionname()); + assertEquals(CompactionType.MINOR, c.getType()); + assertEquals("ready for cleaning", c.getState()); + assertNull(c.getWorkerid()); + } + + @Test + public void testMarkCleaned() throws Exception { + CompactionRequest rqst = new CompactionRequest("foo", "bar", CompactionType.MINOR); + rqst.setPartitionname("ds=today"); + txnHandler.compact(rqst); + assertEquals(0, txnHandler.findReadyToClean().size()); + CompactionInfo ci = txnHandler.findNextToCompact("fred"); + assertNotNull(ci); + + assertEquals(0, txnHandler.findReadyToClean().size()); + txnHandler.markCompacted(ci); + assertNull(txnHandler.findNextToCompact("fred")); + + List toClean = txnHandler.findReadyToClean(); + assertEquals(1, toClean.size()); + assertNull(txnHandler.findNextToCompact("fred")); + txnHandler.markCleaned(ci); + assertNull(txnHandler.findNextToCompact("fred")); + assertEquals(0, txnHandler.findReadyToClean().size()); + + ShowCompactResponse rsp = txnHandler.showCompact(new ShowCompactRequest()); + assertNull(rsp.getCompacts()); + } + + @Test + public void testRevokeFromLocalWorkers() throws Exception { + CompactionRequest rqst = new CompactionRequest("foo", "bar", CompactionType.MINOR); + txnHandler.compact(rqst); + rqst = new CompactionRequest("foo", "baz", CompactionType.MINOR); + txnHandler.compact(rqst); + rqst = new CompactionRequest("foo", "bazzoo", CompactionType.MINOR); + txnHandler.compact(rqst); + assertNotNull(txnHandler.findNextToCompact("fred-193892")); + assertNotNull(txnHandler.findNextToCompact("bob-193892")); + assertNotNull(txnHandler.findNextToCompact("fred-193893")); + txnHandler.revokeFromLocalWorkers("fred"); + + ShowCompactResponse rsp = txnHandler.showCompact(new ShowCompactRequest()); + List compacts = rsp.getCompacts(); + assertEquals(3, compacts.size()); + boolean sawWorkingBob = false; + int initiatedCount = 0; + for (ShowCompactResponseElement c : compacts) { + if (c.getState().equals("working")) { + assertEquals("bob-193892", c.getWorkerid()); + sawWorkingBob = true; + } else if (c.getState().equals("initiated")) { + initiatedCount++; + } else { + fail("Unexpected state"); + } + } + assertTrue(sawWorkingBob); + assertEquals(2, initiatedCount); + } + + @Test + public void testRevokeTimedOutWorkers() throws Exception { + CompactionRequest rqst = new CompactionRequest("foo", "bar", CompactionType.MINOR); + txnHandler.compact(rqst); + rqst = new CompactionRequest("foo", "baz", CompactionType.MINOR); + txnHandler.compact(rqst); + + assertNotNull(txnHandler.findNextToCompact("fred-193892")); + Thread.sleep(200); + assertNotNull(txnHandler.findNextToCompact("fred-193892")); + txnHandler.revokeTimedoutWorkers(100); + + ShowCompactResponse rsp = txnHandler.showCompact(new ShowCompactRequest()); + List compacts = rsp.getCompacts(); + assertEquals(2, compacts.size()); + boolean sawWorking = false, sawInitiated = false; + for (ShowCompactResponseElement c : compacts) { + if (c.getState().equals("working")) sawWorking = true; + else if (c.getState().equals("initiated")) sawInitiated = true; + else fail("Unexpected state"); + } + assertTrue(sawWorking); + assertTrue(sawInitiated); + } + + @Test + public void testLockNoWait() throws Exception { + // Test that we can acquire the lock alone + LockComponent comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, + "mydb"); + comp.setTablename("mytable"); + comp.setPartitionname("mypartition"); + List components = new ArrayList(1); + components.add(comp); + LockRequest req = new LockRequest(components, "me", "localhost"); + LockResponse res = txnHandler.lockNoWait(req); + assertTrue(res.getState() == LockState.ACQUIRED); + txnHandler.unlock(new UnlockRequest(res.getLockid())); + + // test that another lock blocks it + comp = new LockComponent(LockType.SHARED_READ, LockLevel.DB, + "mydb"); + comp.setTablename("mytable"); + comp.setPartitionname("mypartition"); + components.clear(); + components.add(comp); + req = new LockRequest(components, "me", "localhost"); + res = txnHandler.lock(req); + assertEquals(LockState.ACQUIRED, res.getState()); + + comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, + "mydb"); + comp.setTablename("mytable"); + comp.setPartitionname("mypartition"); + components.clear(); + components.add(comp); + req = new LockRequest(components, "me", "localhost"); + res = txnHandler.lockNoWait(req); + assertEquals(LockState.NOT_ACQUIRED, res.getState()); + assertEquals(1, TxnDbUtil.findNumCurrentLocks()); + } + + @Test + public void testFindPotentialCompactions() throws Exception { + // Test that committing unlocks + long txnid = openTxn(); + LockComponent comp = new LockComponent(LockType.SHARED_WRITE, LockLevel.DB, + "mydb"); + comp.setTablename("mytable"); + List components = new ArrayList(1); + components.add(comp); + comp = new LockComponent(LockType.SHARED_WRITE, LockLevel.DB, + "mydb"); + comp.setTablename("yourtable"); + comp.setPartitionname("mypartition"); + components.add(comp); + LockRequest req = new LockRequest(components, "me", "localhost"); + req.setTxnid(txnid); + LockResponse res = txnHandler.lock(req); + assertTrue(res.getState() == LockState.ACQUIRED); + txnHandler.commitTxn(new CommitTxnRequest(txnid)); + assertEquals(0, txnHandler.numLocksInLockTable()); + + List potentials = txnHandler.findPotentialCompactions(100); + assertEquals(2, potentials.size()); + boolean sawMyTable = false, sawYourTable = false; + for (CompactionInfo ci : potentials) { + sawMyTable |= (ci.dbname.equals("mydb") && ci.tableName.equals("mytable") && + ci.partName == null); + sawYourTable |= (ci.dbname.equals("mydb") && ci.tableName.equals("yourtable") && + ci.partName.equals("mypartition")); + } + assertTrue(sawMyTable); + assertTrue(sawYourTable); + } + + // TODO test changes to mark cleaned to clean txns and txn_components + + @Test + public void testMarkCleanedCleansTxnsAndTxnComponents() + throws Exception { + long txnid = openTxn(); + LockComponent comp = new LockComponent(LockType.SHARED_WRITE, LockLevel.DB, + "mydb"); + comp.setTablename("mytable"); + List components = new ArrayList(1); + components.add(comp); + LockRequest req = new LockRequest(components, "me", "localhost"); + req.setTxnid(txnid); + LockResponse res = txnHandler.lock(req); + assertTrue(res.getState() == LockState.ACQUIRED); + txnHandler.abortTxn(new AbortTxnRequest(txnid)); + + txnid = openTxn(); + comp = new LockComponent(LockType.SHARED_WRITE, LockLevel.DB, "mydb"); + comp.setTablename("yourtable"); + components = new ArrayList(1); + components.add(comp); + req = new LockRequest(components, "me", "localhost"); + req.setTxnid(txnid); + res = txnHandler.lock(req); + assertTrue(res.getState() == LockState.ACQUIRED); + txnHandler.abortTxn(new AbortTxnRequest(txnid)); + + txnid = openTxn(); + comp = new LockComponent(LockType.SHARED_WRITE, LockLevel.DB, "mydb"); + comp.setTablename("foo"); + comp.setPartitionname("bar"); + components = new ArrayList(1); + components.add(comp); + req = new LockRequest(components, "me", "localhost"); + req.setTxnid(txnid); + res = txnHandler.lock(req); + assertTrue(res.getState() == LockState.ACQUIRED); + + comp = new LockComponent(LockType.SHARED_WRITE, LockLevel.DB, "mydb"); + comp.setTablename("foo"); + comp.setPartitionname("baz"); + components = new ArrayList(1); + components.add(comp); + req = new LockRequest(components, "me", "localhost"); + req.setTxnid(txnid); + res = txnHandler.lock(req); + assertTrue(res.getState() == LockState.ACQUIRED); + txnHandler.abortTxn(new AbortTxnRequest(txnid)); + + CompactionInfo ci = new CompactionInfo(); + + // Now clean them and check that they are removed from the count. + CompactionRequest rqst = new CompactionRequest("mydb", "mytable", CompactionType.MAJOR); + txnHandler.compact(rqst); + assertEquals(0, txnHandler.findReadyToClean().size()); + ci = txnHandler.findNextToCompact("fred"); + assertNotNull(ci); + txnHandler.markCompacted(ci); + + List toClean = txnHandler.findReadyToClean(); + assertEquals(1, toClean.size()); + txnHandler.markCleaned(ci); + + // Check that we are cleaning up the empty aborted transactions + GetOpenTxnsResponse txnList = txnHandler.getOpenTxns(); + assertEquals(3, txnList.getOpen_txnsSize()); + txnHandler.cleanEmptyAbortedTxns(); + txnList = txnHandler.getOpenTxns(); + assertEquals(2, txnList.getOpen_txnsSize()); + + rqst = new CompactionRequest("mydb", "foo", CompactionType.MAJOR); + rqst.setPartitionname("bar"); + txnHandler.compact(rqst); + assertEquals(0, txnHandler.findReadyToClean().size()); + ci = txnHandler.findNextToCompact("fred"); + assertNotNull(ci); + txnHandler.markCompacted(ci); + + toClean = txnHandler.findReadyToClean(); + assertEquals(1, toClean.size()); + txnHandler.markCleaned(ci); + + txnHandler.openTxns(new OpenTxnRequest(1, "me", "localhost")); + txnHandler.cleanEmptyAbortedTxns(); + txnList = txnHandler.getOpenTxns(); + assertEquals(3, txnList.getOpen_txnsSize()); + } + + @Before + public void setUp() throws Exception { + TxnDbUtil.prepDb(); + txnHandler = new CompactionTxnHandler(conf); + } + + @After + public void tearDown() throws Exception { + TxnDbUtil.cleanDb(); + } + + private long openTxn() throws MetaException { + List txns = txnHandler.openTxns(new OpenTxnRequest(1, "me", "localhost")).getTxn_ids(); + return txns.get(0); + } + +} diff --git metastore/src/test/org/apache/hadoop/hive/metastore/txn/TestTxnHandler.java metastore/src/test/org/apache/hadoop/hive/metastore/txn/TestTxnHandler.java new file mode 100644 index 0000000..560fd5a --- /dev/null +++ metastore/src/test/org/apache/hadoop/hive/metastore/txn/TestTxnHandler.java @@ -0,0 +1,1079 @@ +/** + * 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. + */ +package org.apache.hadoop.hive.metastore.txn; + +import junit.framework.Assert; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.metastore.api.*; +import org.apache.log4j.Level; +import org.apache.log4j.LogManager; +import org.junit.After; +import org.junit.Before; +import org.junit.Ignore; +import org.junit.Test; + +import java.util.ArrayList; +import java.util.List; + +import static junit.framework.Assert.*; + +/** + * Tests for TxnHandler. + */ +public class TestTxnHandler { + static final private String CLASS_NAME = TxnHandler.class.getName(); + static final private Log LOG = LogFactory.getLog(CLASS_NAME); + + private HiveConf conf = new HiveConf(); + private TxnHandler txnHandler; + + public TestTxnHandler() throws Exception { + TxnDbUtil.setConfValues(conf); + LogManager.getLogger(TxnHandler.class.getName()).setLevel(Level.DEBUG); + tearDown(); + } + + @Test + public void testValidTxnsEmpty() throws Exception { + GetOpenTxnsInfoResponse txnsInfo = txnHandler.getOpenTxnsInfo(); + assertEquals(0L, txnsInfo.getTxn_high_water_mark()); + assertTrue(txnsInfo.getOpen_txns().isEmpty()); + GetOpenTxnsResponse txns = txnHandler.getOpenTxns(); + assertEquals(0L, txns.getTxn_high_water_mark()); + assertTrue(txns.getOpen_txns().isEmpty()); + } + + @Test + public void testOpenTxn() throws Exception { + long first = openTxn(); + assertEquals(1L, first); + long second = openTxn(); + assertEquals(2L, second); + GetOpenTxnsInfoResponse txnsInfo = txnHandler.getOpenTxnsInfo(); + assertEquals(2L, txnsInfo.getTxn_high_water_mark()); + assertEquals(2, txnsInfo.getOpen_txns().size()); + assertEquals(1L, txnsInfo.getOpen_txns().get(0).getId()); + assertEquals(TxnState.OPEN, txnsInfo.getOpen_txns().get(0).getState()); + assertEquals(2L, txnsInfo.getOpen_txns().get(1).getId()); + assertEquals(TxnState.OPEN, txnsInfo.getOpen_txns().get(1).getState()); + assertEquals("me", txnsInfo.getOpen_txns().get(1).getUser()); + assertEquals("localhost", txnsInfo.getOpen_txns().get(1).getHostname()); + + GetOpenTxnsResponse txns = txnHandler.getOpenTxns(); + assertEquals(2L, txns.getTxn_high_water_mark()); + assertEquals(2, txns.getOpen_txns().size()); + boolean[] saw = new boolean[3]; + for (int i = 0; i < saw.length; i++) saw[i] = false; + for (Long tid : txns.getOpen_txns()) { + saw[tid.intValue()] = true; + } + for (int i = 1; i < saw.length; i++) assertTrue(saw[i]); + } + + @Test + public void testAbortTxn() throws Exception { + OpenTxnsResponse openedTxns = txnHandler.openTxns(new OpenTxnRequest(2, "me", "localhost")); + List txnList = openedTxns.getTxn_ids(); + long first = txnList.get(0); + assertEquals(1L, first); + long second = txnList.get(1); + assertEquals(2L, second); + txnHandler.abortTxn(new AbortTxnRequest(1)); + GetOpenTxnsInfoResponse txnsInfo = txnHandler.getOpenTxnsInfo(); + assertEquals(2L, txnsInfo.getTxn_high_water_mark()); + assertEquals(2, txnsInfo.getOpen_txns().size()); + assertEquals(1L, txnsInfo.getOpen_txns().get(0).getId()); + assertEquals(TxnState.ABORTED, txnsInfo.getOpen_txns().get(0).getState()); + assertEquals(2L, txnsInfo.getOpen_txns().get(1).getId()); + assertEquals(TxnState.OPEN, txnsInfo.getOpen_txns().get(1).getState()); + + GetOpenTxnsResponse txns = txnHandler.getOpenTxns(); + assertEquals(2L, txns.getTxn_high_water_mark()); + assertEquals(2, txns.getOpen_txns().size()); + boolean[] saw = new boolean[3]; + for (int i = 0; i < saw.length; i++) saw[i] = false; + for (Long tid : txns.getOpen_txns()) { + saw[tid.intValue()] = true; + } + for (int i = 1; i < saw.length; i++) assertTrue(saw[i]); + } + + @Test + public void testAbortInvalidTxn() throws Exception { + boolean caught = false; + try { + txnHandler.abortTxn(new AbortTxnRequest(195L)); + } catch (NoSuchTxnException e) { + caught = true; + } + assertTrue(caught); + } + + @Test + public void testValidTxnsNoneOpen() throws Exception { + txnHandler.openTxns(new OpenTxnRequest(2, "me", "localhost")); + txnHandler.commitTxn(new CommitTxnRequest(1)); + txnHandler.commitTxn(new CommitTxnRequest(2)); + GetOpenTxnsInfoResponse txnsInfo = txnHandler.getOpenTxnsInfo(); + assertEquals(2L, txnsInfo.getTxn_high_water_mark()); + assertEquals(0, txnsInfo.getOpen_txns().size()); + GetOpenTxnsResponse txns = txnHandler.getOpenTxns(); + assertEquals(2L, txns.getTxn_high_water_mark()); + assertEquals(0, txns.getOpen_txns().size()); + } + + @Test + public void testValidTxnsSomeOpen() throws Exception { + txnHandler.openTxns(new OpenTxnRequest(3, "me", "localhost")); + txnHandler.abortTxn(new AbortTxnRequest(1)); + txnHandler.commitTxn(new CommitTxnRequest(2)); + GetOpenTxnsInfoResponse txnsInfo = txnHandler.getOpenTxnsInfo(); + assertEquals(3L, txnsInfo.getTxn_high_water_mark()); + assertEquals(2, txnsInfo.getOpen_txns().size()); + assertEquals(1L, txnsInfo.getOpen_txns().get(0).getId()); + assertEquals(TxnState.ABORTED, txnsInfo.getOpen_txns().get(0).getState()); + assertEquals(3L, txnsInfo.getOpen_txns().get(1).getId()); + assertEquals(TxnState.OPEN, txnsInfo.getOpen_txns().get(1).getState()); + + GetOpenTxnsResponse txns = txnHandler.getOpenTxns(); + assertEquals(3L, txns.getTxn_high_water_mark()); + assertEquals(2, txns.getOpen_txns().size()); + boolean[] saw = new boolean[4]; + for (int i = 0; i < saw.length; i++) saw[i] = false; + for (Long tid : txns.getOpen_txns()) { + saw[tid.intValue()] = true; + } + assertTrue(saw[1]); + assertFalse(saw[2]); + assertTrue(saw[3]); + } + + @Test + public void testLockDifferentDBs() throws Exception { + // Test that two different databases don't collide on their locks + LockComponent comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, "mydb"); + List components = new ArrayList(1); + components.add(comp); + LockRequest req = new LockRequest(components, "me", "localhost"); + LockResponse res = txnHandler.lock(req); + assertTrue(res.getState() == LockState.ACQUIRED); + + comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, "yourdb"); + components.clear(); + components.add(comp); + req = new LockRequest(components, "me", "localhost"); + res = txnHandler.lock(req); + assertTrue(res.getState() == LockState.ACQUIRED); + } + + @Test + public void testLockSameDB() throws Exception { + // Test that two different databases don't collide on their locks + LockComponent comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, "mydb"); + List components = new ArrayList(1); + components.add(comp); + LockRequest req = new LockRequest(components, "me", "localhost"); + LockResponse res = txnHandler.lock(req); + assertTrue(res.getState() == LockState.ACQUIRED); + + comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, "mydb"); + components.clear(); + components.add(comp); + req = new LockRequest(components, "me", "localhost"); + res = txnHandler.lock(req); + assertTrue(res.getState() == LockState.WAITING); + } + + @Test + public void testLockDbLocksTable() throws Exception { + // Test that locking a database prevents locking of tables in the database + LockComponent comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, "mydb"); + List components = new ArrayList(1); + components.add(comp); + LockRequest req = new LockRequest(components, "me", "localhost"); + LockResponse res = txnHandler.lock(req); + assertTrue(res.getState() == LockState.ACQUIRED); + + comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, "mydb"); + comp.setTablename("mytable"); + components.clear(); + components.add(comp); + req = new LockRequest(components, "me", "localhost"); + res = txnHandler.lock(req); + assertTrue(res.getState() == LockState.WAITING); + } + + @Test + public void testLockDbDoesNotLockTableInDifferentDB() throws Exception { + // Test that locking a database prevents locking of tables in the database + LockComponent comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, "mydb"); + List components = new ArrayList(1); + components.add(comp); + LockRequest req = new LockRequest(components, "me", "localhost"); + LockResponse res = txnHandler.lock(req); + assertTrue(res.getState() == LockState.ACQUIRED); + + comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, "yourdb"); + comp.setTablename("mytable"); + components.clear(); + components.add(comp); + req = new LockRequest(components, "me", "localhost"); + res = txnHandler.lock(req); + assertTrue(res.getState() == LockState.ACQUIRED); + } + + @Test + public void testLockDifferentTables() throws Exception { + // Test that two different tables don't collide on their locks + LockComponent comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, "mydb"); + comp.setTablename("mytable"); + List components = new ArrayList(1); + components.add(comp); + LockRequest req = new LockRequest(components, "me", "localhost"); + LockResponse res = txnHandler.lock(req); + assertTrue(res.getState() == LockState.ACQUIRED); + + comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, "mydb"); + comp.setTablename("yourtable"); + components.clear(); + components.add(comp); + req = new LockRequest(components, "me", "localhost"); + res = txnHandler.lock(req); + assertTrue(res.getState() == LockState.ACQUIRED); + } + + @Test + public void testLockSameTable() throws Exception { + // Test that two different tables don't collide on their locks + LockComponent comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, "mydb"); + comp.setTablename("mytable"); + List components = new ArrayList(1); + components.add(comp); + LockRequest req = new LockRequest(components, "me", "localhost"); + LockResponse res = txnHandler.lock(req); + assertTrue(res.getState() == LockState.ACQUIRED); + + comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, "mydb"); + comp.setTablename("mytable"); + components.clear(); + components.add(comp); + req = new LockRequest(components, "me", "localhost"); + res = txnHandler.lock(req); + assertTrue(res.getState() == LockState.WAITING); + } + + @Test + public void testLockTableLocksPartition() throws Exception { + // Test that locking a table prevents locking of partitions of the table + LockComponent comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, "mydb"); + comp.setTablename("mytable"); + List components = new ArrayList(1); + components.add(comp); + LockRequest req = new LockRequest(components, "me", "localhost"); + LockResponse res = txnHandler.lock(req); + assertTrue(res.getState() == LockState.ACQUIRED); + + comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, "mydb"); + comp.setTablename("mytable"); + comp.setPartitionname("mypartition"); + components.clear(); + components.add(comp); + req = new LockRequest(components, "me", "localhost"); + res = txnHandler.lock(req); + assertTrue(res.getState() == LockState.WAITING); + } + + @Test + public void testLockDifferentTableDoesntLockPartition() throws Exception { + // Test that locking a table prevents locking of partitions of the table + LockComponent comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, "mydb"); + comp.setTablename("mytable"); + List components = new ArrayList(1); + components.add(comp); + LockRequest req = new LockRequest(components, "me", "localhost"); + LockResponse res = txnHandler.lock(req); + assertTrue(res.getState() == LockState.ACQUIRED); + + comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, "mydb"); + comp.setTablename("yourtable"); + comp.setPartitionname("mypartition"); + components.clear(); + components.add(comp); + req = new LockRequest(components, "me", "localhost"); + res = txnHandler.lock(req); + assertTrue(res.getState() == LockState.ACQUIRED); + } + + @Test + public void testLockDifferentPartitions() throws Exception { + // Test that two different partitions don't collide on their locks + LockComponent comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, "mydb"); + comp.setTablename("mytable"); + comp.setPartitionname("mypartition"); + List components = new ArrayList(1); + components.add(comp); + LockRequest req = new LockRequest(components, "me", "localhost"); + LockResponse res = txnHandler.lock(req); + assertTrue(res.getState() == LockState.ACQUIRED); + + comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, "mydb"); + comp.setTablename("mytable"); + comp.setPartitionname("yourpartition"); + components.clear(); + components.add(comp); + req = new LockRequest(components, "me", "localhost"); + res = txnHandler.lock(req); + assertTrue(res.getState() == LockState.ACQUIRED); + } + + @Test + public void testLockSamePartition() throws Exception { + // Test that two different partitions don't collide on their locks + LockComponent comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, "mydb"); + comp.setTablename("mytable"); + comp.setPartitionname("mypartition"); + List components = new ArrayList(1); + components.add(comp); + LockRequest req = new LockRequest(components, "me", "localhost"); + LockResponse res = txnHandler.lock(req); + assertTrue(res.getState() == LockState.ACQUIRED); + + comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, "mydb"); + comp.setTablename("mytable"); + comp.setPartitionname("mypartition"); + components.clear(); + components.add(comp); + req = new LockRequest(components, "me", "localhost"); + res = txnHandler.lock(req); + assertTrue(res.getState() == LockState.WAITING); + } + + @Test + public void testLockSRSR() throws Exception { + // Test that two shared read locks can share a partition + LockComponent comp = new LockComponent(LockType.SHARED_READ, LockLevel.DB, "mydb"); + comp.setTablename("mytable"); + comp.setPartitionname("mypartition"); + List components = new ArrayList(1); + components.add(comp); + LockRequest req = new LockRequest(components, "me", "localhost"); + LockResponse res = txnHandler.lock(req); + assertTrue(res.getState() == LockState.ACQUIRED); + + comp = new LockComponent(LockType.SHARED_READ, LockLevel.DB, "mydb"); + comp.setTablename("mytable"); + comp.setPartitionname("mypartition"); + components.clear(); + components.add(comp); + req = new LockRequest(components, "me", "localhost"); + res = txnHandler.lock(req); + assertTrue(res.getState() == LockState.ACQUIRED); + } + + @Test + public void testLockESRSR() throws Exception { + // Test that exclusive lock blocks shared reads + LockComponent comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, "mydb"); + comp.setTablename("mytable"); + comp.setPartitionname("mypartition"); + List components = new ArrayList(1); + components.add(comp); + LockRequest req = new LockRequest(components, "me", "localhost"); + LockResponse res = txnHandler.lock(req); + assertTrue(res.getState() == LockState.ACQUIRED); + + comp = new LockComponent(LockType.SHARED_READ, LockLevel.DB, "mydb"); + comp.setTablename("mytable"); + comp.setPartitionname("mypartition"); + components.clear(); + components.add(comp); + req = new LockRequest(components, "me", "localhost"); + res = txnHandler.lock(req); + assertTrue(res.getState() == LockState.WAITING); + + comp = new LockComponent(LockType.SHARED_READ, LockLevel.DB, "mydb"); + comp.setTablename("mytable"); + comp.setPartitionname("mypartition"); + components.clear(); + components.add(comp); + req = new LockRequest(components, "me", "localhost"); + res = txnHandler.lock(req); + assertTrue(res.getState() == LockState.WAITING); + } + + @Test + public void testLockSRSW() throws Exception { + // Test that write can acquire after read + LockComponent comp = new LockComponent(LockType.SHARED_READ, LockLevel.DB, "mydb"); + comp.setTablename("mytable"); + comp.setPartitionname("mypartition"); + List components = new ArrayList(1); + components.add(comp); + LockRequest req = new LockRequest(components, "me", "localhost"); + LockResponse res = txnHandler.lock(req); + assertTrue(res.getState() == LockState.ACQUIRED); + + comp = new LockComponent(LockType.SHARED_WRITE, LockLevel.DB, "mydb"); + comp.setTablename("mytable"); + comp.setPartitionname("mypartition"); + components.clear(); + components.add(comp); + req = new LockRequest(components, "me", "localhost"); + res = txnHandler.lock(req); + assertTrue(res.getState() == LockState.ACQUIRED); + } + + @Test + public void testLockESRSW() throws Exception { + // Test that exclusive lock blocks read and write + LockComponent comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, "mydb"); + comp.setTablename("mytable"); + comp.setPartitionname("mypartition"); + List components = new ArrayList(1); + components.add(comp); + LockRequest req = new LockRequest(components, "me", "localhost"); + LockResponse res = txnHandler.lock(req); + assertTrue(res.getState() == LockState.ACQUIRED); + + comp = new LockComponent(LockType.SHARED_READ, LockLevel.DB, "mydb"); + comp.setTablename("mytable"); + comp.setPartitionname("mypartition"); + components.clear(); + components.add(comp); + req = new LockRequest(components, "me", "localhost"); + res = txnHandler.lock(req); + assertTrue(res.getState() == LockState.WAITING); + + comp = new LockComponent(LockType.SHARED_WRITE, LockLevel.DB, "mydb"); + comp.setTablename("mytable"); + comp.setPartitionname("mypartition"); + components.clear(); + components.add(comp); + req = new LockRequest(components, "me", "localhost"); + res = txnHandler.lock(req); + assertTrue(res.getState() == LockState.WAITING); + } + + @Test + public void testLockSRE() throws Exception { + // Test that read blocks exclusive + LockComponent comp = new LockComponent(LockType.SHARED_READ, LockLevel.DB, "mydb"); + comp.setTablename("mytable"); + comp.setPartitionname("mypartition"); + List components = new ArrayList(1); + components.add(comp); + LockRequest req = new LockRequest(components, "me", "localhost"); + LockResponse res = txnHandler.lock(req); + assertTrue(res.getState() == LockState.ACQUIRED); + + comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, "mydb"); + comp.setTablename("mytable"); + comp.setPartitionname("mypartition"); + components.clear(); + components.add(comp); + req = new LockRequest(components, "me", "localhost"); + res = txnHandler.lock(req); + assertTrue(res.getState() == LockState.WAITING); + } + + @Test + public void testLockESRE() throws Exception { + // Test that exclusive blocks read and exclusive + LockComponent comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, "mydb"); + comp.setTablename("mytable"); + comp.setPartitionname("mypartition"); + List components = new ArrayList(1); + components.add(comp); + LockRequest req = new LockRequest(components, "me", "localhost"); + LockResponse res = txnHandler.lock(req); + assertTrue(res.getState() == LockState.ACQUIRED); + + comp = new LockComponent(LockType.SHARED_READ, LockLevel.DB, "mydb"); + comp.setTablename("mytable"); + comp.setPartitionname("mypartition"); + components.clear(); + components.add(comp); + req = new LockRequest(components, "me", "localhost"); + res = txnHandler.lock(req); + assertTrue(res.getState() == LockState.WAITING); + + comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, "mydb"); + comp.setTablename("mytable"); + comp.setPartitionname("mypartition"); + components.clear(); + components.add(comp); + req = new LockRequest(components, "me", "localhost"); + res = txnHandler.lock(req); + assertTrue(res.getState() == LockState.WAITING); + } + + @Test + public void testLockSWSR() throws Exception { + // Test that read can acquire after write + LockComponent comp = new LockComponent(LockType.SHARED_WRITE, LockLevel.DB, "mydb"); + comp.setTablename("mytable"); + comp.setPartitionname("mypartition"); + List components = new ArrayList(1); + components.add(comp); + LockRequest req = new LockRequest(components, "me", "localhost"); + LockResponse res = txnHandler.lock(req); + assertTrue(res.getState() == LockState.ACQUIRED); + + comp = new LockComponent(LockType.SHARED_READ, LockLevel.DB, "mydb"); + comp.setTablename("mytable"); + comp.setPartitionname("mypartition"); + components.clear(); + components.add(comp); + req = new LockRequest(components, "me", "localhost"); + res = txnHandler.lock(req); + assertTrue(res.getState() == LockState.ACQUIRED); + } + + @Test + public void testLockSWSWSR() throws Exception { + // Test that write blocks write but read can still acquire + LockComponent comp = new LockComponent(LockType.SHARED_WRITE, LockLevel.DB, "mydb"); + comp.setTablename("mytable"); + comp.setPartitionname("mypartition"); + List components = new ArrayList(1); + components.add(comp); + LockRequest req = new LockRequest(components, "me", "localhost"); + LockResponse res = txnHandler.lock(req); + assertTrue(res.getState() == LockState.ACQUIRED); + + comp = new LockComponent(LockType.SHARED_WRITE, LockLevel.DB, "mydb"); + comp.setTablename("mytable"); + comp.setPartitionname("mypartition"); + components.clear(); + components.add(comp); + req = new LockRequest(components, "me", "localhost"); + res = txnHandler.lock(req); + assertTrue(res.getState() == LockState.WAITING); + + comp = new LockComponent(LockType.SHARED_READ, LockLevel.DB, "mydb"); + comp.setTablename("mytable"); + comp.setPartitionname("mypartition"); + components.clear(); + components.add(comp); + req = new LockRequest(components, "me", "localhost"); + res = txnHandler.lock(req); + assertTrue(res.getState() == LockState.ACQUIRED); + } + + @Test + public void testLockSWSWSW() throws Exception { + // Test that write blocks two writes + LockComponent comp = new LockComponent(LockType.SHARED_WRITE, LockLevel.DB, "mydb"); + comp.setTablename("mytable"); + comp.setPartitionname("mypartition"); + List components = new ArrayList(1); + components.add(comp); + LockRequest req = new LockRequest(components, "me", "localhost"); + LockResponse res = txnHandler.lock(req); + assertTrue(res.getState() == LockState.ACQUIRED); + + comp = new LockComponent(LockType.SHARED_WRITE, LockLevel.DB, "mydb"); + comp.setTablename("mytable"); + comp.setPartitionname("mypartition"); + components.clear(); + components.add(comp); + req = new LockRequest(components, "me", "localhost"); + res = txnHandler.lock(req); + assertTrue(res.getState() == LockState.WAITING); + + comp = new LockComponent(LockType.SHARED_WRITE, LockLevel.DB, "mydb"); + comp.setTablename("mytable"); + comp.setPartitionname("mypartition"); + components.clear(); + components.add(comp); + req = new LockRequest(components, "me", "localhost"); + res = txnHandler.lock(req); + assertTrue(res.getState() == LockState.WAITING); + } + + @Test + public void testLockEESW() throws Exception { + // Test that exclusive blocks exclusive and write + LockComponent comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, "mydb"); + comp.setTablename("mytable"); + comp.setPartitionname("mypartition"); + List components = new ArrayList(1); + components.add(comp); + LockRequest req = new LockRequest(components, "me", "localhost"); + LockResponse res = txnHandler.lock(req); + assertTrue(res.getState() == LockState.ACQUIRED); + + comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, "mydb"); + comp.setTablename("mytable"); + comp.setPartitionname("mypartition"); + components.clear(); + components.add(comp); + req = new LockRequest(components, "me", "localhost"); + res = txnHandler.lock(req); + assertTrue(res.getState() == LockState.WAITING); + + comp = new LockComponent(LockType.SHARED_WRITE, LockLevel.DB, "mydb"); + comp.setTablename("mytable"); + comp.setPartitionname("mypartition"); + components.clear(); + components.add(comp); + req = new LockRequest(components, "me", "localhost"); + res = txnHandler.lock(req); + assertTrue(res.getState() == LockState.WAITING); + } + + @Test + public void testLockEESR() throws Exception { + // Test that exclusive blocks exclusive and read + LockComponent comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, "mydb"); + comp.setTablename("mytable"); + comp.setPartitionname("mypartition"); + List components = new ArrayList(1); + components.add(comp); + LockRequest req = new LockRequest(components, "me", "localhost"); + LockResponse res = txnHandler.lock(req); + assertTrue(res.getState() == LockState.ACQUIRED); + + comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, "mydb"); + comp.setTablename("mytable"); + comp.setPartitionname("mypartition"); + components.clear(); + components.add(comp); + req = new LockRequest(components, "me", "localhost"); + res = txnHandler.lock(req); + assertTrue(res.getState() == LockState.WAITING); + + comp = new LockComponent(LockType.SHARED_READ, LockLevel.DB, "mydb"); + comp.setTablename("mytable"); + comp.setPartitionname("mypartition"); + components.clear(); + components.add(comp); + req = new LockRequest(components, "me", "localhost"); + res = txnHandler.lock(req); + assertTrue(res.getState() == LockState.WAITING); + } + + @Test + public void testCheckLockAcquireAfterWaiting() throws Exception { + LockComponent comp = new LockComponent(LockType.SHARED_WRITE, LockLevel.DB, "mydb"); + comp.setTablename("mytable"); + comp.setPartitionname("mypartition"); + List components = new ArrayList(1); + components.add(comp); + LockRequest req = new LockRequest(components, "me", "localhost"); + LockResponse res = txnHandler.lock(req); + long lockid1 = res.getLockid(); + assertTrue(res.getState() == LockState.ACQUIRED); + + comp = new LockComponent(LockType.SHARED_WRITE, LockLevel.DB, "mydb"); + comp.setTablename("mytable"); + comp.setPartitionname("mypartition"); + components.clear(); + components.add(comp); + req = new LockRequest(components, "me", "localhost"); + res = txnHandler.lock(req); + long lockid2 = res.getLockid(); + assertTrue(res.getState() == LockState.WAITING); + + txnHandler.unlock(new UnlockRequest(lockid1)); + res = txnHandler.checkLock(new CheckLockRequest(lockid2)); + assertTrue(res.getState() == LockState.ACQUIRED); + } + + @Test + public void testCheckLockNoSuchLock() throws Exception { + try { + txnHandler.checkLock(new CheckLockRequest(23L)); + fail("Allowed to check lock on non-existent lock"); + } catch (NoSuchLockException e) { + } + } + + @Test + public void testCheckLockTxnAborted() throws Exception { + // Test that when a transaction is aborted, the heartbeat fails + long txnid = openTxn(); + LockComponent comp = new LockComponent(LockType.SHARED_WRITE, LockLevel.DB, "mydb"); + comp.setTablename("mytable"); + comp.setPartitionname("mypartition"); + List components = new ArrayList(1); + components.add(comp); + LockRequest req = new LockRequest(components, "me", "localhost"); + req.setTxnid(txnid); + LockResponse res = txnHandler.lock(req); + long lockid = res.getLockid(); + txnHandler.abortTxn(new AbortTxnRequest(txnid)); + try { + // This will throw NoSuchLockException (even though it's the + // transaction we've closed) because that will have deleted the lock. + txnHandler.checkLock(new CheckLockRequest(lockid)); + fail("Allowed to check lock on aborted transaction."); + } catch (NoSuchLockException e) { + } + } + + @Test + public void testMultipleLock() throws Exception { + // Test more than one lock can be handled in a lock request + LockComponent comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, "mydb"); + comp.setTablename("mytable"); + comp.setPartitionname("mypartition"); + List components = new ArrayList(2); + components.add(comp); + + comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, "mydb"); + comp.setTablename("mytable"); + comp.setPartitionname("anotherpartition"); + components.add(comp); + LockRequest req = new LockRequest(components, "me", "localhost"); + LockResponse res = txnHandler.lock(req); + long lockid = res.getLockid(); + assertTrue(res.getState() == LockState.ACQUIRED); + res = txnHandler.checkLock(new CheckLockRequest(lockid)); + assertTrue(res.getState() == LockState.ACQUIRED); + txnHandler.unlock(new UnlockRequest(lockid)); + assertEquals(0, txnHandler.numLocksInLockTable()); + } + + @Test + public void testMultipleLockWait() throws Exception { + // Test that two shared read locks can share a partition + LockComponent comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, "mydb"); + comp.setTablename("mytable"); + comp.setPartitionname("mypartition"); + List components = new ArrayList(2); + components.add(comp); + + comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, "mydb"); + comp.setTablename("mytable"); + comp.setPartitionname("anotherpartition"); + components.add(comp); + LockRequest req = new LockRequest(components, "me", "localhost"); + LockResponse res = txnHandler.lock(req); + long lockid1 = res.getLockid(); + assertTrue(res.getState() == LockState.ACQUIRED); + + + comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, "mydb"); + comp.setTablename("mytable"); + comp.setPartitionname("mypartition"); + components = new ArrayList(1); + components.add(comp); + req = new LockRequest(components, "me", "localhost"); + res = txnHandler.lock(req); + long lockid2 = res.getLockid(); + assertTrue(res.getState() == LockState.WAITING); + + txnHandler.unlock(new UnlockRequest(lockid1)); + + res = txnHandler.checkLock(new CheckLockRequest(lockid2)); + assertTrue(res.getState() == LockState.ACQUIRED); + } + + @Test + public void testUnlockOnCommit() throws Exception { + // Test that committing unlocks + long txnid = openTxn(); + LockComponent comp = new LockComponent(LockType.SHARED_WRITE, LockLevel.DB, "mydb"); + comp.setTablename("mytable"); + List components = new ArrayList(1); + components.add(comp); + LockRequest req = new LockRequest(components, "me", "localhost"); + req.setTxnid(txnid); + LockResponse res = txnHandler.lock(req); + assertTrue(res.getState() == LockState.ACQUIRED); + txnHandler.commitTxn(new CommitTxnRequest(txnid)); + assertEquals(0, txnHandler.numLocksInLockTable()); + } + + @Test + public void testUnlockOnAbort() throws Exception { + // Test that committing unlocks + long txnid = openTxn(); + LockComponent comp = new LockComponent(LockType.SHARED_WRITE, LockLevel.DB, "mydb"); + List components = new ArrayList(1); + components.add(comp); + LockRequest req = new LockRequest(components, "me", "localhost"); + req.setTxnid(txnid); + LockResponse res = txnHandler.lock(req); + assertTrue(res.getState() == LockState.ACQUIRED); + txnHandler.abortTxn(new AbortTxnRequest(txnid)); + assertEquals(0, txnHandler.numLocksInLockTable()); + } + + @Test + public void testUnlockWithTxn() throws Exception { + LOG.debug("Starting testUnlockWithTxn"); + // Test that attempting to unlock locks associated with a transaction + // generates an error + long txnid = openTxn(); + LockComponent comp = new LockComponent(LockType.SHARED_WRITE, LockLevel.DB, "mydb"); + comp.setTablename("mytable"); + comp.setPartitionname("mypartition"); + List components = new ArrayList(1); + components.add(comp); + LockRequest req = new LockRequest(components, "me", "localhost"); + req.setTxnid(txnid); + LockResponse res = txnHandler.lock(req); + long lockid = res.getLockid(); + try { + txnHandler.unlock(new UnlockRequest(lockid)); + fail("Allowed to unlock lock associated with transaction."); + } catch (TxnOpenException e) { + } + } + + @Test + public void testHeartbeatTxnAborted() throws Exception { + // Test that when a transaction is aborted, the heartbeat fails + openTxn(); + txnHandler.abortTxn(new AbortTxnRequest(1)); + HeartbeatRequest h = new HeartbeatRequest(); + h.setTxnid(1); + try { + txnHandler.heartbeat(h); + fail("Told there was a txn, when it should have been aborted."); + } catch (TxnAbortedException e) { + } + } + + @Test + public void testHeartbeatNoTxn() throws Exception { + // Test that when a transaction is aborted, the heartbeat fails + HeartbeatRequest h = new HeartbeatRequest(); + h.setTxnid(939393L); + try { + txnHandler.heartbeat(h); + fail("Told there was a txn, when there wasn't."); + } catch (NoSuchTxnException e) { + } + } + + @Test + public void testHeartbeatLock() throws Exception { + conf.setIntVar(HiveConf.ConfVars.HIVE_TXN_TIMEOUT, 1); + HeartbeatRequest h = new HeartbeatRequest(); + LockComponent comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, "mydb"); + comp.setTablename("mytable"); + comp.setPartitionname("mypartition"); + List components = new ArrayList(1); + components.add(comp); + LockRequest req = new LockRequest(components, "me", "localhost"); + LockResponse res = txnHandler.lock(req); + assertTrue(res.getState() == LockState.ACQUIRED); + h.setLockid(res.getLockid()); + for (int i = 0; i < 30; i++) { + try { + txnHandler.heartbeat(h); + } catch (NoSuchLockException e) { + fail("Told there was no lock, when the heartbeat should have kept it."); + } + } + } + + @Test + public void testLockTimeout() throws Exception { + long timeout = txnHandler.setTimeout(1); + LockComponent comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, "mydb"); + comp.setTablename("mytable"); + comp.setPartitionname("mypartition"); + List components = new ArrayList(1); + components.add(comp); + LockRequest req = new LockRequest(components, "me", "localhost"); + LockResponse res = txnHandler.lock(req); + assertTrue(res.getState() == LockState.ACQUIRED); + Thread.currentThread().sleep(10); + try { + txnHandler.checkLock(new CheckLockRequest(res.getLockid())); + fail("Told there was a lock, when it should have timed out."); + } catch (NoSuchLockException e) { + } finally { + txnHandler.setTimeout(timeout); + } + } + + @Test + public void testHeartbeatNoLock() throws Exception { + HeartbeatRequest h = new HeartbeatRequest(); + h.setLockid(29389839L); + try { + txnHandler.heartbeat(h); + fail("Told there was a lock, when there wasn't."); + } catch (NoSuchLockException e) { + } + } + + @Ignore // This test breaks the others when it unsets the value + @Test + public void testNoJDBCDriver() throws Exception { + HiveConf confCopy = new HiveConf(conf); + confCopy.unset(HiveConf.ConfVars.HIVE_TXN_JDBC_DRIVER.varname); + boolean sawException = false; + try { + TxnHandler tt = new TxnHandler(confCopy); + } catch (Exception e) { + if (e instanceof RuntimeException && e.getMessage().contains("JDBC " + + "driver for transaction db not set")) { + sawException = true; + } else { + throw e; + } + } + assertTrue(sawException); + } + + @Test + public void testCompactMajorWithPartition() throws Exception { + CompactionRequest rqst = new CompactionRequest("foo", "bar", CompactionType.MAJOR); + rqst.setPartitionname("ds=today"); + txnHandler.compact(rqst); + ShowCompactResponse rsp = txnHandler.showCompact(new ShowCompactRequest()); + List compacts = rsp.getCompacts(); + assertEquals(1, compacts.size()); + ShowCompactResponseElement c = compacts.get(0); + assertEquals("foo", c.getDbname()); + assertEquals("bar", c.getTablename()); + assertEquals("ds=today", c.getPartitionname()); + assertEquals(CompactionType.MAJOR, c.getType()); + assertEquals("initiated", c.getState()); + assertEquals(0L, c.getStart()); + } + + @Test + public void testCompactMinorNoPartition() throws Exception { + CompactionRequest rqst = new CompactionRequest("foo", "bar", CompactionType.MINOR); + rqst.setRunas("fred"); + txnHandler.compact(rqst); + + ShowCompactResponse rsp = txnHandler.showCompact(new ShowCompactRequest()); + List compacts = rsp.getCompacts(); + assertEquals(1, compacts.size()); + ShowCompactResponseElement c = compacts.get(0); + assertEquals("foo", c.getDbname()); + assertEquals("bar", c.getTablename()); + assertNull(c.getPartitionname()); + assertEquals(CompactionType.MINOR, c.getType()); + assertEquals("initiated", c.getState()); + assertEquals(0L, c.getStart()); + assertEquals("fred", c.getRunAs()); + } + + @Test + public void showLocks() throws Exception { + long begining = System.currentTimeMillis(); + long txnid = openTxn(); + LockComponent comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, "mydb"); + List components = new ArrayList(1); + components.add(comp); + LockRequest req = new LockRequest(components, "me", "localhost"); + req.setTxnid(txnid); + LockResponse res = txnHandler.lock(req); + + // Open txn + txnid = openTxn(); + comp = new LockComponent(LockType.SHARED_READ, LockLevel.TABLE, "mydb"); + comp.setTablename("mytable"); + components = new ArrayList(1); + components.add(comp); + req = new LockRequest(components, "me", "localhost"); + req.setTxnid(txnid); + res = txnHandler.lock(req); + + // Locks not associated with a txn + components = new ArrayList(1); + comp = new LockComponent(LockType.SHARED_WRITE, LockLevel.PARTITION, "yourdb"); + comp.setTablename("yourtable"); + comp.setPartitionname("yourpartition"); + components.add(comp); + req = new LockRequest(components, "you", "remotehost"); + res = txnHandler.lock(req); + + ShowLocksResponse rsp = txnHandler.showLocks(new ShowLocksRequest()); + List locks = rsp.getLocks(); + assertEquals(3, locks.size()); + boolean[] saw = new boolean[locks.size()]; + for (int i = 0; i < saw.length; i++) saw[i] = false; + for (ShowLocksResponseElement lock : locks) { + if (lock.getLockid() == 1) { + assertEquals(1, lock.getTxnid()); + assertEquals("mydb", lock.getDbname()); + assertNull(lock.getTablename()); + assertNull(lock.getPartname()); + assertEquals(LockState.ACQUIRED, lock.getState()); + assertEquals(LockType.EXCLUSIVE, lock.getType()); + assertTrue(begining <= lock.getLastheartbeat() && + System.currentTimeMillis() >= lock.getLastheartbeat()); + assertTrue("Expected acquired at " + lock.getAcquiredat() + " to be between " + begining + + " and " + System.currentTimeMillis(), + begining <= lock.getAcquiredat() && System.currentTimeMillis() >= lock.getAcquiredat()); + assertEquals("me", lock.getUser()); + assertEquals("localhost", lock.getHostname()); + saw[0] = true; + } else if (lock.getLockid() == 2) { + assertEquals(2, lock.getTxnid()); + assertEquals("mydb", lock.getDbname()); + assertEquals("mytable", lock.getTablename()); + assertNull(lock.getPartname()); + assertEquals(LockState.WAITING, lock.getState()); + assertEquals(LockType.SHARED_READ, lock.getType()); + assertTrue(begining <= lock.getLastheartbeat() && + System.currentTimeMillis() >= lock.getLastheartbeat()); + assertEquals(0, lock.getAcquiredat()); + assertEquals("me", lock.getUser()); + assertEquals("localhost", lock.getHostname()); + saw[1] = true; + } else if (lock.getLockid() == 3) { + assertEquals(0, lock.getTxnid()); + assertEquals("yourdb", lock.getDbname()); + assertEquals("yourtable", lock.getTablename()); + assertEquals("yourpartition", lock.getPartname()); + assertEquals(LockState.ACQUIRED, lock.getState()); + assertEquals(LockType.SHARED_WRITE, lock.getType()); + assertTrue(begining <= lock.getLastheartbeat() && + System.currentTimeMillis() >= lock.getLastheartbeat()); + assertTrue(begining <= lock.getAcquiredat() && + System.currentTimeMillis() >= lock.getAcquiredat()); + assertEquals("you", lock.getUser()); + assertEquals("remotehost", lock.getHostname()); + saw[2] = true; + } else { + fail("Unknown lock id"); + } + } + for (int i = 0; i < saw.length; i++) assertTrue("Didn't see lock id " + i, saw[i]); + } + + @Before + public void setUp() throws Exception { + TxnDbUtil.prepDb(); + txnHandler = new TxnHandler(conf); + } + + @After + public void tearDown() throws Exception { + TxnDbUtil.cleanDb(); + } + + private long openTxn() throws MetaException { + List txns = txnHandler.openTxns(new OpenTxnRequest(1, "me", "localhost")).getTxn_ids(); + return txns.get(0); + } + +} diff --git ql/src/java/org/apache/hadoop/hive/ql/Context.java ql/src/java/org/apache/hadoop/hive/ql/Context.java index e699c6b..b34978c 100644 --- ql/src/java/org/apache/hadoop/hive/ql/Context.java +++ ql/src/java/org/apache/hadoop/hive/ql/Context.java @@ -18,18 +18,6 @@ package org.apache.hadoop.hive.ql; -import java.io.DataInput; -import java.io.FileNotFoundException; -import java.io.IOException; -import java.net.URI; -import java.text.SimpleDateFormat; -import java.util.Date; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Random; -import java.util.concurrent.ConcurrentHashMap; - import org.antlr.runtime.TokenRewriteStream; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -46,10 +34,23 @@ import org.apache.hadoop.hive.ql.lockmgr.HiveLock; import org.apache.hadoop.hive.ql.lockmgr.HiveLockManager; import org.apache.hadoop.hive.ql.lockmgr.HiveLockObj; +import org.apache.hadoop.hive.ql.lockmgr.HiveTxnManager; import org.apache.hadoop.hive.ql.plan.LoadTableDesc; import org.apache.hadoop.hive.shims.ShimLoader; import org.apache.hadoop.util.StringUtils; +import java.io.DataInput; +import java.io.FileNotFoundException; +import java.io.IOException; +import java.net.URI; +import java.text.SimpleDateFormat; +import java.util.Date; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Random; +import java.util.concurrent.ConcurrentHashMap; + /** * Context for Semantic Analyzers. Usage: not reusable - construct a new one for * each query should call clear() at end of use to remove temporary folders @@ -93,6 +94,9 @@ protected List hiveLocks; protected HiveLockManager hiveLockMgr; + // Transaction manager for this query + protected HiveTxnManager hiveTxnManager; + private boolean needLockMgr; // Keep track of the mapping from load table desc to the output and the lock @@ -533,15 +537,12 @@ public void setHiveLocks(List hiveLocks) { this.hiveLocks = hiveLocks; } - public HiveLockManager getHiveLockMgr() { - if (hiveLockMgr != null) { - hiveLockMgr.refresh(); - } - return hiveLockMgr; + public HiveTxnManager getHiveTxnManager() { + return hiveTxnManager; } - public void setHiveLockMgr(HiveLockManager hiveLockMgr) { - this.hiveLockMgr = hiveLockMgr; + public void setHiveTxnManager(HiveTxnManager txnMgr) { + hiveTxnManager = txnMgr; } public void setOriginalTracker(String originalTracker) { diff --git ql/src/java/org/apache/hadoop/hive/ql/Driver.java ql/src/java/org/apache/hadoop/hive/ql/Driver.java index 332cadb..318e21a 100644 --- ql/src/java/org/apache/hadoop/hive/ql/Driver.java +++ ql/src/java/org/apache/hadoop/hive/ql/Driver.java @@ -19,40 +19,18 @@ package org.apache.hadoop.hive.ql; -import java.io.DataInput; -import java.io.IOException; -import java.io.Serializable; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.HashSet; -import java.util.Iterator; -import java.util.LinkedHashMap; -import java.util.LinkedList; -import java.util.List; -import java.util.Map; -import java.util.Queue; -import java.util.Set; -import java.util.concurrent.ConcurrentLinkedQueue; - import org.apache.commons.lang.StringUtils; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.conf.HiveConf.ConfVars; +import org.apache.hadoop.hive.metastore.IMetaStoreClient; import org.apache.hadoop.hive.metastore.MetaStoreUtils; import org.apache.hadoop.hive.metastore.api.Database; import org.apache.hadoop.hive.metastore.api.FieldSchema; import org.apache.hadoop.hive.metastore.api.Schema; -import org.apache.hadoop.hive.ql.exec.ConditionalTask; -import org.apache.hadoop.hive.ql.exec.FetchTask; -import org.apache.hadoop.hive.ql.exec.Operator; -import org.apache.hadoop.hive.ql.exec.TableScanOperator; -import org.apache.hadoop.hive.ql.exec.Task; -import org.apache.hadoop.hive.ql.exec.TaskFactory; -import org.apache.hadoop.hive.ql.exec.TaskResult; -import org.apache.hadoop.hive.ql.exec.TaskRunner; -import org.apache.hadoop.hive.ql.exec.Utilities; +import org.apache.hadoop.hive.ql.exec.*; import org.apache.hadoop.hive.ql.history.HiveHistory.Keys; import org.apache.hadoop.hive.ql.hooks.Entity; import org.apache.hadoop.hive.ql.hooks.ExecuteWithHookContext; @@ -63,39 +41,15 @@ import org.apache.hadoop.hive.ql.hooks.PreExecute; import org.apache.hadoop.hive.ql.hooks.ReadEntity; import org.apache.hadoop.hive.ql.hooks.WriteEntity; -import org.apache.hadoop.hive.ql.lockmgr.HiveLock; -import org.apache.hadoop.hive.ql.lockmgr.HiveLockManager; -import org.apache.hadoop.hive.ql.lockmgr.HiveLockManagerCtx; -import org.apache.hadoop.hive.ql.lockmgr.HiveLockMode; -import org.apache.hadoop.hive.ql.lockmgr.HiveLockObj; -import org.apache.hadoop.hive.ql.lockmgr.HiveLockObject; +import org.apache.hadoop.hive.ql.lockmgr.*; import org.apache.hadoop.hive.ql.lockmgr.HiveLockObject.HiveLockObjectData; -import org.apache.hadoop.hive.ql.lockmgr.LockException; import org.apache.hadoop.hive.ql.log.PerfLogger; -import org.apache.hadoop.hive.ql.metadata.AuthorizationException; -import org.apache.hadoop.hive.ql.metadata.DummyPartition; -import org.apache.hadoop.hive.ql.metadata.Hive; -import org.apache.hadoop.hive.ql.metadata.HiveException; -import org.apache.hadoop.hive.ql.metadata.Partition; -import org.apache.hadoop.hive.ql.metadata.Table; +import org.apache.hadoop.hive.ql.metadata.*; import org.apache.hadoop.hive.ql.metadata.formatting.JsonMetaDataFormatter; import org.apache.hadoop.hive.ql.metadata.formatting.MetaDataFormatUtils; import org.apache.hadoop.hive.ql.metadata.formatting.MetaDataFormatter; import org.apache.hadoop.hive.ql.optimizer.ppr.PartitionPruner; -import org.apache.hadoop.hive.ql.parse.ASTNode; -import org.apache.hadoop.hive.ql.parse.BaseSemanticAnalyzer; -import org.apache.hadoop.hive.ql.parse.HiveSemanticAnalyzerHook; -import org.apache.hadoop.hive.ql.parse.HiveSemanticAnalyzerHookContext; -import org.apache.hadoop.hive.ql.parse.HiveSemanticAnalyzerHookContextImpl; -import org.apache.hadoop.hive.ql.parse.ImportSemanticAnalyzer; -import org.apache.hadoop.hive.ql.parse.ParseContext; -import org.apache.hadoop.hive.ql.parse.ParseDriver; -import org.apache.hadoop.hive.ql.parse.ParseUtils; -import org.apache.hadoop.hive.ql.parse.PrunedPartitionList; -import org.apache.hadoop.hive.ql.parse.SemanticAnalyzer; -import org.apache.hadoop.hive.ql.parse.SemanticAnalyzerFactory; -import org.apache.hadoop.hive.ql.parse.SemanticException; -import org.apache.hadoop.hive.ql.parse.VariableSubstitution; +import org.apache.hadoop.hive.ql.parse.*; import org.apache.hadoop.hive.ql.plan.HiveOperation; import org.apache.hadoop.hive.ql.plan.OperatorDesc; import org.apache.hadoop.hive.ql.plan.TableDesc; @@ -112,7 +66,13 @@ import org.apache.hadoop.mapred.ClusterStatus; import org.apache.hadoop.mapred.JobClient; import org.apache.hadoop.mapred.JobConf; -import org.apache.hadoop.util.ReflectionUtils; + +import java.io.DataInput; +import java.io.IOException; +import java.io.Serializable; +import java.util.*; +import java.util.concurrent.ConcurrentLinkedQueue; + public class Driver implements CommandProcessor { @@ -130,11 +90,10 @@ private Context ctx; private QueryPlan plan; private Schema schema; - private HiveLockManager hiveLockMgr; - private String errorMessage; private String SQLState; private Throwable downstreamError; + private HiveTxnManager txnMgr; // A limit on the number of threads that can be launched private int maxthreads; @@ -143,48 +102,29 @@ private String userName; + private void createTxnManager() throws SemanticException { + if (txnMgr == null) { + try { + txnMgr = TxnManagerFactory.getTxnManagerFactory().getTxnManager(conf); + } catch (LockException e) { + throw new SemanticException(e.getMessage(), e); + } + } + // the reason that we set the txn manager for the cxt here is because each + // query has its own ctx object. The txn mgr is shared across the + // same instance of Driver, which can run multiple queries. + ctx.setHiveTxnManager(txnMgr); + } + private boolean checkConcurrency() throws SemanticException { boolean supportConcurrency = conf.getBoolVar(HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY); if (!supportConcurrency) { LOG.info("Concurrency mode is disabled, not creating a lock manager"); return false; } - createLockManager(); - // the reason that we set the lock manager for the cxt here is because each - // query has its own ctx object. The hiveLockMgr is shared accross the - // same instance of Driver, which can run multiple queries. - ctx.setHiveLockMgr(hiveLockMgr); return true; } - private void createLockManager() throws SemanticException { - if (hiveLockMgr != null) { - return; - } - String lockMgr = conf.getVar(HiveConf.ConfVars.HIVE_LOCK_MANAGER); - LOG.info("Creating lock manager of type " + lockMgr); - if ((lockMgr == null) || (lockMgr.isEmpty())) { - throw new SemanticException(ErrorMsg.LOCKMGR_NOT_SPECIFIED.getMsg()); - } - try { - hiveLockMgr = (HiveLockManager) ReflectionUtils.newInstance(conf.getClassByName(lockMgr), - conf); - hiveLockMgr.setContext(new HiveLockManagerCtx(conf)); - } catch (Exception e1) { - // set hiveLockMgr to null just in case this invalid manager got set to - // next query's ctx. - if (hiveLockMgr != null) { - try { - hiveLockMgr.close(); - } catch (LockException e2) { - //nothing can do here - } - hiveLockMgr = null; - } - throw new SemanticException(ErrorMsg.LOCKMGR_NOT_INITIALIZED.getMsg() + e1.getMessage(), e1); - } - } - @Override public void init() { Operator.resetId(); @@ -858,25 +798,20 @@ public QueryPlan getPlan() { return locks; } - /** - * Dedup the list of lock objects so that there is only one lock per table/partition. - * If there is both a shared and exclusive lock for the same object, this will deduped - * to just a single exclusive lock. - * @param lockObjects - */ - static void dedupLockObjects(List lockObjects) { - Map lockMap = new HashMap(); - for (HiveLockObj lockObj : lockObjects) { - String lockName = lockObj.getName(); - HiveLockObj foundLock = lockMap.get(lockName); - if (foundLock == null || lockObj.getMode() == HiveLockMode.EXCLUSIVE) { - lockMap.put(lockName, lockObj); - } - } - // copy set of deduped locks back to original list - lockObjects.clear(); - for (HiveLockObj lockObj : lockMap.values()) { - lockObjects.add(lockObj); + // Write the current set of valid transactions into the conf file so that it can be read by + // the input format. + private int recordValidTxns() { + try { + IMetaStoreClient.ValidTxnList txns = txnMgr.getValidTxns(); + ctx.getConf().set(IMetaStoreClient.ValidTxnList.VALID_TXNS_KEY, txns.toString()); + return 0; + } catch (LockException e) { + errorMessage = "FAILED: Error in determing valid transactions: " + e.getMessage(); + SQLState = ErrorMsg.findSQLState(e.getMessage()); + downstreamError = e; + console.printError(errorMessage, "\n" + + org.apache.hadoop.util.StringUtils.stringifyException(e)); + return 10; } } @@ -886,96 +821,21 @@ static void dedupLockObjects(List lockObjects) { * pretty simple. If all the locks cannot be obtained, error out. Deadlock is avoided by making * sure that the locks are lexicographically sorted. **/ - public int acquireReadWriteLocks() { + private int acquireReadWriteLocks() { PerfLogger perfLogger = PerfLogger.getPerfLogger(); perfLogger.PerfLogBegin(CLASS_NAME, PerfLogger.ACQUIRE_READ_WRITE_LOCKS); - try { - boolean supportConcurrency = conf.getBoolVar(HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY); - if (!supportConcurrency) { - return 0; - } - - List lockObjects = new ArrayList(); - - // Sort all the inputs, outputs. - // If a lock needs to be acquired on any partition, a read lock needs to be acquired on all - // its parents also - for (ReadEntity input : plan.getInputs()) { - if (input.getType() == ReadEntity.Type.DATABASE) { - lockObjects.addAll(getLockObjects(input.getDatabase(), null, null, HiveLockMode.SHARED)); - } else if (input.getType() == ReadEntity.Type.TABLE) { - lockObjects.addAll(getLockObjects(null, input.getTable(), null, HiveLockMode.SHARED)); - } else { - lockObjects.addAll(getLockObjects(null, null, input.getPartition(), HiveLockMode.SHARED)); - } - } - - for (WriteEntity output : plan.getOutputs()) { - List lockObj = null; - if (output.getType() == WriteEntity.Type.DATABASE) { - lockObjects.addAll(getLockObjects(output.getDatabase(), null, null, - output.isComplete() ? HiveLockMode.EXCLUSIVE : HiveLockMode.SHARED)); - } else if (output.getTyp() == WriteEntity.Type.TABLE) { - lockObj = getLockObjects(null, output.getTable(), null, - output.isComplete() ? HiveLockMode.EXCLUSIVE : HiveLockMode.SHARED); - } else if (output.getTyp() == WriteEntity.Type.PARTITION) { - lockObj = getLockObjects(null, null, output.getPartition(), HiveLockMode.EXCLUSIVE); - } - // In case of dynamic queries, it is possible to have incomplete dummy partitions - else if (output.getTyp() == WriteEntity.Type.DUMMYPARTITION) { - lockObj = getLockObjects(null, null, output.getPartition(), HiveLockMode.SHARED); - } - - if(lockObj != null) { - lockObjects.addAll(lockObj); - ctx.getOutputLockObjects().put(output, lockObj); - } - } - if (lockObjects.isEmpty() && !ctx.isNeedLockMgr()) { - return 0; - } - - HiveLockObjectData lockData = - new HiveLockObjectData(plan.getQueryId(), - String.valueOf(System.currentTimeMillis()), - "IMPLICIT", - plan.getQueryStr()); - - // Lock the database also - String currentDb = SessionState.get().getCurrentDatabase(); - lockObjects.add( - new HiveLockObj( - new HiveLockObject(currentDb, lockData), - HiveLockMode.SHARED - ) - ); - - dedupLockObjects(lockObjects); - List hiveLocks = ctx.getHiveLockMgr().lock(lockObjects, false); - - if (hiveLocks == null) { - throw new SemanticException(ErrorMsg.LOCK_CANNOT_BE_ACQUIRED.getMsg()); - } else { - ctx.setHiveLocks(hiveLocks); - } - - return (0); - } catch (SemanticException e) { - errorMessage = "FAILED: Error in acquiring locks: " + e.getMessage(); - SQLState = ErrorMsg.findSQLState(e.getMessage()); - downstreamError = e; - console.printError(errorMessage, "\n" - + org.apache.hadoop.util.StringUtils.stringifyException(e)); - return (10); + try { + txnMgr.acquireLocks(plan, ctx, userName); + return 0; } catch (LockException e) { errorMessage = "FAILED: Error in acquiring locks: " + e.getMessage(); SQLState = ErrorMsg.findSQLState(e.getMessage()); downstreamError = e; console.printError(errorMessage, "\n" + org.apache.hadoop.util.StringUtils.stringifyException(e)); - return (10); + return 10; } finally { perfLogger.PerfLogEnd(CLASS_NAME, PerfLogger.ACQUIRE_READ_WRITE_LOCKS); } @@ -986,12 +846,12 @@ else if (output.getTyp() == WriteEntity.Type.DUMMYPARTITION) { * list of hive locks to be released Release all the locks specified. If some of the * locks have already been released, ignore them **/ - private void releaseLocks(List hiveLocks) { + private void releaseLocks(List hiveLocks) throws LockException { PerfLogger perfLogger = PerfLogger.getPerfLogger(); perfLogger.PerfLogBegin(CLASS_NAME, PerfLogger.RELEASE_LOCKS); if (hiveLocks != null) { - ctx.getHiveLockMgr().releaseLocks(hiveLocks); + ctx.getHiveTxnManager().getLockManager().releaseLocks(hiveLocks); } ctx.setHiveLocks(null); @@ -1078,7 +938,12 @@ private int compileInternal(String command) { ret = compile(command); } if (ret != 0) { - releaseLocks(ctx.getHiveLocks()); + try { + releaseLocks(ctx.getHiveLocks()); + } catch (LockException e) { + LOG.warn("Exception in releasing locks. " + + org.apache.hadoop.util.StringUtils.stringifyException(e)); + } } return ret; } @@ -1128,6 +993,7 @@ private CommandProcessorResponse runInternal(String command, boolean alreadyComp boolean ckLock = false; try { ckLock = checkConcurrency(); + createTxnManager(); } catch (SemanticException e) { errorMessage = "FAILED: Error in semantic analysis: " + e.getMessage(); SQLState = ErrorMsg.findSQLState(e.getMessage()); @@ -1163,10 +1029,17 @@ private CommandProcessorResponse runInternal(String command, boolean alreadyComp } } + ret = recordValidTxns(); + if (ret != 0) return new CommandProcessorResponse(ret, errorMessage, SQLState); + if (requireLock) { ret = acquireReadWriteLocks(); if (ret != 0) { - releaseLocks(ctx.getHiveLocks()); + try { + releaseLocks(ctx.getHiveLocks()); + } catch (LockException e) { + // Not much to do here + } return new CommandProcessorResponse(ret, errorMessage, SQLState); } } @@ -1174,12 +1047,25 @@ private CommandProcessorResponse runInternal(String command, boolean alreadyComp ret = execute(); if (ret != 0) { //if needRequireLock is false, the release here will do nothing because there is no lock - releaseLocks(ctx.getHiveLocks()); + try { + releaseLocks(ctx.getHiveLocks()); + } catch (LockException e) { + // Nothing to do here + } return new CommandProcessorResponse(ret, errorMessage, SQLState); } //if needRequireLock is false, the release here will do nothing because there is no lock - releaseLocks(ctx.getHiveLocks()); + try { + releaseLocks(ctx.getHiveLocks()); + } catch (LockException e) { + errorMessage = "FAILED: Hive Internal Error: " + Utilities.getNameMessage(e); + SQLState = ErrorMsg.findSQLState(e.getMessage()); + downstreamError = e; + console.printError(errorMessage + "\n" + + org.apache.hadoop.util.StringUtils.stringifyException(e)); + return new CommandProcessorResponse(12, errorMessage, SQLState); + } perfLogger.PerfLogEnd(CLASS_NAME, PerfLogger.DRIVER_RUN); perfLogger.close(LOG, plan); @@ -1731,17 +1617,16 @@ public int close() { public void destroy() { if (ctx != null) { - releaseLocks(ctx.getHiveLocks()); - } - - if (hiveLockMgr != null) { try { - hiveLockMgr.close(); - } catch(LockException e) { - LOG.warn("Exception in closing hive lock manager. " - + org.apache.hadoop.util.StringUtils.stringifyException(e)); + releaseLocks(ctx.getHiveLocks()); + } catch (LockException e) { + LOG.warn("Exception when releasing locking in destroy: " + + e.getMessage()); } } + if (txnMgr != null) { + txnMgr.closeTxnManager(); + } } public org.apache.hadoop.hive.ql.plan.api.Query getQueryPlan() throws IOException { diff --git ql/src/java/org/apache/hadoop/hive/ql/ErrorMsg.java ql/src/java/org/apache/hadoop/hive/ql/ErrorMsg.java index 1ddee49..ecd4c5d 100644 --- ql/src/java/org/apache/hadoop/hive/ql/ErrorMsg.java +++ ql/src/java/org/apache/hadoop/hive/ql/ErrorMsg.java @@ -18,6 +18,11 @@ package org.apache.hadoop.hive.ql; +import org.antlr.runtime.tree.Tree; +import org.apache.hadoop.hive.ql.metadata.HiveUtils; +import org.apache.hadoop.hive.ql.parse.ASTNode; +import org.apache.hadoop.hive.ql.parse.ASTNodeOrigin; + import java.text.MessageFormat; import java.util.HashMap; import java.util.Map; @@ -371,12 +376,35 @@ INVALID_HDFS_URI(10251, "{0} is not a hdfs uri", true), INVALID_DIR(10252, "{0} is not a directory", true), NO_VALID_LOCATIONS(10253, "Could not find any valid location to place the jars. " + - "Please update hive.jar.directory or hive.user.install.directory with a valid location", false), + "Please update hive.jar.directory or hive.user.install.directory with a valid location", false), UNNSUPPORTED_AUTHORIZATION_PRINCIPAL_TYPE_GROUP(10254, "Principal type GROUP is not supported in this authorization setting", "28000"), INVALID_TABLE_NAME(10255, "Invalid table name {0}", true), INSERT_INTO_IMMUTABLE_TABLE(10256, "Inserting into a non-empty immutable table is not allowed"), + TXNMGR_NOT_SPECIFIED(10260, "Transaction manager not specified correctly, " + + "set hive.txn.manager"), + TXNMGR_NOT_INSTANTIATED(10261, "Transaction manager could not be " + + "instantiated, check hive.txn.manager"), + TXN_NO_SUCH_TRANSACTION(10262, "No record of transaction could be found, " + + "may have timed out"), + TXN_ABORTED(10263, "Transaction manager has aborted the transaction."), + + LOCK_NO_SUCH_LOCK(10270, "No record of lock could be found, " + + "may have timed out"), + LOCK_REQUEST_UNSUPPORTED(10271, "Current transaction manager does not " + + "support explicit lock requests. Transaction manager: "), + + METASTORE_COMMUNICATION_FAILED(10280, "Error communicating with the " + + "metastore"), + METASTORE_COULD_NOT_INITIATE(10281, "Unable to initiate connection to the " + + "metastore."), + INVALID_COMPACTION_TYPE(10282, "Invalid compaction type, supported values are 'major' and " + + "'minor'"), + NO_COMPACTION_PARTITION(10283, "You must specify a partition to compact for partitioned tables"), + TOO_MANY_COMPACTION_PARTITIONS(10284, "Compaction can only be requested on one partition at a " + + "time."), + //========================== 20000 range starts here ========================// SCRIPT_INIT_ERROR(20000, "Unable to initialize custom script."), SCRIPT_IO_ERROR(20001, "An error occurred while reading or writing to your custom script. " diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java index 34e8dac..ab6f95d 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java @@ -47,11 +47,8 @@ import org.apache.commons.lang.StringUtils; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.fs.FSDataOutputStream; -import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.*; import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.FsShell; -import org.apache.hadoop.fs.Path; import org.apache.hadoop.hive.common.type.HiveDecimal; import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.conf.HiveConf.ConfVars; @@ -59,24 +56,7 @@ import org.apache.hadoop.hive.metastore.ProtectMode; import org.apache.hadoop.hive.metastore.TableType; import org.apache.hadoop.hive.metastore.Warehouse; -import org.apache.hadoop.hive.metastore.api.AlreadyExistsException; -import org.apache.hadoop.hive.metastore.api.Database; -import org.apache.hadoop.hive.metastore.api.FieldSchema; -import org.apache.hadoop.hive.metastore.api.HiveObjectPrivilege; -import org.apache.hadoop.hive.metastore.api.HiveObjectRef; -import org.apache.hadoop.hive.metastore.api.HiveObjectType; -import org.apache.hadoop.hive.metastore.api.Index; -import org.apache.hadoop.hive.metastore.api.InvalidOperationException; -import org.apache.hadoop.hive.metastore.api.MetaException; -import org.apache.hadoop.hive.metastore.api.NoSuchObjectException; -import org.apache.hadoop.hive.metastore.api.Order; -import org.apache.hadoop.hive.metastore.api.PrincipalType; -import org.apache.hadoop.hive.metastore.api.PrivilegeBag; -import org.apache.hadoop.hive.metastore.api.PrivilegeGrantInfo; -import org.apache.hadoop.hive.metastore.api.Role; -import org.apache.hadoop.hive.metastore.api.SerDeInfo; -import org.apache.hadoop.hive.metastore.api.SkewedInfo; -import org.apache.hadoop.hive.metastore.api.StorageDescriptor; +import org.apache.hadoop.hive.metastore.api.*; import org.apache.hadoop.hive.ql.Context; import org.apache.hadoop.hive.ql.DriverContext; import org.apache.hadoop.hive.ql.ErrorMsg; @@ -88,29 +68,16 @@ import org.apache.hadoop.hive.ql.io.rcfile.merge.MergeWork; import org.apache.hadoop.hive.ql.io.rcfile.truncate.ColumnTruncateTask; import org.apache.hadoop.hive.ql.io.rcfile.truncate.ColumnTruncateWork; -import org.apache.hadoop.hive.ql.lockmgr.HiveLock; -import org.apache.hadoop.hive.ql.lockmgr.HiveLockManager; -import org.apache.hadoop.hive.ql.lockmgr.HiveLockMode; -import org.apache.hadoop.hive.ql.lockmgr.HiveLockObject; +import org.apache.hadoop.hive.ql.lockmgr.*; import org.apache.hadoop.hive.ql.lockmgr.HiveLockObject.HiveLockObjectData; -import org.apache.hadoop.hive.ql.metadata.CheckResult; -import org.apache.hadoop.hive.ql.metadata.Hive; -import org.apache.hadoop.hive.ql.metadata.HiveException; -import org.apache.hadoop.hive.ql.metadata.HiveMetaStoreChecker; -import org.apache.hadoop.hive.ql.metadata.HiveStorageHandler; -import org.apache.hadoop.hive.ql.metadata.HiveUtils; -import org.apache.hadoop.hive.ql.metadata.InvalidTableException; +import org.apache.hadoop.hive.ql.metadata.*; import org.apache.hadoop.hive.ql.metadata.Partition; import org.apache.hadoop.hive.ql.metadata.Table; import org.apache.hadoop.hive.ql.metadata.formatting.MetaDataFormatUtils; import org.apache.hadoop.hive.ql.metadata.formatting.MetaDataFormatter; import org.apache.hadoop.hive.ql.parse.AlterTablePartMergeFilesDesc; import org.apache.hadoop.hive.ql.parse.BaseSemanticAnalyzer; -import org.apache.hadoop.hive.ql.plan.AddPartitionDesc; -import org.apache.hadoop.hive.ql.plan.AlterDatabaseDesc; -import org.apache.hadoop.hive.ql.plan.AlterIndexDesc; -import org.apache.hadoop.hive.ql.plan.AlterTableAlterPartDesc; -import org.apache.hadoop.hive.ql.plan.AlterTableDesc; +import org.apache.hadoop.hive.ql.plan.*; import org.apache.hadoop.hive.ql.plan.AlterTableDesc.AlterTableTypes; import org.apache.hadoop.hive.ql.plan.AlterTableExchangePartition; import org.apache.hadoop.hive.ql.plan.AlterTableSimpleDesc; @@ -1069,7 +1036,7 @@ private int createIndex(Hive db, CreateIndexDesc crtIndex) throws HiveException MetaStoreUtils.getIndexTableName(SessionState.get().getCurrentDatabase(), crtIndex.getTableName(), crtIndex.getIndexName()); Table indexTable = db.getTable(indexTableName); - work.getOutputs().add(new WriteEntity(indexTable)); + work.getOutputs().add(new WriteEntity(indexTable, WriteEntity.WriteType.DDL)); } return 0; } @@ -1162,7 +1129,7 @@ private int alterIndex(Hive db, AlterIndexDesc alterIndex) throws HiveException private int addPartitions(Hive db, AddPartitionDesc addPartitionDesc) throws HiveException { List parts = db.createPartitions(addPartitionDesc); for (Partition part : parts) { - work.getOutputs().add(new WriteEntity(part)); + work.getOutputs().add(new WriteEntity(part, WriteEntity.WriteType.INSERT)); } return 0; } @@ -1188,7 +1155,7 @@ private int renamePartition(Hive db, RenamePartitionDesc renamePartitionDesc) th Partition newPart = db .getPartition(tbl, renamePartitionDesc.getNewPartSpec(), false); work.getInputs().add(new ReadEntity(oldPart)); - work.getOutputs().add(new WriteEntity(newPart)); + work.getOutputs().add(new WriteEntity(newPart, WriteEntity.WriteType.DDL)); return 0; } @@ -1230,7 +1197,7 @@ private int alterTableAlterPart(Hive db, AlterTableAlterPartDesc alterPartitionD } work.getInputs().add(new ReadEntity(tbl)); - work.getOutputs().add(new WriteEntity(tbl)); + work.getOutputs().add(new WriteEntity(tbl, WriteEntity.WriteType.DDL)); return 0; } @@ -1259,7 +1226,7 @@ private int touch(Hive db, AlterTableSimpleDesc touchDesc) throw new HiveException("Uable to update table"); } work.getInputs().add(new ReadEntity(tbl)); - work.getOutputs().add(new WriteEntity(tbl)); + work.getOutputs().add(new WriteEntity(tbl, WriteEntity.WriteType.DDL_METADATA_ONLY)); } else { Partition part = db.getPartition(tbl, touchDesc.getPartSpec(), false); if (part == null) { @@ -1271,7 +1238,7 @@ private int touch(Hive db, AlterTableSimpleDesc touchDesc) throw new HiveException(e); } work.getInputs().add(new ReadEntity(part)); - work.getOutputs().add(new WriteEntity(part)); + work.getOutputs().add(new WriteEntity(part, WriteEntity.WriteType.DDL_METADATA_ONLY)); } return 0; } @@ -2514,7 +2481,11 @@ private int showFunctions(ShowFunctionsDesc showFuncs) throws HiveException { */ private int showLocks(ShowLocksDesc showLocks) throws HiveException { Context ctx = driverContext.getCtx(); - HiveLockManager lockMgr = ctx.getHiveLockMgr(); + HiveTxnManager txnManager = ctx.getHiveTxnManager(); + HiveLockManager lockMgr = txnManager.getLockManager(); + + if (txnManager.useNewShowLocksFormat()) return showLocksNewFormat(showLocks, lockMgr); + boolean isExt = showLocks.isExt(); if (lockMgr == null) { throw new HiveException("show Locks LockManager not specified"); @@ -2529,9 +2500,12 @@ private int showLocks(ShowLocksDesc showLocks) throws HiveException { List locks = null; if (showLocks.getTableName() == null) { + // TODO should be doing security check here. Users should not be + // able to see each other's locks. locks = lockMgr.getLocks(false, isExt); } else { + // TODO make this work locks = lockMgr.getLocks(getHiveObject(showLocks.getTableName(), showLocks.getPartSpec()), true, isExt); @@ -2595,7 +2569,95 @@ public int compare(HiveLock o1, HiveLock o2) { return 0; } - /** + private int showLocksNewFormat(ShowLocksDesc showLocks, HiveLockManager lm) + throws HiveException { + + DbLockManager lockMgr; + if (!(lm instanceof DbLockManager)) { + throw new RuntimeException("New lock format only supported with db lock manager."); + } + lockMgr = (DbLockManager)lm; + + ShowLocksResponse rsp = lockMgr.getLocks(); + + // write the results in the file + DataOutputStream os = null; + try { + Path resFile = new Path(showLocks.getResFile()); + FileSystem fs = resFile.getFileSystem(conf); + os = fs.create(resFile); + + // Write a header + os.writeBytes("Lock ID"); + os.write(separator); + os.writeBytes("Database"); + os.write(separator); + os.writeBytes("Table"); + os.write(separator); + os.writeBytes("Partition"); + os.write(separator); + os.writeBytes("State"); + os.write(separator); + os.writeBytes("Type"); + os.write(separator); + os.writeBytes("Transaction ID"); + os.write(separator); + os.writeBytes("Last Hearbeat"); + os.write(separator); + os.writeBytes("Acquired At"); + os.write(separator); + os.writeBytes("User"); + os.write(separator); + os.writeBytes("Hostname"); + os.write(terminator); + + List locks = rsp.getLocks(); + if (locks != null) { + for (ShowLocksResponseElement lock : locks) { + os.writeBytes(Long.toString(lock.getLockid())); + os.write(separator); + os.writeBytes(lock.getDbname()); + os.write(separator); + os.writeBytes((lock.getTablename() == null) ? "NULL" : lock.getTablename()); + os.write(separator); + os.writeBytes((lock.getPartname() == null) ? "NULL" : lock.getPartname()); + os.write(separator); + os.writeBytes(lock.getState().toString()); + os.write(separator); + os.writeBytes(lock.getType().toString()); + os.write(separator); + os.writeBytes((lock.getTxnid() == 0) ? "NULL" : Long.toString(lock.getTxnid())); + os.write(separator); + os.writeBytes(Long.toString(lock.getLastheartbeat())); + os.write(separator); + os.writeBytes((lock.getAcquiredat() == 0) ? "NULL" : Long.toString(lock.getAcquiredat())); + os.write(separator); + os.writeBytes(lock.getUser()); + os.write(separator); + os.writeBytes(lock.getHostname()); + os.write(separator); + os.write(terminator); + } + + } + + os.close(); + os = null; + } catch (FileNotFoundException e) { + LOG.warn("show function: " + stringifyException(e)); + return 1; + } catch (IOException e) { + LOG.warn("show function: " + stringifyException(e)); + return 1; + } catch (Exception e) { + throw new HiveException(e.toString()); + } finally { + IOUtils.closeStream((FSDataOutputStream) os); + } + return 0; + } + + /** * Lock the table/partition specified * * @param lockTbl @@ -2606,7 +2668,12 @@ public int compare(HiveLock o1, HiveLock o2) { */ private int lockTable(LockTableDesc lockTbl) throws HiveException { Context ctx = driverContext.getCtx(); - HiveLockManager lockMgr = ctx.getHiveLockMgr(); + HiveTxnManager txnManager = ctx.getHiveTxnManager(); + if (!txnManager.supportsExplicitLock()) { + throw new HiveException(ErrorMsg.LOCK_REQUEST_UNSUPPORTED, + conf.getVar(HiveConf.ConfVars.HIVE_TXN_MANAGER)); + } + HiveLockManager lockMgr = txnManager.getLockManager(); if (lockMgr == null) { throw new HiveException("lock Table LockManager not specified"); } @@ -2655,7 +2722,12 @@ private int lockTable(LockTableDesc lockTbl) throws HiveException { */ private int lockDatabase(LockDatabaseDesc lockDb) throws HiveException { Context ctx = driverContext.getCtx(); - HiveLockManager lockMgr = ctx.getHiveLockMgr(); + HiveTxnManager txnManager = ctx.getHiveTxnManager(); + if (!txnManager.supportsExplicitLock()) { + throw new HiveException(ErrorMsg.LOCK_REQUEST_UNSUPPORTED, + conf.getVar(HiveConf.ConfVars.HIVE_TXN_MANAGER)); + } + HiveLockManager lockMgr = txnManager.getLockManager(); if (lockMgr == null) { throw new HiveException("lock Database LockManager not specified"); } @@ -2691,7 +2763,12 @@ private int lockDatabase(LockDatabaseDesc lockDb) throws HiveException { */ private int unlockDatabase(UnlockDatabaseDesc unlockDb) throws HiveException { Context ctx = driverContext.getCtx(); - HiveLockManager lockMgr = ctx.getHiveLockMgr(); + HiveTxnManager txnManager = ctx.getHiveTxnManager(); + if (!txnManager.supportsExplicitLock()) { + throw new HiveException(ErrorMsg.LOCK_REQUEST_UNSUPPORTED, + conf.getVar(HiveConf.ConfVars.HIVE_TXN_MANAGER)); + } + HiveLockManager lockMgr = txnManager.getLockManager(); if (lockMgr == null) { throw new HiveException("unlock Database LockManager not specified"); } @@ -2749,7 +2826,12 @@ private HiveLockObject getHiveObject(String tabName, */ private int unlockTable(UnlockTableDesc unlockTbl) throws HiveException { Context ctx = driverContext.getCtx(); - HiveLockManager lockMgr = ctx.getHiveLockMgr(); + HiveTxnManager txnManager = ctx.getHiveTxnManager(); + if (!txnManager.supportsExplicitLock()) { + throw new HiveException(ErrorMsg.LOCK_REQUEST_UNSUPPORTED, + conf.getVar(HiveConf.ConfVars.HIVE_TXN_MANAGER)); + } + HiveLockManager lockMgr = txnManager.getLockManager(); if (lockMgr == null) { throw new HiveException("unlock Table LockManager not specified"); } @@ -3496,7 +3578,7 @@ private int alterTable(Hive db, AlterTableDesc alterTbl) throws HiveException { } tbl.setNumBuckets(alterTbl.getNumberBuckets()); } - } else { + } else { throw new HiveException(ErrorMsg.UNSUPPORTED_ALTER_TBL_OP, alterTbl.getOp().toString()); } @@ -3533,17 +3615,17 @@ private int alterTable(Hive db, AlterTableDesc alterTbl) throws HiveException { // passed if(part != null) { work.getInputs().add(new ReadEntity(part)); - work.getOutputs().add(new WriteEntity(part)); + work.getOutputs().add(new WriteEntity(part, WriteEntity.WriteType.DDL)); } else if (allPartitions != null ){ for (Partition tmpPart: allPartitions) { work.getInputs().add(new ReadEntity(tmpPart)); - work.getOutputs().add(new WriteEntity(tmpPart)); + work.getOutputs().add(new WriteEntity(tmpPart, WriteEntity.WriteType.DDL)); } } else { work.getInputs().add(new ReadEntity(oldTbl)); - work.getOutputs().add(new WriteEntity(tbl)); + work.getOutputs().add(new WriteEntity(tbl, WriteEntity.WriteType.DDL)); } return 0; } @@ -3581,7 +3663,7 @@ private void dropPartitions(Hive db, Table tbl, DropTableDesc dropTbl) throws Hi dropTbl.getPartSpecs(), true, dropTbl.getIgnoreProtection(), true); for (Partition partition : droppedParts) { console.printInfo("Dropped the partition " + partition.getName()); - work.getOutputs().add(new WriteEntity(partition)); + work.getOutputs().add(new WriteEntity(partition, WriteEntity.WriteType.DDL)); }; } @@ -3635,7 +3717,7 @@ private void dropTable(Hive db, Table tbl, DropTableDesc dropTbl) throws HiveExc // drop the table db.dropTable(dropTbl.getTableName()); if (tbl != null) { - work.getOutputs().add(new WriteEntity(tbl)); + work.getOutputs().add(new WriteEntity(tbl, WriteEntity.WriteType.DDL)); } } @@ -3910,7 +3992,7 @@ private int createTable(Hive db, CreateTableDesc crtTbl) throws HiveException { // create the table db.createTable(tbl, crtTbl.getIfNotExists()); - work.getOutputs().add(new WriteEntity(tbl)); + work.getOutputs().add(new WriteEntity(tbl, WriteEntity.WriteType.DDL)); return 0; } @@ -4018,7 +4100,7 @@ private int createTableLike(Hive db, CreateTableLikeDesc crtTbl) throws HiveExce // create the table db.createTable(tbl, crtTbl.getIfNotExists()); - work.getOutputs().add(new WriteEntity(tbl)); + work.getOutputs().add(new WriteEntity(tbl, WriteEntity.WriteType.DDL)); return 0; } @@ -4054,7 +4136,7 @@ private int createView(Hive db, CreateViewDesc crtView) throws HiveException { } catch (InvalidOperationException e) { throw new HiveException(e); } - work.getOutputs().add(new WriteEntity(oldview)); + work.getOutputs().add(new WriteEntity(oldview, WriteEntity.WriteType.DDL)); } else { // create new view Table tbl = db.newTable(crtView.getViewName()); @@ -4081,7 +4163,7 @@ private int createView(Hive db, CreateViewDesc crtView) throws HiveException { } db.createTable(tbl, crtView.getIfNotExists()); - work.getOutputs().add(new WriteEntity(tbl)); + work.getOutputs().add(new WriteEntity(tbl, WriteEntity.WriteType.DDL)); } return 0; } diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/MoveTask.java ql/src/java/org/apache/hadoop/hive/ql/exec/MoveTask.java index ed7787d..a190155 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/MoveTask.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/MoveTask.java @@ -18,16 +18,6 @@ package org.apache.hadoop.hive.ql.exec; -import java.io.IOException; -import java.io.Serializable; -import java.security.AccessControlException; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.HashSet; -import java.util.LinkedHashMap; -import java.util.List; -import java.util.Map; - import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.fs.FileStatus; @@ -56,17 +46,16 @@ import org.apache.hadoop.hive.ql.optimizer.physical.BucketingSortingCtx.BucketCol; import org.apache.hadoop.hive.ql.optimizer.physical.BucketingSortingCtx.SortCol; import org.apache.hadoop.hive.ql.parse.BaseSemanticAnalyzer; -import org.apache.hadoop.hive.ql.plan.DynamicPartitionCtx; -import org.apache.hadoop.hive.ql.plan.LoadFileDesc; -import org.apache.hadoop.hive.ql.plan.LoadMultiFilesDesc; -import org.apache.hadoop.hive.ql.plan.LoadTableDesc; -import org.apache.hadoop.hive.ql.plan.MapWork; -import org.apache.hadoop.hive.ql.plan.MapredWork; -import org.apache.hadoop.hive.ql.plan.MoveWork; +import org.apache.hadoop.hive.ql.plan.*; import org.apache.hadoop.hive.ql.plan.api.StageType; import org.apache.hadoop.hive.ql.session.SessionState; import org.apache.hadoop.util.StringUtils; +import java.io.IOException; +import java.io.Serializable; +import java.security.AccessControlException; +import java.util.*; + /** * MoveTask implementation. **/ @@ -175,7 +164,7 @@ private void releaseLocks(LoadTableDesc ltd) throws HiveException { } Context ctx = driverContext.getCtx(); - HiveLockManager lockMgr = ctx.getHiveLockMgr(); + HiveLockManager lockMgr = ctx.getHiveTxnManager().getLockManager(); WriteEntity output = ctx.getLoadTableOutputMap().get(ltd); List lockObjects = ctx.getOutputLockObjects().get(output); if (lockObjects == null) { @@ -284,7 +273,9 @@ public int execute(DriverContext driverContext) { db.loadTable(tbd.getSourcePath(), tbd.getTable() .getTableName(), tbd.getReplace(), tbd.getHoldDDLTime()); if (work.getOutputs() != null) { - work.getOutputs().add(new WriteEntity(table)); + work.getOutputs().add(new WriteEntity(table, + (tbd.getReplace() ? WriteEntity.WriteType.INSERT_OVERWRITE : + WriteEntity.WriteType.INSERT))); } } else { LOG.info("Partition is: " + tbd.getPartitionSpec().toString()); @@ -376,7 +367,9 @@ public int execute(DriverContext driverContext) { updatePartitionBucketSortColumns(table, partn, bucketCols, numBuckets, sortCols); } - WriteEntity enty = new WriteEntity(partn); + WriteEntity enty = new WriteEntity(partn, + (tbd.getReplace() ? WriteEntity.WriteType.INSERT_OVERWRITE : + WriteEntity.WriteType.INSERT)); if (work.getOutputs() != null) { work.getOutputs().add(enty); } @@ -417,7 +410,9 @@ public int execute(DriverContext driverContext) { dc = new DataContainer(table.getTTable(), partn.getTPartition()); // add this partition to post-execution hook if (work.getOutputs() != null) { - work.getOutputs().add(new WriteEntity(partn)); + work.getOutputs().add(new WriteEntity(partn, + (tbd.getReplace() ? WriteEntity.WriteType.INSERT_OVERWRITE + : WriteEntity.WriteType.INSERT))); } } } diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java index 5995c14..b966d33 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java @@ -2099,6 +2099,9 @@ public static void copyTableJobPropertiesToConf(TableDesc tbl, JobConf job) { for (Map.Entry entry : jobProperties.entrySet()) { job.set(entry.getKey(), entry.getValue()); } + // copy the bucket count + job.set(hive_metastoreConstants.BUCKET_COUNT, + tbl.getProperties().getProperty(hive_metastoreConstants.BUCKET_COUNT)); } private static final Object INPUT_SUMMARY_LOCK = new Object(); diff --git ql/src/java/org/apache/hadoop/hive/ql/hooks/WriteEntity.java ql/src/java/org/apache/hadoop/hive/ql/hooks/WriteEntity.java index ae8b4bb..44a3924 100644 --- ql/src/java/org/apache/hadoop/hive/ql/hooks/WriteEntity.java +++ ql/src/java/org/apache/hadoop/hive/ql/hooks/WriteEntity.java @@ -18,14 +18,14 @@ package org.apache.hadoop.hive.ql.hooks; -import java.io.Serializable; - import org.apache.hadoop.fs.Path; import org.apache.hadoop.hive.metastore.api.Database; import org.apache.hadoop.hive.ql.metadata.DummyPartition; import org.apache.hadoop.hive.ql.metadata.Partition; import org.apache.hadoop.hive.ql.metadata.Table; +import java.io.Serializable; + /** * This class encapsulates an object that is being written to by the query. This * object may be a table, partition, dfs directory or a local directory. @@ -34,6 +34,18 @@ private boolean isTempURI = false; + public static enum WriteType { + DDL, // for use in DDL statements that will touch data, + // will result in an exclusive lock, + DDL_METADATA_ONLY, // for use in DDL statements that touch only + // metadata and don't need a lock + INSERT, + INSERT_OVERWRITE, + UPDATE, + DELETE}; + + private WriteType writeType; + /** * Only used by serialization. */ @@ -41,8 +53,9 @@ public WriteEntity() { super(); } - public WriteEntity(Database database) { + public WriteEntity(Database database, WriteType type) { super(database, true); + writeType = type; } /** @@ -51,12 +64,14 @@ public WriteEntity(Database database) { * @param t * Table that is written to. */ - public WriteEntity(Table t) { - this(t, true); + public WriteEntity(Table t, WriteType type) { + super(t, true); + writeType = type; } - public WriteEntity(Table t, boolean complete) { + public WriteEntity(Table t, WriteType type, boolean complete) { super(t, complete); + writeType = type; } /** @@ -65,12 +80,14 @@ public WriteEntity(Table t, boolean complete) { * @param p * Partition that is written to. */ - public WriteEntity(Partition p) { + public WriteEntity(Partition p, WriteType type) { super(p, true); + writeType = type; } - public WriteEntity(DummyPartition p, boolean complete) { + public WriteEntity(DummyPartition p, WriteType type, boolean complete) { super(p, complete); + writeType = type; } /** @@ -101,6 +118,15 @@ public WriteEntity(Path d, boolean islocal, boolean isTemp) { } /** + * Determine which type of write this is. This is needed by the lock + * manager so it can understand what kind of lock to acquire. + * @return write type + */ + public WriteType getWriteType() { + return writeType; + } + + /** * Equals function. */ @Override diff --git ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DbLockManager.java ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DbLockManager.java new file mode 100644 index 0000000..535912f --- /dev/null +++ ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DbLockManager.java @@ -0,0 +1,220 @@ +/** + * 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. + */ +package org.apache.hadoop.hive.ql.lockmgr; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hive.metastore.HiveMetaStoreClient; +import org.apache.hadoop.hive.metastore.api.*; +import org.apache.hadoop.hive.ql.ErrorMsg; +import org.apache.thrift.TException; + +import java.util.ArrayList; +import java.util.HashSet; +import java.util.List; +import java.util.Set; + +/** + * An implementation of HiveLockManager for use with {@link org.apache.hadoop.hive.ql.lockmgr.DbTxnManager}. + * Note, this lock manager is not meant to stand alone. It cannot be used + * without the DbTxnManager. + */ +public class DbLockManager implements HiveLockManager{ + + static final private String CLASS_NAME = DbLockManager.class.getName(); + static final private Log LOG = LogFactory.getLog(CLASS_NAME); + + private static final long MAX_SLEEP = 15000; + private HiveLockManagerCtx context; + private Set locks; + private HiveMetaStoreClient client; + private long nextSleep = 50; + + DbLockManager(HiveMetaStoreClient client) { + locks = new HashSet(); + this.client = client; + } + + @Override + public void setContext(HiveLockManagerCtx ctx) throws LockException { + context = ctx; + } + + @Override + public HiveLock lock(HiveLockObject key, HiveLockMode mode, + boolean keepAlive) throws LockException { + throw new UnsupportedOperationException(); + } + + @Override + public List lock(List objs, boolean keepAlive) throws + LockException { + throw new UnsupportedOperationException(); + } + + /** + * Send a lock request to the metastore. This is intended for use by + * {@link DbTxnManager}. + * @param lock lock request + * @throws LockException + */ + List lock(LockRequest lock) throws LockException { + try { + LOG.debug("Requesting lock"); + LockResponse res = client.lock(lock); + while (res.getState() == LockState.WAITING) { + backoff(); + res = client.checkLock(res.getLockid()); + + } + DbHiveLock hl = new DbHiveLock(res.getLockid()); + locks.add(hl); + if (res.getState() != LockState.ACQUIRED) { + throw new LockException(ErrorMsg.LOCK_CANNOT_BE_ACQUIRED.getMsg()); + } + List locks = new ArrayList(1); + locks.add(hl); + return locks; + } catch (NoSuchTxnException e) { + LOG.error("Metastore could not find txnid " + lock.getTxnid()); + throw new LockException(ErrorMsg.TXNMGR_NOT_INSTANTIATED.getMsg(), e); + } catch (TxnAbortedException e) { + LOG.error("Transaction " + lock.getTxnid() + " already aborted."); + throw new LockException(ErrorMsg.TXN_ABORTED.getMsg(), e); + } catch (TException e) { + throw new LockException(ErrorMsg.METASTORE_COMMUNICATION_FAILED.getMsg(), + e); + } + } + + @Override + public void unlock(HiveLock hiveLock) throws LockException { + long lockId = ((DbHiveLock)hiveLock).lockId; + try { + LOG.debug("Unlocking id:" + lockId); + client.unlock(lockId); + boolean removed = locks.remove((DbHiveLock)hiveLock); + LOG.debug("Removed a lock " + removed); + } catch (NoSuchLockException e) { + LOG.error("Metastore could find no record of lock " + lockId); + throw new LockException(ErrorMsg.LOCK_NO_SUCH_LOCK.getMsg(), e); + } catch (TxnOpenException e) { + throw new RuntimeException("Attempt to unlock lock " + lockId + + "associated with an open transaction, " + e.getMessage(), e); + } catch (TException e) { + throw new LockException(ErrorMsg.METASTORE_COMMUNICATION_FAILED.getMsg(), + e); + } + } + + @Override + public void releaseLocks(List hiveLocks) { + for (HiveLock lock : hiveLocks) { + try { + unlock(lock); + } catch (LockException e) { + // Not sure why this method doesn't throw any exceptions, + // but since the interface doesn't allow it we'll just swallow them and + // move on. + } + } + } + + @Override + public List getLocks(boolean verifyTablePartitions, + boolean fetchData) throws LockException { + return new ArrayList(locks); + } + + @Override + public List getLocks(HiveLockObject key, + boolean verifyTablePartitions, + boolean fetchData) throws LockException { + throw new UnsupportedOperationException(); + } + + public ShowLocksResponse getLocks() throws LockException { + try { + return client.showLocks(); + } catch (TException e) { + throw new LockException(ErrorMsg.METASTORE_COMMUNICATION_FAILED.getMsg(), e); + } + } + + @Override + public void close() throws LockException { + for (HiveLock lock : locks) { + unlock(lock); + } + locks.clear(); + } + + @Override + public void prepareRetry() throws LockException { + // NOP + } + + @Override + public void refresh() { + // NOP + } + + static class DbHiveLock extends HiveLock { + + long lockId; + + DbHiveLock(long id) { + lockId = id; + } + + @Override + public HiveLockObject getHiveLockObject() { + throw new UnsupportedOperationException(); + } + + @Override + public HiveLockMode getHiveLockMode() { + throw new UnsupportedOperationException(); + } + + @Override + public boolean equals(Object other) { + if (other instanceof DbHiveLock) { + return lockId == ((DbHiveLock)other).lockId; + } else { + return false; + } + } + + @Override + public int hashCode() { + return (int)(lockId % Integer.MAX_VALUE); + } + } + + // Sleep before we send checkLock again, but do it with a back off + // off so we don't sit and hammer the metastore in a tight loop + private void backoff() { + nextSleep *= 2; + if (nextSleep > MAX_SLEEP) nextSleep = MAX_SLEEP; + try { + Thread.sleep(nextSleep); + } catch (InterruptedException e) { + } + } +} diff --git ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DbTxnManager.java ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DbTxnManager.java new file mode 100644 index 0000000..7773f66 --- /dev/null +++ ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DbTxnManager.java @@ -0,0 +1,317 @@ +/** + * 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. + */ +package org.apache.hadoop.hive.ql.lockmgr; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hive.metastore.HiveMetaStoreClient; +import org.apache.hadoop.hive.metastore.IMetaStoreClient; +import org.apache.hadoop.hive.metastore.LockComponentBuilder; +import org.apache.hadoop.hive.metastore.LockRequestBuilder; +import org.apache.hadoop.hive.metastore.api.*; +import org.apache.hadoop.hive.ql.Context; +import org.apache.hadoop.hive.ql.ErrorMsg; +import org.apache.hadoop.hive.ql.QueryPlan; +import org.apache.hadoop.hive.ql.hooks.Entity; +import org.apache.hadoop.hive.ql.hooks.ReadEntity; +import org.apache.hadoop.hive.ql.hooks.WriteEntity; +import org.apache.hadoop.hive.ql.metadata.Table; +import org.apache.thrift.TException; + +import java.util.List; + +/** + * An implementation of HiveTxnManager that stores the transactions in the + * metastore database. + */ +public class DbTxnManager extends HiveTxnManagerImpl { + + static final private String CLASS_NAME = DbTxnManager.class.getName(); + static final private Log LOG = LogFactory.getLog(CLASS_NAME); + + private DbLockManager lockMgr = null; + private HiveMetaStoreClient client = null; + private long txnId = 0; + + DbTxnManager() { + } + + @Override + public void openTxn(String user) throws LockException { + init(); + try { + txnId = client.openTxn(user); + LOG.debug("Opened txn " + txnId); + } catch (TException e) { + throw new LockException(ErrorMsg.METASTORE_COMMUNICATION_FAILED.getMsg(), + e); + } + } + + @Override + public HiveLockManager getLockManager() throws LockException { + init(); + if (lockMgr == null) { + lockMgr = new DbLockManager(client); + } + return lockMgr; + } + + @Override + public void acquireLocks(QueryPlan plan, Context ctx, String username) throws LockException { + init(); + // Make sure we've built the lock manager + getLockManager(); + + boolean atLeastOneLock = false; + + LockRequestBuilder rqstBuilder = new LockRequestBuilder(); + LOG.debug("Setting lock request transaction to " + txnId); + rqstBuilder.setTransactionId(txnId) + .setUser(username); + + // For each source to read, get a shared lock + for (ReadEntity input : plan.getInputs()) { + LockComponentBuilder compBuilder = new LockComponentBuilder(); + compBuilder.setShared(); + + Table t = null; + switch (input.getType()) { + case DATABASE: + compBuilder.setDbName(input.getDatabase().getName()); + break; + + case TABLE: + t = input.getTable(); + compBuilder.setDbName(t.getDbName()); + compBuilder.setTableName(t.getTableName()); + break; + + case PARTITION: + case DUMMYPARTITION: + compBuilder.setPartitionName(input.getPartition().getName()); + t = input.getPartition().getTable(); + compBuilder.setDbName(t.getDbName()); + compBuilder.setTableName(t.getTableName()); + break; + + default: + // This is a file or something we don't hold locks for. + continue; + } + LockComponent comp = compBuilder.build(); + LOG.debug("Adding lock component to lock request " + comp.toString()); + rqstBuilder.addLockComponent(comp); + atLeastOneLock = true; + } + + // For each source to write to, get the appropriate lock type. If it's + // an OVERWRITE, we need to get an exclusive lock. If it's an insert (no + // overwrite) than we need a shared. If it's update or delete then we + // need a SEMI-SHARED. + for (WriteEntity output : plan.getOutputs()) { + if (output.getType() == Entity.Type.DFS_DIR || output.getType() == + Entity.Type.LOCAL_DIR) { + // We don't lock files or directories. + continue; + } + LockComponentBuilder compBuilder = new LockComponentBuilder(); + Table t = null; + LOG.debug("output is null " + (output == null)); + switch (output.getWriteType()) { + case DDL: + case INSERT_OVERWRITE: + compBuilder.setExclusive(); + break; + + case INSERT: + case DDL_METADATA_ONLY: + compBuilder.setShared(); + break; + + case UPDATE: + case DELETE: + compBuilder.setSemiShared(); + break; + + default: + throw new RuntimeException("Unknown write type " + + output.getWriteType().toString()); + + } + switch (output.getType()) { + case DATABASE: + compBuilder.setDbName(output.getDatabase().getName()); + break; + + case TABLE: + t = output.getTable(); + compBuilder.setDbName(t.getDbName()); + compBuilder.setTableName(t.getTableName()); + break; + + case PARTITION: + case DUMMYPARTITION: + compBuilder.setPartitionName(output.getPartition().getName()); + t = output.getPartition().getTable(); + compBuilder.setDbName(t.getDbName()); + compBuilder.setTableName(t.getTableName()); + break; + + default: + // This is a file or something we don't hold locks for. + continue; + } + LockComponent comp = compBuilder.build(); + LOG.debug("Adding lock component to lock request " + comp.toString()); + rqstBuilder.addLockComponent(comp); + atLeastOneLock = true; + } + + // Make sure we need locks. It's possible there's nothing to lock in + // this operation. + if (!atLeastOneLock) return; + + List locks = lockMgr.lock(rqstBuilder.build()); + ctx.setHiveLocks(locks); + } + + @Override + public void commitTxn() throws LockException { + if (txnId == 0) { + throw new RuntimeException("Attempt to commit before opening a " + + "transaction"); + } + try { + LOG.debug("Committing txn " + txnId); + client.commitTxn(txnId); + } catch (NoSuchTxnException e) { + LOG.error("Metastore could not find txn " + txnId); + throw new LockException(ErrorMsg.TXN_NO_SUCH_TRANSACTION.getMsg() , e); + } catch (TxnAbortedException e) { + LOG.error("Transaction " + txnId + " aborted"); + throw new LockException(ErrorMsg.TXN_ABORTED.getMsg(), e); + } catch (TException e) { + throw new LockException(ErrorMsg.METASTORE_COMMUNICATION_FAILED.getMsg(), + e); + } finally { + txnId = 0; + } + } + + @Override + public void rollbackTxn() throws LockException { + if (txnId == 0) { + throw new RuntimeException("Attempt to rollback before opening a " + + "transaction"); + } + try { + LOG.debug("Rolling back txn " + txnId); + client.rollbackTxn(txnId); + } catch (NoSuchTxnException e) { + LOG.error("Metastore could not find txn " + txnId); + throw new LockException(ErrorMsg.TXN_NO_SUCH_TRANSACTION.getMsg() , e); + } catch (TException e) { + throw new LockException(ErrorMsg.METASTORE_COMMUNICATION_FAILED.getMsg(), + e); + } finally { + txnId = 0; + } + } + + @Override + public void heartbeat() throws LockException { + LOG.debug("Heartbeating lock and transaction " + txnId); + List locks = lockMgr.getLocks(false, false); + if (locks.size() == 0) { + if (txnId == 0) { + // No locks, no txn, we outta here. + return; + } else { + // Create one dummy lock so we can go through the loop below + DbLockManager.DbHiveLock dummyLock = new DbLockManager.DbHiveLock(0L); + locks.add(dummyLock); + } + } + for (HiveLock lock : locks) { + long lockId = ((DbLockManager.DbHiveLock)lock).lockId; + try { + client.heartbeat(txnId, lockId); + } catch (NoSuchLockException e) { + LOG.error("Unable to find lock " + lockId); + throw new LockException(ErrorMsg.LOCK_NO_SUCH_LOCK.getMsg(), e); + } catch (NoSuchTxnException e) { + LOG.error("Unable to find transaction " + txnId); + throw new LockException(ErrorMsg.TXN_NO_SUCH_TRANSACTION.getMsg(), e); + } catch (TxnAbortedException e) { + LOG.error("Transaction aborted " + txnId); + throw new LockException(ErrorMsg.TXN_ABORTED.getMsg(), e); + } catch (TException e) { + throw new LockException( + ErrorMsg.METASTORE_COMMUNICATION_FAILED.getMsg(), e); + } + } + } + + @Override + public IMetaStoreClient.ValidTxnList getValidTxns() throws LockException { + init(); + try { + return client.getValidTxns(); + } catch (TException e) { + throw new LockException(ErrorMsg.METASTORE_COMMUNICATION_FAILED.getMsg(), + e); + } + } + + @Override + public boolean supportsExplicitLock() { + return false; + } + + @Override + public boolean useNewShowLocksFormat() { + return true; + } + + @Override + protected void destruct() { + try { + if (txnId > 0) rollbackTxn(); + if (lockMgr != null) lockMgr.close(); + } catch (Exception e) { + // Not much we can do about it here. + } + } + + private void init() throws LockException { + if (client == null) { + if (conf == null) { + throw new RuntimeException("Must call setHiveConf before any other " + + "methods."); + } + try { + client = new HiveMetaStoreClient(conf); + } catch (MetaException e) { + throw new LockException(ErrorMsg.METASTORE_COULD_NOT_INITIATE.getMsg(), + e); + } + } + } + +} diff --git ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DummyTxnManager.java ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DummyTxnManager.java new file mode 100644 index 0000000..be8784b --- /dev/null +++ ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DummyTxnManager.java @@ -0,0 +1,343 @@ +/** + * 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. + */ +package org.apache.hadoop.hive.ql.lockmgr; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.metastore.IMetaStoreClient; +import org.apache.hadoop.hive.metastore.api.Database; +import org.apache.hadoop.hive.metastore.api.GetOpenTxnsResponse; +import org.apache.hadoop.hive.ql.Context; +import org.apache.hadoop.hive.ql.ErrorMsg; +import org.apache.hadoop.hive.ql.QueryPlan; +import org.apache.hadoop.hive.ql.exec.Utilities; +import org.apache.hadoop.hive.ql.hooks.ReadEntity; +import org.apache.hadoop.hive.ql.hooks.WriteEntity; +import org.apache.hadoop.hive.ql.metadata.DummyPartition; +import org.apache.hadoop.hive.ql.metadata.HiveException; +import org.apache.hadoop.hive.ql.metadata.Partition; +import org.apache.hadoop.hive.ql.metadata.Table; +import org.apache.hadoop.hive.ql.session.SessionState; +import org.apache.hadoop.util.ReflectionUtils; + +import java.util.*; + +/** + * An implementation of {@link HiveTxnManager} that does not support + * transactions. This provides default Hive behavior. + */ +class DummyTxnManager extends HiveTxnManagerImpl { + private static final int separator = Utilities.tabCode; + private static final int terminator = Utilities.newLineCode; + static final private Log LOG = + LogFactory.getLog(DummyTxnManager.class.getName()); + + private HiveLockManager lockMgr; + + @Override + public void openTxn(String user) throws LockException { + // No-op + } + + @Override + public HiveLockManager getLockManager() throws LockException { + if (lockMgr == null) { + boolean supportConcurrency = + conf.getBoolVar(HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY); + if (supportConcurrency) { + String lockMgrName = + conf.getVar(HiveConf.ConfVars.HIVE_LOCK_MANAGER); + if ((lockMgrName == null) || (lockMgrName.isEmpty())) { + throw new LockException(ErrorMsg.LOCKMGR_NOT_SPECIFIED.getMsg()); + } + + try { + LOG.info("Creating lock manager of type " + lockMgrName); + lockMgr = (HiveLockManager)ReflectionUtils.newInstance( + conf.getClassByName(lockMgrName), conf); + lockMgr.setContext(new HiveLockManagerCtx(conf)); + } catch (Exception e) { + // set hiveLockMgr to null just in case this invalid manager got set to + // next query's ctx. + if (lockMgr != null) { + try { + lockMgr.close(); + } catch (LockException e1) { + //nothing can do here + } + lockMgr = null; + } + throw new LockException(ErrorMsg.LOCKMGR_NOT_INITIALIZED.getMsg() + + e.getMessage()); + } + } else { + LOG.info("Concurrency mode is disabled, not creating a lock manager"); + return null; + } + } + // Force a re-read of the configuration file. This is done because + // different queries in the session may be using the same lock manager. + lockMgr.refresh(); + return lockMgr; + } + + @Override + public void acquireLocks(QueryPlan plan, Context ctx, String username) throws LockException { + // Make sure we've built the lock manager + getLockManager(); + + // If the lock manager is still null, then it means we aren't using a + // lock manager + if (lockMgr == null) return; + + List lockObjects = new ArrayList(); + + // Sort all the inputs, outputs. + // If a lock needs to be acquired on any partition, a read lock needs to be acquired on all + // its parents also + for (ReadEntity input : plan.getInputs()) { + LOG.debug("Adding " + input.getName() + " to list of lock inputs"); + if (input.getType() == ReadEntity.Type.DATABASE) { + lockObjects.addAll(getLockObjects(plan, input.getDatabase(), null, + null, HiveLockMode.SHARED)); + } else if (input.getType() == ReadEntity.Type.TABLE) { + lockObjects.addAll(getLockObjects(plan, null, input.getTable(), null, + HiveLockMode.SHARED)); + } else { + lockObjects.addAll(getLockObjects(plan, null, null, + input.getPartition(), + HiveLockMode.SHARED)); + } + } + + for (WriteEntity output : plan.getOutputs()) { + LOG.debug("Adding " + output.getName() + " to list of lock outputs"); + List lockObj = null; + if (output.getType() == WriteEntity.Type.DATABASE) { + lockObjects.addAll(getLockObjects(plan, output.getDatabase(), null, + null, + output.isComplete() ? HiveLockMode.EXCLUSIVE : HiveLockMode.SHARED)); + } else if (output.getTyp() == WriteEntity.Type.TABLE) { + lockObj = getLockObjects(plan, null, output.getTable(), null, + output.isComplete() ? HiveLockMode.EXCLUSIVE : HiveLockMode.SHARED); + } else if (output.getTyp() == WriteEntity.Type.PARTITION) { + lockObj = getLockObjects(plan, null, null, output.getPartition(), + HiveLockMode.EXCLUSIVE); + } + // In case of dynamic queries, it is possible to have incomplete dummy partitions + else if (output.getTyp() == WriteEntity.Type.DUMMYPARTITION) { + lockObj = getLockObjects(plan, null, null, output.getPartition(), + HiveLockMode.SHARED); + } + + if(lockObj != null) { + lockObjects.addAll(lockObj); + ctx.getOutputLockObjects().put(output, lockObj); + } + } + + if (lockObjects.isEmpty() && !ctx.isNeedLockMgr()) { + return; + } + + HiveLockObject.HiveLockObjectData lockData = + new HiveLockObject.HiveLockObjectData(plan.getQueryId(), + String.valueOf(System.currentTimeMillis()), + "IMPLICIT", + plan.getQueryStr()); + + // Lock the database also + String currentDb = SessionState.get().getCurrentDatabase(); + lockObjects.add( + new HiveLockObj( + new HiveLockObject(currentDb, lockData), + HiveLockMode.SHARED + ) + ); + + dedupLockObjects(lockObjects); + List hiveLocks = lockMgr.lock(lockObjects, false); + + if (hiveLocks == null) { + throw new LockException(ErrorMsg.LOCK_CANNOT_BE_ACQUIRED.getMsg()); + } else { + ctx.setHiveLocks(hiveLocks); + } + } + + @Override + public void commitTxn() throws LockException { + // No-op + } + + @Override + public void rollbackTxn() throws LockException { + // No-op + } + + @Override + public void heartbeat() throws LockException { + // No-op + } + + @Override + public IMetaStoreClient.ValidTxnList getValidTxns() throws LockException { + return new IMetaStoreClient.ValidTxnList() { + @Override + public boolean isTxnCommitted(long txnid) { + return false; + } + + @Override + public RangeResponse isTxnRangeCommitted(long min, long max) { + return RangeResponse.NONE; + } + + @Override + public GetOpenTxnsResponse getOpenTxns() { + return null; + } + + @Override + public void fromString(String src) { + } + + @Override + public String toString() { + return ""; + } + + }; + } + + @Override + public boolean supportsExplicitLock() { + return true; + } + + @Override + public boolean useNewShowLocksFormat() { + return false; + } + + + protected void destruct() { + if (lockMgr != null) { + try { + lockMgr.close(); + } catch (LockException e) { + // Not much I can do about it. + LOG.warn("Got exception when closing lock manager " + e.getMessage()); + } + } + } + + /** + * Dedup the list of lock objects so that there is only one lock per table/partition. + * If there is both a shared and exclusive lock for the same object, this will deduped + * to just a single exclusive lock. Package level so that the unit tests + * can access it. Not intended for use outside this class. + * @param lockObjects + */ + static void dedupLockObjects(List lockObjects) { + Map lockMap = new HashMap(); + for (HiveLockObj lockObj : lockObjects) { + String lockName = lockObj.getName(); + HiveLockObj foundLock = lockMap.get(lockName); + if (foundLock == null || lockObj.getMode() == HiveLockMode.EXCLUSIVE) { + lockMap.put(lockName, lockObj); + } + } + // copy set of deduped locks back to original list + lockObjects.clear(); + for (HiveLockObj lockObj : lockMap.values()) { + lockObjects.add(lockObj); + } + } + + private List getLockObjects(QueryPlan plan, Database db, + Table t, Partition p, + HiveLockMode mode) + throws LockException { + List locks = new LinkedList(); + + HiveLockObject.HiveLockObjectData lockData = + new HiveLockObject.HiveLockObjectData(plan.getQueryId(), + String.valueOf(System.currentTimeMillis()), + "IMPLICIT", + plan.getQueryStr()); + + if (db != null) { + locks.add(new HiveLockObj(new HiveLockObject(db.getName(), lockData), + mode)); + return locks; + } + + if (t != null) { + locks.add(new HiveLockObj(new HiveLockObject(t, lockData), mode)); + mode = HiveLockMode.SHARED; + locks.add(new HiveLockObj(new HiveLockObject(t.getDbName(), lockData), mode)); + return locks; + } + + if (p != null) { + if (!(p instanceof DummyPartition)) { + locks.add(new HiveLockObj(new HiveLockObject(p, lockData), mode)); + } + + // All the parents are locked in shared mode + mode = HiveLockMode.SHARED; + + // For dummy partitions, only partition name is needed + String name = p.getName(); + + if (p instanceof DummyPartition) { + name = p.getName().split("@")[2]; + } + + String partialName = ""; + String[] partns = name.split("/"); + int len = p instanceof DummyPartition ? partns.length : partns.length - 1; + Map partialSpec = new LinkedHashMap(); + for (int idx = 0; idx < len; idx++) { + String partn = partns[idx]; + partialName += partn; + String[] nameValue = partn.split("="); + assert(nameValue.length == 2); + partialSpec.put(nameValue[0], nameValue[1]); + try { + locks.add(new HiveLockObj( + new HiveLockObject(new DummyPartition(p.getTable(), p.getTable().getDbName() + + "/" + p.getTable().getTableName() + + "/" + partialName, + partialSpec), lockData), mode)); + partialName += "/"; + } catch (HiveException e) { + throw new LockException(e.getMessage()); + } + } + + locks.add(new HiveLockObj(new HiveLockObject(p.getTable(), lockData), mode)); + locks.add(new HiveLockObj(new HiveLockObject(p.getTable().getDbName(), lockData), mode)); + } + return locks; + } + + +} diff --git ql/src/java/org/apache/hadoop/hive/ql/lockmgr/EmbeddedLockManager.java ql/src/java/org/apache/hadoop/hive/ql/lockmgr/EmbeddedLockManager.java index 9550001..11434a0 100644 --- ql/src/java/org/apache/hadoop/hive/ql/lockmgr/EmbeddedLockManager.java +++ ql/src/java/org/apache/hadoop/hive/ql/lockmgr/EmbeddedLockManager.java @@ -18,24 +18,14 @@ package org.apache.hadoop.hive.ql.lockmgr; -import java.util.ArrayList; -import java.util.Collections; -import java.util.Comparator; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Stack; -import java.util.concurrent.locks.ReentrantLock; - import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.ql.lockmgr.HiveLockObject.HiveLockObjectData; -import org.apache.hadoop.hive.ql.metadata.DummyPartition; -import org.apache.hadoop.hive.ql.metadata.Hive; -import org.apache.hadoop.hive.ql.metadata.HiveException; -import org.apache.hadoop.hive.ql.metadata.Partition; -import org.apache.hadoop.hive.ql.metadata.Table; +import org.apache.hadoop.hive.ql.metadata.*; + +import java.util.*; +import java.util.concurrent.locks.ReentrantLock; /** * shared lock manager for dedicated hive server. all locks are managed in memory diff --git ql/src/java/org/apache/hadoop/hive/ql/lockmgr/HiveLockManager.java ql/src/java/org/apache/hadoop/hive/ql/lockmgr/HiveLockManager.java index eb15993..b2eb997 100644 --- ql/src/java/org/apache/hadoop/hive/ql/lockmgr/HiveLockManager.java +++ ql/src/java/org/apache/hadoop/hive/ql/lockmgr/HiveLockManager.java @@ -20,6 +20,11 @@ import java.util.List; +/** + * Manager for locks in Hive. Users should not instantiate a lock manager + * directly. Instead they should get an instance from their instance of + * {@link HiveTxnManager}. + */ public interface HiveLockManager { public void setContext(HiveLockManagerCtx ctx) throws LockException; diff --git ql/src/java/org/apache/hadoop/hive/ql/lockmgr/HiveLockMode.java ql/src/java/org/apache/hadoop/hive/ql/lockmgr/HiveLockMode.java index a8fbf26..37af243 100644 --- ql/src/java/org/apache/hadoop/hive/ql/lockmgr/HiveLockMode.java +++ ql/src/java/org/apache/hadoop/hive/ql/lockmgr/HiveLockMode.java @@ -19,6 +19,8 @@ package org.apache.hadoop.hive.ql.lockmgr; public enum HiveLockMode { - SHARED, EXCLUSIVE; + SHARED, + EXCLUSIVE, + SEMI_SHARED; // SEMI_SHARED can share with SHARED but not with itself } diff --git ql/src/java/org/apache/hadoop/hive/ql/lockmgr/HiveTxnManager.java ql/src/java/org/apache/hadoop/hive/ql/lockmgr/HiveTxnManager.java new file mode 100644 index 0000000..6f92a4e --- /dev/null +++ ql/src/java/org/apache/hadoop/hive/ql/lockmgr/HiveTxnManager.java @@ -0,0 +1,122 @@ +/** + * 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. + */ +package org.apache.hadoop.hive.ql.lockmgr; + +import org.apache.hadoop.hive.metastore.IMetaStoreClient; +import org.apache.hadoop.hive.ql.Context; +import org.apache.hadoop.hive.ql.QueryPlan; + +/** + * An interface that allows Hive to manage transactions. All classes + * implementing this should extend {@link HiveTxnManagerImpl} rather than + * implementing this directly. + */ +public interface HiveTxnManager { + + /** + * Open a new transaction. + * @param user Hive user who is opening this transaction. + * @throws LockException if a transaction is already open. + */ + void openTxn(String user) throws LockException; + + /** + * Get the lock manager. This must be used rather than instantiating an + * instance of the lock manager directly as the transaction manager will + * choose which lock manager to instantiate. + * @return the instance of the lock manager + * @throws LockException if there is an issue obtaining the lock manager. + */ + HiveLockManager getLockManager() throws LockException; + + /** + * Acquire all of the locks needed by a query. If used with a query that + * requires transactions, this should be called after {@link #openTxn(String)}. + * A list of acquired locks will be stored in the + * {@link org.apache.hadoop.hive.ql.Context} object and can be retrieved + * via {@link org.apache.hadoop.hive.ql.Context#getHiveLocks}. + * @param plan query plan + * @param ctx Context for this query + * @param username name of the user for this query + * @throws LockException if there is an error getting the locks + */ + void acquireLocks(QueryPlan plan, Context ctx, String username) throws LockException; + + /** + * Commit the current transaction. This will release all locks obtained in + * {@link #acquireLocks(org.apache.hadoop.hive.ql.QueryPlan, + * org.apache.hadoop.hive.ql.Context, java.lang.String)}. + * @throws LockException if there is no current transaction or the + * transaction has already been committed or aborted. + */ + void commitTxn() throws LockException; + + /** + * Abort the current transaction. This will release all locks obtained in + * {@link #acquireLocks(org.apache.hadoop.hive.ql.QueryPlan, + * org.apache.hadoop.hive.ql.Context, java.lang.String)}. + * @throws LockException if there is no current transaction or the + * transaction has already been committed or aborted. + */ + void rollbackTxn() throws LockException; + + /** + * Send a heartbeat to the transaction management storage so other Hive + * clients know that the transaction and locks held by this client are + * still valid. For implementations that do not require heartbeats this + * can be a no-op. + * @throws LockException If current transaction exists or the transaction + * has already been committed or aborted. + */ + void heartbeat() throws LockException; + + /** + * Get the transactions that are currently valid. The resulting + * {@link IMetaStoreClient.ValidTxnList} object is a thrift object and can + * be passed to the processing + * tasks for use in the reading the data. This call should be made once up + * front by the planner and should never be called on the backend, + * as this will violate the isolation level semantics. + * @return list of valid transactions. + * @throws LockException + */ + IMetaStoreClient.ValidTxnList getValidTxns() throws LockException; + + /** + * This call closes down the transaction manager. All open transactions + * are aborted. If no transactions are open but locks are held those locks + * are released. This method should be called if processing of a session + * is being halted in an abnormal way. It avoids locks and transactions + * timing out. + */ + void closeTxnManager(); + + /** + * Indicate whether this lock manager supports the use of lock + * database or lock table. + * @return + */ + boolean supportsExplicitLock(); + + /** + * Indicate whether this transaction manager returns information about locks in the new format + * for show locks or the old one. + * @return true if the new format should be used. + */ + boolean useNewShowLocksFormat(); +} diff --git ql/src/java/org/apache/hadoop/hive/ql/lockmgr/HiveTxnManagerImpl.java ql/src/java/org/apache/hadoop/hive/ql/lockmgr/HiveTxnManagerImpl.java new file mode 100644 index 0000000..eccb8d1 --- /dev/null +++ ql/src/java/org/apache/hadoop/hive/ql/lockmgr/HiveTxnManagerImpl.java @@ -0,0 +1,47 @@ +/** + * 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. + */ +package org.apache.hadoop.hive.ql.lockmgr; + +import org.apache.hadoop.hive.conf.HiveConf; + +/** + * An implementation HiveTxnManager that includes internal methods that all + * transaction managers need to implement but that we don't want to expose to + * outside. + */ +abstract class HiveTxnManagerImpl implements HiveTxnManager { + + protected HiveConf conf; + + void setHiveConf(HiveConf c) { + conf = c; + } + + abstract protected void destruct(); + + @Override + public void closeTxnManager() { + destruct(); + } + + @Override + protected void finalize() throws Throwable { + destruct(); + } + +} diff --git ql/src/java/org/apache/hadoop/hive/ql/lockmgr/TxnManagerFactory.java ql/src/java/org/apache/hadoop/hive/ql/lockmgr/TxnManagerFactory.java new file mode 100644 index 0000000..4d616d0 --- /dev/null +++ ql/src/java/org/apache/hadoop/hive/ql/lockmgr/TxnManagerFactory.java @@ -0,0 +1,82 @@ +/** + * 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. + */ +package org.apache.hadoop.hive.ql.lockmgr; + +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.ql.ErrorMsg; +import org.apache.hadoop.util.ReflectionUtils; + +/** + * A factory to get an instance of {@link HiveTxnManager}. This should + * always be called rather than building a transaction manager via reflection. + * This factory will read the configuration file to determine which + * transaction manager to instantiate. It will stash the chosen transaction + * manager into the Context object, and subsequently return it from there so + * that if there are multiple Hive threads running, + * each will get it's appropriate transaction manager. + */ +public class TxnManagerFactory { + + private static TxnManagerFactory self; + + /** + * Get the singleton instance of this factory. + * @return this factory + */ + public static synchronized TxnManagerFactory getTxnManagerFactory() { + if (self == null) { + self = new TxnManagerFactory(); + } + return self; + } + + /** + * Create a new transaction manager. The transaction manager to + * instantiate will be determined by the hive.txn.manager value in the + * configuration. This should not be called if a transaction manager has + * already been constructed and stored in the Context object. + * @param conf HiveConf object used to construct the transaction manager + * @return the transaction manager + * @throws LockException if there is an error constructing the transaction + * manager. + */ + public HiveTxnManager getTxnManager(HiveConf conf) throws + LockException { + HiveTxnManager txnMgr = null; + + // Determine the transaction manager to use from the configuration. + String txnMgrName = conf.getVar(HiveConf.ConfVars.HIVE_TXN_MANAGER); + if (txnMgrName == null || txnMgrName.isEmpty()) { + throw new LockException(ErrorMsg.TXNMGR_NOT_SPECIFIED.getMsg()); + } + + // Instantiate the chosen transaction manager + try { + HiveTxnManagerImpl impl = (HiveTxnManagerImpl)ReflectionUtils.newInstance( + conf.getClassByName(txnMgrName), conf); + impl.setHiveConf(conf); + txnMgr = impl; + } catch (ClassNotFoundException e) { + throw new LockException(ErrorMsg.TXNMGR_NOT_INSTANTIATED.getMsg()); + } + return txnMgr; + } + + private TxnManagerFactory() { + } +} diff --git ql/src/java/org/apache/hadoop/hive/ql/lockmgr/zookeeper/ZooKeeperHiveLockManager.java ql/src/java/org/apache/hadoop/hive/ql/lockmgr/zookeeper/ZooKeeperHiveLockManager.java index 093be45..65b9136 100644 --- ql/src/java/org/apache/hadoop/hive/ql/lockmgr/zookeeper/ZooKeeperHiveLockManager.java +++ ql/src/java/org/apache/hadoop/hive/ql/lockmgr/zookeeper/ZooKeeperHiveLockManager.java @@ -18,36 +18,14 @@ package org.apache.hadoop.hive.ql.lockmgr.zookeeper; -import java.io.IOException; -import java.net.InetAddress; -import java.util.ArrayList; -import java.util.Collections; -import java.util.Comparator; -import java.util.HashMap; -import java.util.LinkedList; -import java.util.List; -import java.util.Map; -import java.util.Queue; -import java.util.regex.Matcher; -import java.util.regex.Pattern; - +import com.google.common.annotations.VisibleForTesting; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.ql.ErrorMsg; -import org.apache.hadoop.hive.ql.lockmgr.HiveLock; -import org.apache.hadoop.hive.ql.lockmgr.HiveLockManager; -import org.apache.hadoop.hive.ql.lockmgr.HiveLockManagerCtx; -import org.apache.hadoop.hive.ql.lockmgr.HiveLockMode; -import org.apache.hadoop.hive.ql.lockmgr.HiveLockObj; -import org.apache.hadoop.hive.ql.lockmgr.HiveLockObject; +import org.apache.hadoop.hive.ql.lockmgr.*; import org.apache.hadoop.hive.ql.lockmgr.HiveLockObject.HiveLockObjectData; -import org.apache.hadoop.hive.ql.lockmgr.LockException; -import org.apache.hadoop.hive.ql.metadata.DummyPartition; -import org.apache.hadoop.hive.ql.metadata.Hive; -import org.apache.hadoop.hive.ql.metadata.HiveException; -import org.apache.hadoop.hive.ql.metadata.Partition; -import org.apache.hadoop.hive.ql.metadata.Table; +import org.apache.hadoop.hive.ql.metadata.*; import org.apache.hadoop.hive.ql.session.SessionState.LogHelper; import org.apache.zookeeper.CreateMode; import org.apache.zookeeper.KeeperException; @@ -55,7 +33,11 @@ import org.apache.zookeeper.ZooDefs.Ids; import org.apache.zookeeper.ZooKeeper; -import com.google.common.annotations.VisibleForTesting; +import java.io.IOException; +import java.net.InetAddress; +import java.util.*; +import java.util.regex.Matcher; +import java.util.regex.Pattern; public class ZooKeeperHiveLockManager implements HiveLockManager { HiveLockManagerCtx ctx; diff --git ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java index a3af5f3..1713e9d 100644 --- ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java +++ ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java @@ -70,9 +70,10 @@ import org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat; import org.apache.hadoop.hive.ql.io.RCFileInputFormat; import org.apache.hadoop.hive.ql.lib.Node; -import org.apache.hadoop.hive.ql.metadata.Hive; -import org.apache.hadoop.hive.ql.metadata.HiveException; -import org.apache.hadoop.hive.ql.metadata.HiveUtils; +import org.apache.hadoop.hive.ql.lockmgr.HiveTxnManager; +import org.apache.hadoop.hive.ql.lockmgr.LockException; +import org.apache.hadoop.hive.ql.lockmgr.TxnManagerFactory; +import org.apache.hadoop.hive.ql.metadata.*; import org.apache.hadoop.hive.ql.metadata.Partition; import org.apache.hadoop.hive.ql.metadata.Table; import org.apache.hadoop.hive.ql.parse.authorization.AuthorizationParseUtils; @@ -585,7 +586,7 @@ private void analyzeAlterDatabaseProperties(ASTNode ast) throws SemanticExceptio private void addAlterDbDesc(AlterDatabaseDesc alterDesc) throws SemanticException { Database database = getDatabase(alterDesc.getDatabaseName()); - outputs.add(new WriteEntity(database)); + outputs.add(new WriteEntity(database, WriteEntity.WriteType.DDL_METADATA_ONLY)); rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(), alterDesc), conf)); } @@ -740,12 +741,12 @@ private void analyzeDropDatabase(ASTNode ast) throws SemanticException { if (tableNames != null) { for (String tableName : tableNames) { Table table = getTable(dbName, tableName, true); - outputs.add(new WriteEntity(table)); + outputs.add(new WriteEntity(table, WriteEntity.WriteType.DDL)); } } } inputs.add(new ReadEntity(database)); - outputs.add(new WriteEntity(database)); + outputs.add(new WriteEntity(database, WriteEntity.WriteType.DDL)); DropDatabaseDesc dropDatabaseDesc = new DropDatabaseDesc(dbName, ifExists, ifCascade); rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(), dropDatabaseDesc), conf)); @@ -771,7 +772,7 @@ private void analyzeDropTable(ASTNode ast, boolean expectView) Table tab = getTable(tableName, throwException); if (tab != null) { inputs.add(new ReadEntity(tab)); - outputs.add(new WriteEntity(tab)); + outputs.add(new WriteEntity(tab, WriteEntity.WriteType.DDL)); } DropTableDesc dropTblDesc = new DropTableDesc(tableName, expectView, ifExists); @@ -796,19 +797,19 @@ private void analyzeTruncateTable(ASTNode ast) throws SemanticException { Map partSpec = getPartSpec((ASTNode) root.getChild(1)); if (partSpec == null) { if (!table.isPartitioned()) { - outputs.add(new WriteEntity(table)); + outputs.add(new WriteEntity(table, WriteEntity.WriteType.DDL)); } else { for (Partition partition : getPartitions(table, null, false)) { - outputs.add(new WriteEntity(partition)); + outputs.add(new WriteEntity(partition, WriteEntity.WriteType.DDL)); } } } else { if (isFullSpec(table, partSpec)) { Partition partition = getPartition(table, partSpec, true); - outputs.add(new WriteEntity(partition)); + outputs.add(new WriteEntity(partition, WriteEntity.WriteType.DDL)); } else { for (Partition partition : getPartitions(table, partSpec, false)) { - outputs.add(new WriteEntity(partition)); + outputs.add(new WriteEntity(partition, WriteEntity.WriteType.DDL)); } } } @@ -1347,17 +1348,17 @@ private void addInputsOutputsAlterTable(String tableName, Map pa Table tab = getTable(tableName, true); if (partSpec == null || partSpec.isEmpty()) { inputs.add(new ReadEntity(tab)); - outputs.add(new WriteEntity(tab)); + outputs.add(new WriteEntity(tab, WriteEntity.WriteType.DDL)); } else { inputs.add(new ReadEntity(tab)); if (desc == null || desc.getOp() != AlterTableDesc.AlterTableTypes.ALTERPROTECTMODE) { Partition part = getPartition(tab, partSpec, true); - outputs.add(new WriteEntity(part)); + outputs.add(new WriteEntity(part, WriteEntity.WriteType.DDL)); } else { for (Partition part : getPartitions(tab, partSpec, true)) { - outputs.add(new WriteEntity(part)); + outputs.add(new WriteEntity(part, WriteEntity.WriteType.DDL)); } } } @@ -2236,7 +2237,7 @@ private void analyzeShowDbLocks(ASTNode ast) throws SemanticException { ctx.setNeedLockMgr(true); } - /** + /** * Add the task according to the parsed command tree. This is used for the CLI * command "LOCK TABLE ..;". * @@ -2566,7 +2567,7 @@ private void analyzeAlterTableAddParts(CommonTree ast, boolean expectView) Table tab = getTable(tblName, true); boolean isView = tab.isView(); validateAlterTableType(tab, AlterTableTypes.ADDPARTITION, expectView); - outputs.add(new WriteEntity(tab)); + outputs.add(new WriteEntity(tab, WriteEntity.WriteType.DDL)); int numCh = ast.getChildCount(); int start = ifNotExists ? 2 : 1; @@ -2666,7 +2667,7 @@ private Partition getPartitionForOutput(Table tab, Map currentPa try { Partition partition = db.getPartition(tab, currentPart, false); if (partition != null) { - outputs.add(new WriteEntity(partition)); + outputs.add(new WriteEntity(partition, WriteEntity.WriteType.INSERT)); } return partition; } catch (HiveException e) { @@ -2700,7 +2701,7 @@ private void analyzeAlterTableTouch(CommonTree ast) AlterTableSimpleDesc touchDesc = new AlterTableSimpleDesc( SessionState.get().getCurrentDatabase(), tblName, null, AlterTableDesc.AlterTableTypes.TOUCH); - outputs.add(new WriteEntity(tab)); + outputs.add(new WriteEntity(tab, WriteEntity.WriteType.DDL_METADATA_ONLY)); rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(), touchDesc), conf)); } else { @@ -2965,7 +2966,7 @@ private void addTablePartsOutputs(String tblName, List> part } } for (Partition p : parts) { - outputs.add(new WriteEntity(p)); + outputs.add(new WriteEntity(p, WriteEntity.WriteType.DDL)); } } } @@ -3009,7 +3010,7 @@ private void addTableDropPartsOutputs(Table tab, throw new SemanticException( ErrorMsg.DROP_COMMAND_NOT_ALLOWED_FOR_PARTITION.getMsg(p.getCompleteName())); } - outputs.add(new WriteEntity(p)); + outputs.add(new WriteEntity(p, WriteEntity.WriteType.DELETE)); } } } @@ -3033,7 +3034,7 @@ private void analyzeAltertableSkewedby(ASTNode ast) throws SemanticException { Table tab = getTable(tableName, true); inputs.add(new ReadEntity(tab)); - outputs.add(new WriteEntity(tab)); + outputs.add(new WriteEntity(tab, WriteEntity.WriteType.DDL)); validateAlterTableType(tab, AlterTableTypes.ADDSKEWEDBY); diff --git ql/src/java/org/apache/hadoop/hive/ql/parse/ImportSemanticAnalyzer.java ql/src/java/org/apache/hadoop/hive/ql/parse/ImportSemanticAnalyzer.java index ceb4c8a..92ec334 100644 --- ql/src/java/org/apache/hadoop/hive/ql/parse/ImportSemanticAnalyzer.java +++ ql/src/java/org/apache/hadoop/hive/ql/parse/ImportSemanticAnalyzer.java @@ -18,18 +18,6 @@ package org.apache.hadoop.hive.ql.parse; -import java.io.IOException; -import java.net.URI; -import java.net.URISyntaxException; -import java.util.ArrayList; -import java.util.Collections; -import java.util.Comparator; -import java.util.Iterator; -import java.util.LinkedHashMap; -import java.util.List; -import java.util.Map; -import java.util.TreeMap; - import org.antlr.runtime.tree.Tree; import org.apache.commons.lang.ObjectUtils; import org.apache.hadoop.fs.FileStatus; @@ -50,15 +38,15 @@ import org.apache.hadoop.hive.ql.metadata.HiveException; import org.apache.hadoop.hive.ql.metadata.InvalidTableException; import org.apache.hadoop.hive.ql.metadata.Table; -import org.apache.hadoop.hive.ql.plan.AddPartitionDesc; -import org.apache.hadoop.hive.ql.plan.CopyWork; -import org.apache.hadoop.hive.ql.plan.CreateTableDesc; -import org.apache.hadoop.hive.ql.plan.DDLWork; -import org.apache.hadoop.hive.ql.plan.LoadTableDesc; -import org.apache.hadoop.hive.ql.plan.MoveWork; +import org.apache.hadoop.hive.ql.plan.*; import org.apache.hadoop.hive.ql.session.SessionState; import org.apache.hadoop.hive.serde.serdeConstants; +import java.io.IOException; +import java.net.URI; +import java.net.URISyntaxException; +import java.util.*; + /** * ImportSemanticAnalyzer. * @@ -237,7 +225,8 @@ public void analyzeInternal(ASTNode ast) throws SemanticException { .toString())); loadTable(fromURI, table); } - outputs.add(new WriteEntity(table)); + // Set this to read because we can't overwrite any existing partitions + outputs.add(new WriteEntity(table, WriteEntity.WriteType.DDL_METADATA_ONLY)); } catch (InvalidTableException e) { LOG.debug("table " + tblDesc.getTableName() + " does not exist"); diff --git ql/src/java/org/apache/hadoop/hive/ql/parse/LoadSemanticAnalyzer.java ql/src/java/org/apache/hadoop/hive/ql/parse/LoadSemanticAnalyzer.java index 59aeb96..9ec22ca 100644 --- ql/src/java/org/apache/hadoop/hive/ql/parse/LoadSemanticAnalyzer.java +++ ql/src/java/org/apache/hadoop/hive/ql/parse/LoadSemanticAnalyzer.java @@ -18,14 +18,6 @@ package org.apache.hadoop.hive.ql.parse; -import java.io.IOException; -import java.io.Serializable; -import java.net.URI; -import java.net.URISyntaxException; -import java.util.LinkedHashMap; -import java.util.List; -import java.util.Map; - import org.antlr.runtime.tree.Tree; import org.apache.commons.lang.StringUtils; import org.apache.hadoop.fs.FileStatus; @@ -47,6 +39,14 @@ import org.apache.hadoop.hive.ql.plan.MoveWork; import org.apache.hadoop.hive.ql.plan.StatsWork; +import java.io.IOException; +import java.io.Serializable; +import java.net.URI; +import java.net.URISyntaxException; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; + /** * LoadSemanticAnalyzer. * @@ -238,7 +238,9 @@ public void analyzeInternal(ASTNode ast) throws SemanticException { Map partSpec = ts.getPartSpec(); if (partSpec == null) { partSpec = new LinkedHashMap(); - outputs.add(new WriteEntity(ts.tableHandle)); + outputs.add(new WriteEntity(ts.tableHandle, + (isOverWrite ? WriteEntity.WriteType.INSERT_OVERWRITE : + WriteEntity.WriteType.INSERT))); } else { try{ Partition part = Hive.get().getPartition(ts.tableHandle, partSpec, false); @@ -247,9 +249,13 @@ public void analyzeInternal(ASTNode ast) throws SemanticException { throw new SemanticException(ErrorMsg.OFFLINE_TABLE_OR_PARTITION. getMsg(ts.tableName + ":" + part.getName())); } - outputs.add(new WriteEntity(part)); + outputs.add(new WriteEntity(part, + (isOverWrite ? WriteEntity.WriteType.INSERT_OVERWRITE : + WriteEntity.WriteType.INSERT))); } else { - outputs.add(new WriteEntity(ts.tableHandle)); + outputs.add(new WriteEntity(ts.tableHandle, + (isOverWrite ? WriteEntity.WriteType.INSERT_OVERWRITE : + WriteEntity.WriteType.INSERT))); } } catch(HiveException e) { throw new SemanticException(e); diff --git ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java index 89d2a9c..4e04b7f 100644 --- ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java +++ ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java @@ -5602,7 +5602,7 @@ private Operator genFileSinkPlan(String dest, QB qb, Operator input) // in the case of DP, we will register WriteEntity in MoveTask when the // list of dynamically created partitions are known. if ((dpCtx == null || dpCtx.getNumDPCols() == 0)) { - output = new WriteEntity(dest_tab); + output = new WriteEntity(dest_tab, determineWriteType(ltd, isNonNativeTable)); if (!outputs.add(output)) { throw new SemanticException(ErrorMsg.OUTPUT_SPECIFIED_MULTIPLE_TIMES .getMsg(dest_tab.getTableName())); @@ -5611,7 +5611,7 @@ private Operator genFileSinkPlan(String dest, QB qb, Operator input) if ((dpCtx != null) && (dpCtx.getNumDPCols() >= 0)) { // No static partition specified if (dpCtx.getNumSPCols() == 0) { - output = new WriteEntity(dest_tab, false); + output = new WriteEntity(dest_tab, determineWriteType(ltd, isNonNativeTable), false); outputs.add(output); } // part of the partition specified @@ -5625,7 +5625,7 @@ private Operator genFileSinkPlan(String dest, QB qb, Operator input) new DummyPartition(dest_tab, dest_tab.getDbName() + "@" + dest_tab.getTableName() + "@" + ppath, partSpec); - output = new WriteEntity(p, false); + output = new WriteEntity(p, WriteEntity.WriteType.INSERT, false); outputs.add(output); } catch (HiveException e) { throw new SemanticException(e.getMessage(), e); @@ -5712,7 +5712,9 @@ private Operator genFileSinkPlan(String dest, QB qb, Operator input) ltd.setHoldDDLTime(true); } loadTableWork.add(ltd); - if (!outputs.add(new WriteEntity(dest_part))) { + if (!outputs.add(new WriteEntity(dest_part, (ltd.getReplace() ? + WriteEntity.WriteType.INSERT_OVERWRITE : + WriteEntity.WriteType.INSERT)))) { throw new SemanticException(ErrorMsg.OUTPUT_SPECIFIED_MULTIPLE_TIMES .getMsg(dest_tab.getTableName() + "@" + dest_part.getName())); } @@ -8818,7 +8820,7 @@ private void setupStats(TableScanDesc tsDesc, QBParseInfo qbp, Table tab, String tsDesc.setStatsAggPrefix(tab.getDbName()+"."+k); // set up WritenEntity for replication - outputs.add(new WriteEntity(tab)); + outputs.add(new WriteEntity(tab, WriteEntity.WriteType.DDL_METADATA_ONLY)); // add WriteEntity for each matching partition if (tab.isPartitioned()) { @@ -8829,7 +8831,7 @@ private void setupStats(TableScanDesc tsDesc, QBParseInfo qbp, Table tab, String if (partitions != null) { for (Partition partn : partitions) { // inputs.add(new ReadEntity(partn)); // is this needed at all? - outputs.add(new WriteEntity(partn)); + outputs.add(new WriteEntity(partn, WriteEntity.WriteType.DDL_METADATA_ONLY)); } } } @@ -9878,7 +9880,7 @@ private ASTNode analyzeCreateTable(ASTNode ast, QB qb) String[] qualified = Hive.getQualifiedNames(tableName); String dbName = qualified.length == 1 ? SessionState.get().getCurrentDatabase() : qualified[0]; Database database = getDatabase(dbName); - outputs.add(new WriteEntity(database)); + outputs.add(new WriteEntity(database, WriteEntity.WriteType.DDL_METADATA_ONLY)); // Handle different types of CREATE TABLE command CreateTableDesc crtTblDesc = null; switch (command_type) { @@ -11256,4 +11258,13 @@ private void addAlternateGByKeyMappings(ASTNode gByExpr, ColumnInfo colInfo, gByRR.put(tab_alias, col_alias, colInfo); } } + + private WriteEntity.WriteType determineWriteType(LoadTableDesc ltd, boolean isNonNativeTable) { + // Don't know the characteristics of non-native tables, + // and don't have a rational way to guess, so assume the most + // conservative case. + if (isNonNativeTable) return WriteEntity.WriteType.INSERT_OVERWRITE; + else return (ltd.getReplace() ? WriteEntity.WriteType.INSERT_OVERWRITE : + WriteEntity.WriteType.INSERT); + } } diff --git ql/src/java/org/apache/hadoop/hive/ql/parse/authorization/HiveAuthorizationTaskFactoryImpl.java ql/src/java/org/apache/hadoop/hive/ql/parse/authorization/HiveAuthorizationTaskFactoryImpl.java index 4a811eb..2c555af 100644 --- ql/src/java/org/apache/hadoop/hive/ql/parse/authorization/HiveAuthorizationTaskFactoryImpl.java +++ ql/src/java/org/apache/hadoop/hive/ql/parse/authorization/HiveAuthorizationTaskFactoryImpl.java @@ -250,9 +250,9 @@ private PrivilegeObjectDesc analyzePrivilegeObject(ASTNode ast, Table tbl = getTable(SessionState.get().getCurrentDatabase(), subject.getObject()); if (subject.getPartSpec() != null) { Partition part = getPartition(tbl, subject.getPartSpec()); - outputs.add(new WriteEntity(part)); + outputs.add(new WriteEntity(part, WriteEntity.WriteType.DDL_METADATA_ONLY)); } else { - outputs.add(new WriteEntity(tbl)); + outputs.add(new WriteEntity(tbl, WriteEntity.WriteType.DDL_METADATA_ONLY)); } } diff --git ql/src/test/org/apache/hadoop/hive/ql/TestDriver.java ql/src/test/org/apache/hadoop/hive/ql/TestDriver.java deleted file mode 100644 index cadb293..0000000 --- ql/src/test/org/apache/hadoop/hive/ql/TestDriver.java +++ /dev/null @@ -1,75 +0,0 @@ -/** - * 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. - */ - -package org.apache.hadoop.hive.ql; - -import java.util.*; -import junit.framework.TestCase; - -import org.apache.hadoop.hive.ql.lockmgr.HiveLockMode; -import org.apache.hadoop.hive.ql.lockmgr.HiveLockObj; -import org.apache.hadoop.hive.ql.lockmgr.HiveLockObject; -import org.apache.hadoop.hive.ql.lockmgr.HiveLockObject.HiveLockObjectData; - -public class TestDriver extends TestCase { - public void testDedupLockObjects() { - List lockObjs = new ArrayList(); - String path1 = "path1"; - String path2 = "path2"; - HiveLockObjectData lockData1 = new HiveLockObjectData( - "query1", "1", "IMPLICIT", "drop table table1"); - HiveLockObjectData lockData2 = new HiveLockObjectData( - "query1", "1", "IMPLICIT", "drop table table1"); - - // Start with the following locks: - // [path1, shared] - // [path1, exclusive] - // [path2, shared] - // [path2, shared] - // [path2, shared] - lockObjs.add(new HiveLockObj(new HiveLockObject(path1, lockData1), HiveLockMode.SHARED)); - String name1 = lockObjs.get(lockObjs.size() - 1).getName(); - lockObjs.add(new HiveLockObj(new HiveLockObject(path1, lockData1), HiveLockMode.EXCLUSIVE)); - lockObjs.add(new HiveLockObj(new HiveLockObject(path2, lockData2), HiveLockMode.SHARED)); - String name2 = lockObjs.get(lockObjs.size() - 1).getName(); - lockObjs.add(new HiveLockObj(new HiveLockObject(path2, lockData2), HiveLockMode.SHARED)); - lockObjs.add(new HiveLockObj(new HiveLockObject(path2, lockData2), HiveLockMode.SHARED)); - - Driver.dedupLockObjects(lockObjs); - - // After dedup we should be left with 2 locks: - // [path1, exclusive] - // [path2, shared] - assertEquals("Locks should be deduped", 2, lockObjs.size()); - - Comparator cmp = new Comparator() { - public int compare(HiveLockObj lock1, HiveLockObj lock2) { - return lock1.getName().compareTo(lock2.getName()); - } - }; - Collections.sort(lockObjs, cmp); - - HiveLockObj lockObj = lockObjs.get(0); - assertEquals(name1, lockObj.getName()); - assertEquals(HiveLockMode.EXCLUSIVE, lockObj.getMode()); - - lockObj = lockObjs.get(1); - assertEquals(name2, lockObj.getName()); - assertEquals(HiveLockMode.SHARED, lockObj.getMode()); - } -} diff --git ql/src/test/org/apache/hadoop/hive/ql/lockmgr/TestDbTxnManager.java ql/src/test/org/apache/hadoop/hive/ql/lockmgr/TestDbTxnManager.java new file mode 100644 index 0000000..01e5085 --- /dev/null +++ ql/src/test/org/apache/hadoop/hive/ql/lockmgr/TestDbTxnManager.java @@ -0,0 +1,276 @@ +/** + * 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. + */ +package org.apache.hadoop.hive.ql.lockmgr; + +import junit.framework.Assert; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.metastore.api.FieldSchema; +import org.apache.hadoop.hive.metastore.txn.TxnDbUtil; +import org.apache.hadoop.hive.ql.Context; +import org.apache.hadoop.hive.ql.QueryPlan; +import org.apache.hadoop.hive.ql.hooks.ReadEntity; +import org.apache.hadoop.hive.ql.hooks.WriteEntity; +import org.apache.hadoop.hive.ql.metadata.Partition; +import org.apache.hadoop.hive.ql.metadata.Table; +import org.apache.log4j.Level; +import org.apache.log4j.LogManager; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import java.util.*; + +/** + * Unit tests for {@link DbTxnManager}. + */ +public class TestDbTxnManager { + + private HiveConf conf = new HiveConf(); + private HiveTxnManager txnMgr; + private Context ctx; + private int nextInput; + private int nextOutput; + HashSet readEntities; + HashSet writeEntities; + + public TestDbTxnManager() throws Exception { + TxnDbUtil.setConfValues(conf); + ctx = new Context(conf); + LogManager.getRootLogger().setLevel(Level.DEBUG); + tearDown(); + } + + @Test + public void testSingleReadTable() throws Exception { + addTableInput(); + QueryPlan qp = new MockQueryPlan(this); + txnMgr.acquireLocks(qp, ctx, "fred"); + List locks = ctx.getHiveLocks(); + Assert.assertEquals(1, locks.size()); + Assert.assertEquals(1, + TxnDbUtil.countLockComponents(((DbLockManager.DbHiveLock) locks.get(0)).lockId)); + txnMgr.getLockManager().unlock(locks.get(0)); + locks = txnMgr.getLockManager().getLocks(false, false); + Assert.assertEquals(0, locks.size()); + } + + @Test + public void testSingleReadPartition() throws Exception { + addPartitionInput(newTable(true)); + QueryPlan qp = new MockQueryPlan(this); + txnMgr.acquireLocks(qp, ctx, null); + List locks = ctx.getHiveLocks(); + Assert.assertEquals(1, locks.size()); + Assert.assertEquals(1, + TxnDbUtil.countLockComponents(((DbLockManager.DbHiveLock) locks.get(0)).lockId)); + txnMgr.getLockManager().unlock(locks.get(0)); + locks = txnMgr.getLockManager().getLocks(false, false); + Assert.assertEquals(0, locks.size()); + + } + + @Test + public void testSingleReadMultiPartition() throws Exception { + Table t = newTable(true); + addPartitionInput(t); + addPartitionInput(t); + addPartitionInput(t); + QueryPlan qp = new MockQueryPlan(this); + txnMgr.acquireLocks(qp, ctx, "fred"); + List locks = ctx.getHiveLocks(); + Assert.assertEquals(1, locks.size()); + Assert.assertEquals(3, + TxnDbUtil.countLockComponents(((DbLockManager.DbHiveLock) locks.get(0)).lockId)); + txnMgr.getLockManager().unlock(locks.get(0)); + locks = txnMgr.getLockManager().getLocks(false, false); + Assert.assertEquals(0, locks.size()); + } + + @Test + public void testJoin() throws Exception { + Table t = newTable(true); + addPartitionInput(t); + addPartitionInput(t); + addPartitionInput(t); + addTableInput(); + QueryPlan qp = new MockQueryPlan(this); + txnMgr.acquireLocks(qp, ctx, "fred"); + List locks = ctx.getHiveLocks(); + Assert.assertEquals(1, locks.size()); + Assert.assertEquals(4, + TxnDbUtil.countLockComponents(((DbLockManager.DbHiveLock) locks.get(0)).lockId)); + txnMgr.getLockManager().unlock(locks.get(0)); + locks = txnMgr.getLockManager().getLocks(false, false); + Assert.assertEquals(0, locks.size()); + } + + @Test + public void testSingleWriteTable() throws Exception { + WriteEntity we = addTableOutput(WriteEntity.WriteType.INSERT); + QueryPlan qp = new MockQueryPlan(this); + txnMgr.acquireLocks(qp, ctx, "fred"); + List locks = ctx.getHiveLocks(); + Assert.assertEquals(1, locks.size()); + Assert.assertEquals(1, + TxnDbUtil.countLockComponents(((DbLockManager.DbHiveLock) locks.get(0)).lockId)); + txnMgr.getLockManager().unlock(locks.get(0)); + locks = txnMgr.getLockManager().getLocks(false, false); + Assert.assertEquals(0, locks.size()); + } + + @Test + public void testReadWrite() throws Exception { + Table t = newTable(true); + addPartitionInput(t); + addPartitionInput(t); + addPartitionInput(t); + WriteEntity we = addTableOutput(WriteEntity.WriteType.INSERT); + QueryPlan qp = new MockQueryPlan(this); + txnMgr.acquireLocks(qp, ctx, "fred"); + List locks = ctx.getHiveLocks(); + Assert.assertEquals(1, locks.size()); + Assert.assertEquals(4, + TxnDbUtil.countLockComponents(((DbLockManager.DbHiveLock) locks.get(0)).lockId)); + txnMgr.getLockManager().unlock(locks.get(0)); + locks = txnMgr.getLockManager().getLocks(false, false); + Assert.assertEquals(0, locks.size()); + } + + @Test + public void testUpdate() throws Exception { + WriteEntity we = addTableOutput(WriteEntity.WriteType.UPDATE); + QueryPlan qp = new MockQueryPlan(this); + txnMgr.acquireLocks(qp, ctx, "fred"); + List locks = ctx.getHiveLocks(); + Assert.assertEquals(1, locks.size()); + Assert.assertEquals(1, + TxnDbUtil.countLockComponents(((DbLockManager.DbHiveLock) locks.get(0)).lockId)); + txnMgr.getLockManager().unlock(locks.get(0)); + locks = txnMgr.getLockManager().getLocks(false, false); + Assert.assertEquals(0, locks.size()); + } + + @Test + public void testDelete() throws Exception { + WriteEntity we = addTableOutput(WriteEntity.WriteType.DELETE); + QueryPlan qp = new MockQueryPlan(this); + txnMgr.acquireLocks(qp, ctx, "fred"); + List locks = ctx.getHiveLocks(); + Assert.assertEquals(1, locks.size()); + Assert.assertEquals(1, + TxnDbUtil.countLockComponents(((DbLockManager.DbHiveLock) locks.get(0)).lockId)); + txnMgr.getLockManager().unlock(locks.get(0)); + locks = txnMgr.getLockManager().getLocks(false, false); + Assert.assertEquals(0, locks.size()); + } + + @Test + public void testDDL() throws Exception { + WriteEntity we = addTableOutput(WriteEntity.WriteType.DDL); + QueryPlan qp = new MockQueryPlan(this); + txnMgr.acquireLocks(qp, ctx, "fred"); + List locks = ctx.getHiveLocks(); + Assert.assertEquals(1, locks.size()); + Assert.assertEquals(1, + TxnDbUtil.countLockComponents(((DbLockManager.DbHiveLock) locks.get(0)).lockId)); + txnMgr.getLockManager().unlock(locks.get(0)); + locks = txnMgr.getLockManager().getLocks(false, false); + Assert.assertEquals(0, locks.size()); + } + + @Before + public void setUp() throws Exception { + TxnDbUtil.prepDb(); + txnMgr = TxnManagerFactory.getTxnManagerFactory().getTxnManager(conf); + Assert.assertTrue(txnMgr instanceof DbTxnManager); + nextInput = 1; + nextOutput = 1; + readEntities = new HashSet(); + writeEntities = new HashSet(); + } + + @After + public void tearDown() throws Exception { + TxnDbUtil.cleanDb(); + } + + private static class MockQueryPlan extends QueryPlan { + private HashSet inputs; + private HashSet outputs; + + MockQueryPlan(TestDbTxnManager test) { + HashSet r = test.readEntities; + HashSet w = test.writeEntities; + inputs = (r == null) ? new HashSet() : r; + outputs = (w == null) ? new HashSet() : w; + } + + @Override + public HashSet getInputs() { + return inputs; + } + + @Override + public HashSet getOutputs() { + return outputs; + } + } + + private Table newTable(boolean isPartitioned) { + Table t = new Table("default", "table" + Integer.toString(nextInput++)); + if (isPartitioned) { + FieldSchema fs = new FieldSchema(); + fs.setName("version"); + fs.setType("String"); + List partCols = new ArrayList(1); + partCols.add(fs); + t.setPartCols(partCols); + } + return t; + } + + private void addTableInput() { + ReadEntity re = new ReadEntity(newTable(false)); + readEntities.add(re); + } + + private void addPartitionInput(Table t) throws Exception { + Map partSpec = new HashMap(); + partSpec.put("version", Integer.toString(nextInput++)); + Partition p = new Partition(t, partSpec, new Path("/dev/null")); + ReadEntity re = new ReadEntity(p); + readEntities.add(re); + } + + private WriteEntity addTableOutput(WriteEntity.WriteType writeType) { + WriteEntity we = new WriteEntity(newTable(false), writeType); + writeEntities.add(we); + return we; + } + + private WriteEntity addPartitionOutput(Table t, WriteEntity.WriteType writeType) + throws Exception { + Map partSpec = new HashMap(); + partSpec.put("version", Integer.toString(nextInput++)); + Partition p = new Partition(t, partSpec, new Path("/dev/null")); + WriteEntity we = new WriteEntity(p, writeType); + writeEntities.add(we); + return we; + } +} diff --git ql/src/test/org/apache/hadoop/hive/ql/lockmgr/TestDummyTxnManager.java ql/src/test/org/apache/hadoop/hive/ql/lockmgr/TestDummyTxnManager.java new file mode 100644 index 0000000..5abb729 --- /dev/null +++ ql/src/test/org/apache/hadoop/hive/ql/lockmgr/TestDummyTxnManager.java @@ -0,0 +1,78 @@ +/** + * 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. + */ + +package org.apache.hadoop.hive.ql.lockmgr; + +import junit.framework.Assert; +import org.apache.hadoop.hive.ql.lockmgr.HiveLockObject.HiveLockObjectData; +import org.junit.Test; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.Comparator; +import java.util.List; + +public class TestDummyTxnManager { + + @Test + public void testDedupLockObjects() { + List lockObjs = new ArrayList(); + String path1 = "path1"; + String path2 = "path2"; + HiveLockObjectData lockData1 = new HiveLockObjectData( + "query1", "1", "IMPLICIT", "drop table table1"); + HiveLockObjectData lockData2 = new HiveLockObjectData( + "query1", "1", "IMPLICIT", "drop table table1"); + + // Start with the following locks: + // [path1, shared] + // [path1, exclusive] + // [path2, shared] + // [path2, shared] + // [path2, shared] + lockObjs.add(new HiveLockObj(new HiveLockObject(path1, lockData1), HiveLockMode.SHARED)); + String name1 = lockObjs.get(lockObjs.size() - 1).getName(); + lockObjs.add(new HiveLockObj(new HiveLockObject(path1, lockData1), HiveLockMode.EXCLUSIVE)); + lockObjs.add(new HiveLockObj(new HiveLockObject(path2, lockData2), HiveLockMode.SHARED)); + String name2 = lockObjs.get(lockObjs.size() - 1).getName(); + lockObjs.add(new HiveLockObj(new HiveLockObject(path2, lockData2), HiveLockMode.SHARED)); + lockObjs.add(new HiveLockObj(new HiveLockObject(path2, lockData2), HiveLockMode.SHARED)); + + DummyTxnManager.dedupLockObjects(lockObjs); + + // After dedup we should be left with 2 locks: + // [path1, exclusive] + // [path2, shared] + Assert.assertEquals("Locks should be deduped", 2, lockObjs.size()); + + Comparator cmp = new Comparator() { + public int compare(HiveLockObj lock1, HiveLockObj lock2) { + return lock1.getName().compareTo(lock2.getName()); + } + }; + Collections.sort(lockObjs, cmp); + + HiveLockObj lockObj = lockObjs.get(0); + Assert.assertEquals(name1, lockObj.getName()); + Assert.assertEquals(HiveLockMode.EXCLUSIVE, lockObj.getMode()); + + lockObj = lockObjs.get(1); + Assert.assertEquals(name2, lockObj.getName()); + Assert.assertEquals(HiveLockMode.SHARED, lockObj.getMode()); + } +} diff --git ql/src/test/queries/clientnegative/dbtxnmgr_nodblock.q ql/src/test/queries/clientnegative/dbtxnmgr_nodblock.q new file mode 100644 index 0000000..1c658c7 --- /dev/null +++ ql/src/test/queries/clientnegative/dbtxnmgr_nodblock.q @@ -0,0 +1,6 @@ +set hive.support.concurrency=true; +set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager; + +drop database if exists drop_nodblock; +create database drop_nodblock; +lock database drop_nodblock shared; diff --git ql/src/test/queries/clientnegative/dbtxnmgr_nodbunlock.q ql/src/test/queries/clientnegative/dbtxnmgr_nodbunlock.q new file mode 100644 index 0000000..ef4b323 --- /dev/null +++ ql/src/test/queries/clientnegative/dbtxnmgr_nodbunlock.q @@ -0,0 +1,6 @@ +set hive.support.concurrency=true; +set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager; + +drop database if exists drop_nodbunlock; +create database drop_nodbunlock; +unlock database drop_nodbunlock; diff --git ql/src/test/queries/clientnegative/dbtxnmgr_notablelock.q ql/src/test/queries/clientnegative/dbtxnmgr_notablelock.q new file mode 100644 index 0000000..4a0c6c2 --- /dev/null +++ ql/src/test/queries/clientnegative/dbtxnmgr_notablelock.q @@ -0,0 +1,6 @@ +set hive.support.concurrency=true; +set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager; + +drop table if exists drop_notablelock; +create table drop_notablelock (c int); +lock table drop_notablelock shared; diff --git ql/src/test/queries/clientnegative/dbtxnmgr_notableunlock.q ql/src/test/queries/clientnegative/dbtxnmgr_notableunlock.q new file mode 100644 index 0000000..0b00046 --- /dev/null +++ ql/src/test/queries/clientnegative/dbtxnmgr_notableunlock.q @@ -0,0 +1,6 @@ +set hive.support.concurrency=true; +set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager; + +drop table if exists drop_notableunlock; +create table drop_notableunlock (c int); +unlock table drop_notableunlock; diff --git ql/src/test/queries/clientpositive/dbtxnmgr_query1.q ql/src/test/queries/clientpositive/dbtxnmgr_query1.q new file mode 100644 index 0000000..970069a --- /dev/null +++ ql/src/test/queries/clientpositive/dbtxnmgr_query1.q @@ -0,0 +1,17 @@ +set hive.support.concurrency=true; +set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager; + +create table T1(key string, val string) stored as textfile; + +LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1; + +select * from T1; + +create table T2(key string, val string) stored as textfile; + +insert into table T2 select * from T1; + +select * from T2; + +drop table T1; +drop table T2; diff --git ql/src/test/queries/clientpositive/dbtxnmgr_query2.q ql/src/test/queries/clientpositive/dbtxnmgr_query2.q new file mode 100644 index 0000000..00942e5 --- /dev/null +++ ql/src/test/queries/clientpositive/dbtxnmgr_query2.q @@ -0,0 +1,17 @@ +set hive.support.concurrency=true; +set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager; + +create table T1(key string, val string) stored as textfile; + +LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1; + +select * from T1; + +create table T2(key string, val string) stored as textfile; + +insert overwrite table T2 select * from T1; + +select * from T2; + +drop table T1; +drop table T2; diff --git ql/src/test/queries/clientpositive/dbtxnmgr_query3.q ql/src/test/queries/clientpositive/dbtxnmgr_query3.q new file mode 100644 index 0000000..75b642b --- /dev/null +++ ql/src/test/queries/clientpositive/dbtxnmgr_query3.q @@ -0,0 +1,21 @@ +set hive.support.concurrency=true; +set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager; + +create table T1(key string, val string) stored as textfile; + +LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1; + +select * from T1; + +create table T2(key string, val string) partitioned by (pval string) stored as textfile; + +insert into table T2 partition (pval = '1') select * from T1; + +select * from T2; + +insert overwrite table T2 partition (pval = '1') select * from T1; + +select * from T2; + +drop table T1; +drop table T2; diff --git ql/src/test/queries/clientpositive/dbtxnmgr_query4.q ql/src/test/queries/clientpositive/dbtxnmgr_query4.q new file mode 100644 index 0000000..57eb442 --- /dev/null +++ ql/src/test/queries/clientpositive/dbtxnmgr_query4.q @@ -0,0 +1,19 @@ +set hive.support.concurrency=true; +set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager; +set hive.exec.dynamic.partition.mode=nonstrict; +set hive.exec.dynamic.partition=true; + +create table T1(key string, val string) stored as textfile; + +LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1; + +select * from T1; + +create table T2(key string) partitioned by (val string) stored as textfile; + +insert overwrite table T2 partition (val) select key, val from T1; + +select * from T2; + +drop table T1; +drop table T2; diff --git ql/src/test/queries/clientpositive/dbtxnmgr_query5.q ql/src/test/queries/clientpositive/dbtxnmgr_query5.q new file mode 100644 index 0000000..d22b98f --- /dev/null +++ ql/src/test/queries/clientpositive/dbtxnmgr_query5.q @@ -0,0 +1,24 @@ +set hive.support.concurrency=true; +set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager; + +create database foo; + +use foo; + +create table T1(key string, val string) partitioned by (ds string) stored as textfile; + +alter table T1 add partition (ds='today'); + +create view V1 as select key from T1; + +show tables; + +describe T1; + +drop view V1; + +drop table T1; + +show databases; + +drop database foo; diff --git ql/src/test/results/clientnegative/dbtxnmgr_nodblock.q.out ql/src/test/results/clientnegative/dbtxnmgr_nodblock.q.out new file mode 100644 index 0000000..4786679 --- /dev/null +++ ql/src/test/results/clientnegative/dbtxnmgr_nodblock.q.out @@ -0,0 +1,11 @@ +PREHOOK: query: drop database if exists drop_nodblock +PREHOOK: type: DROPDATABASE +POSTHOOK: query: drop database if exists drop_nodblock +POSTHOOK: type: DROPDATABASE +PREHOOK: query: create database drop_nodblock +PREHOOK: type: CREATEDATABASE +POSTHOOK: query: create database drop_nodblock +POSTHOOK: type: CREATEDATABASE +PREHOOK: query: lock database drop_nodblock shared +PREHOOK: type: LOCKDATABASE +FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.DDLTask. Current transaction manager does not support explicit lock requests. Transaction manager: org.apache.hadoop.hive.ql.lockmgr.DbTxnManager diff --git ql/src/test/results/clientnegative/dbtxnmgr_nodbunlock.q.out ql/src/test/results/clientnegative/dbtxnmgr_nodbunlock.q.out new file mode 100644 index 0000000..14abcaf --- /dev/null +++ ql/src/test/results/clientnegative/dbtxnmgr_nodbunlock.q.out @@ -0,0 +1,11 @@ +PREHOOK: query: drop database if exists drop_nodbunlock +PREHOOK: type: DROPDATABASE +POSTHOOK: query: drop database if exists drop_nodbunlock +POSTHOOK: type: DROPDATABASE +PREHOOK: query: create database drop_nodbunlock +PREHOOK: type: CREATEDATABASE +POSTHOOK: query: create database drop_nodbunlock +POSTHOOK: type: CREATEDATABASE +PREHOOK: query: unlock database drop_nodbunlock +PREHOOK: type: UNLOCKDATABASE +FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.DDLTask. Current transaction manager does not support explicit lock requests. Transaction manager: org.apache.hadoop.hive.ql.lockmgr.DbTxnManager diff --git ql/src/test/results/clientnegative/dbtxnmgr_notablelock.q.out ql/src/test/results/clientnegative/dbtxnmgr_notablelock.q.out new file mode 100644 index 0000000..19aed99 --- /dev/null +++ ql/src/test/results/clientnegative/dbtxnmgr_notablelock.q.out @@ -0,0 +1,14 @@ +PREHOOK: query: drop table if exists drop_notablelock +PREHOOK: type: DROPTABLE +POSTHOOK: query: drop table if exists drop_notablelock +POSTHOOK: type: DROPTABLE +PREHOOK: query: create table drop_notablelock (c int) +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +POSTHOOK: query: create table drop_notablelock (c int) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@drop_notablelock +PREHOOK: query: lock table drop_notablelock shared +PREHOOK: type: LOCKTABLE +FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.DDLTask. Current transaction manager does not support explicit lock requests. Transaction manager: org.apache.hadoop.hive.ql.lockmgr.DbTxnManager diff --git ql/src/test/results/clientnegative/dbtxnmgr_notableunlock.q.out ql/src/test/results/clientnegative/dbtxnmgr_notableunlock.q.out new file mode 100644 index 0000000..38b9d1e --- /dev/null +++ ql/src/test/results/clientnegative/dbtxnmgr_notableunlock.q.out @@ -0,0 +1,14 @@ +PREHOOK: query: drop table if exists drop_notableunlock +PREHOOK: type: DROPTABLE +POSTHOOK: query: drop table if exists drop_notableunlock +POSTHOOK: type: DROPTABLE +PREHOOK: query: create table drop_notableunlock (c int) +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +POSTHOOK: query: create table drop_notableunlock (c int) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@drop_notableunlock +PREHOOK: query: unlock table drop_notableunlock +PREHOOK: type: UNLOCKTABLE +FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.DDLTask. Current transaction manager does not support explicit lock requests. Transaction manager: org.apache.hadoop.hive.ql.lockmgr.DbTxnManager diff --git ql/src/test/results/clientpositive/dbtxnmgr_query1.q.out ql/src/test/results/clientpositive/dbtxnmgr_query1.q.out new file mode 100644 index 0000000..4f5b613 --- /dev/null +++ ql/src/test/results/clientpositive/dbtxnmgr_query1.q.out @@ -0,0 +1,82 @@ +PREHOOK: query: create table T1(key string, val string) stored as textfile +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +POSTHOOK: query: create table T1(key string, val string) stored as textfile +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@T1 +PREHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1 +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@t1 +POSTHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1 +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@t1 +PREHOOK: query: select * from T1 +PREHOOK: type: QUERY +PREHOOK: Input: default@t1 +#### A masked pattern was here #### +POSTHOOK: query: select * from T1 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@t1 +#### A masked pattern was here #### +1 11 +2 12 +3 13 +7 17 +8 18 +8 28 +PREHOOK: query: create table T2(key string, val string) stored as textfile +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +POSTHOOK: query: create table T2(key string, val string) stored as textfile +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@T2 +PREHOOK: query: insert into table T2 select * from T1 +PREHOOK: type: QUERY +PREHOOK: Input: default@t1 +PREHOOK: Output: default@t2 +POSTHOOK: query: insert into table T2 select * from T1 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@t1 +POSTHOOK: Output: default@t2 +POSTHOOK: Lineage: t2.key SIMPLE [(t1)t1.FieldSchema(name:key, type:string, comment:null), ] +POSTHOOK: Lineage: t2.val SIMPLE [(t1)t1.FieldSchema(name:val, type:string, comment:null), ] +PREHOOK: query: select * from T2 +PREHOOK: type: QUERY +PREHOOK: Input: default@t2 +#### A masked pattern was here #### +POSTHOOK: query: select * from T2 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@t2 +#### A masked pattern was here #### +POSTHOOK: Lineage: t2.key SIMPLE [(t1)t1.FieldSchema(name:key, type:string, comment:null), ] +POSTHOOK: Lineage: t2.val SIMPLE [(t1)t1.FieldSchema(name:val, type:string, comment:null), ] +1 11 +2 12 +3 13 +7 17 +8 18 +8 28 +PREHOOK: query: drop table T1 +PREHOOK: type: DROPTABLE +PREHOOK: Input: default@t1 +PREHOOK: Output: default@t1 +POSTHOOK: query: drop table T1 +POSTHOOK: type: DROPTABLE +POSTHOOK: Input: default@t1 +POSTHOOK: Output: default@t1 +POSTHOOK: Lineage: t2.key SIMPLE [(t1)t1.FieldSchema(name:key, type:string, comment:null), ] +POSTHOOK: Lineage: t2.val SIMPLE [(t1)t1.FieldSchema(name:val, type:string, comment:null), ] +PREHOOK: query: drop table T2 +PREHOOK: type: DROPTABLE +PREHOOK: Input: default@t2 +PREHOOK: Output: default@t2 +POSTHOOK: query: drop table T2 +POSTHOOK: type: DROPTABLE +POSTHOOK: Input: default@t2 +POSTHOOK: Output: default@t2 +POSTHOOK: Lineage: t2.key SIMPLE [(t1)t1.FieldSchema(name:key, type:string, comment:null), ] +POSTHOOK: Lineage: t2.val SIMPLE [(t1)t1.FieldSchema(name:val, type:string, comment:null), ] diff --git ql/src/test/results/clientpositive/dbtxnmgr_query2.q.out ql/src/test/results/clientpositive/dbtxnmgr_query2.q.out new file mode 100644 index 0000000..2d426a4 --- /dev/null +++ ql/src/test/results/clientpositive/dbtxnmgr_query2.q.out @@ -0,0 +1,82 @@ +PREHOOK: query: create table T1(key string, val string) stored as textfile +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +POSTHOOK: query: create table T1(key string, val string) stored as textfile +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@T1 +PREHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1 +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@t1 +POSTHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1 +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@t1 +PREHOOK: query: select * from T1 +PREHOOK: type: QUERY +PREHOOK: Input: default@t1 +#### A masked pattern was here #### +POSTHOOK: query: select * from T1 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@t1 +#### A masked pattern was here #### +1 11 +2 12 +3 13 +7 17 +8 18 +8 28 +PREHOOK: query: create table T2(key string, val string) stored as textfile +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +POSTHOOK: query: create table T2(key string, val string) stored as textfile +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@T2 +PREHOOK: query: insert overwrite table T2 select * from T1 +PREHOOK: type: QUERY +PREHOOK: Input: default@t1 +PREHOOK: Output: default@t2 +POSTHOOK: query: insert overwrite table T2 select * from T1 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@t1 +POSTHOOK: Output: default@t2 +POSTHOOK: Lineage: t2.key SIMPLE [(t1)t1.FieldSchema(name:key, type:string, comment:null), ] +POSTHOOK: Lineage: t2.val SIMPLE [(t1)t1.FieldSchema(name:val, type:string, comment:null), ] +PREHOOK: query: select * from T2 +PREHOOK: type: QUERY +PREHOOK: Input: default@t2 +#### A masked pattern was here #### +POSTHOOK: query: select * from T2 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@t2 +#### A masked pattern was here #### +POSTHOOK: Lineage: t2.key SIMPLE [(t1)t1.FieldSchema(name:key, type:string, comment:null), ] +POSTHOOK: Lineage: t2.val SIMPLE [(t1)t1.FieldSchema(name:val, type:string, comment:null), ] +1 11 +2 12 +3 13 +7 17 +8 18 +8 28 +PREHOOK: query: drop table T1 +PREHOOK: type: DROPTABLE +PREHOOK: Input: default@t1 +PREHOOK: Output: default@t1 +POSTHOOK: query: drop table T1 +POSTHOOK: type: DROPTABLE +POSTHOOK: Input: default@t1 +POSTHOOK: Output: default@t1 +POSTHOOK: Lineage: t2.key SIMPLE [(t1)t1.FieldSchema(name:key, type:string, comment:null), ] +POSTHOOK: Lineage: t2.val SIMPLE [(t1)t1.FieldSchema(name:val, type:string, comment:null), ] +PREHOOK: query: drop table T2 +PREHOOK: type: DROPTABLE +PREHOOK: Input: default@t2 +PREHOOK: Output: default@t2 +POSTHOOK: query: drop table T2 +POSTHOOK: type: DROPTABLE +POSTHOOK: Input: default@t2 +POSTHOOK: Output: default@t2 +POSTHOOK: Lineage: t2.key SIMPLE [(t1)t1.FieldSchema(name:key, type:string, comment:null), ] +POSTHOOK: Lineage: t2.val SIMPLE [(t1)t1.FieldSchema(name:val, type:string, comment:null), ] diff --git ql/src/test/results/clientpositive/dbtxnmgr_query3.q.out ql/src/test/results/clientpositive/dbtxnmgr_query3.q.out new file mode 100644 index 0000000..16c6152 --- /dev/null +++ ql/src/test/results/clientpositive/dbtxnmgr_query3.q.out @@ -0,0 +1,120 @@ +PREHOOK: query: create table T1(key string, val string) stored as textfile +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +POSTHOOK: query: create table T1(key string, val string) stored as textfile +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@T1 +PREHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1 +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@t1 +POSTHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1 +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@t1 +PREHOOK: query: select * from T1 +PREHOOK: type: QUERY +PREHOOK: Input: default@t1 +#### A masked pattern was here #### +POSTHOOK: query: select * from T1 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@t1 +#### A masked pattern was here #### +1 11 +2 12 +3 13 +7 17 +8 18 +8 28 +PREHOOK: query: create table T2(key string, val string) partitioned by (pval string) stored as textfile +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +POSTHOOK: query: create table T2(key string, val string) partitioned by (pval string) stored as textfile +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@T2 +PREHOOK: query: insert into table T2 partition (pval = '1') select * from T1 +PREHOOK: type: QUERY +PREHOOK: Input: default@t1 +PREHOOK: Output: default@t2@pval=1 +POSTHOOK: query: insert into table T2 partition (pval = '1') select * from T1 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@t1 +POSTHOOK: Output: default@t2@pval=1 +POSTHOOK: Lineage: t2 PARTITION(pval=1).key SIMPLE [(t1)t1.FieldSchema(name:key, type:string, comment:null), ] +POSTHOOK: Lineage: t2 PARTITION(pval=1).val SIMPLE [(t1)t1.FieldSchema(name:val, type:string, comment:null), ] +PREHOOK: query: select * from T2 +PREHOOK: type: QUERY +PREHOOK: Input: default@t2 +PREHOOK: Input: default@t2@pval=1 +#### A masked pattern was here #### +POSTHOOK: query: select * from T2 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@t2 +POSTHOOK: Input: default@t2@pval=1 +#### A masked pattern was here #### +POSTHOOK: Lineage: t2 PARTITION(pval=1).key SIMPLE [(t1)t1.FieldSchema(name:key, type:string, comment:null), ] +POSTHOOK: Lineage: t2 PARTITION(pval=1).val SIMPLE [(t1)t1.FieldSchema(name:val, type:string, comment:null), ] +1 11 1 +2 12 1 +3 13 1 +7 17 1 +8 18 1 +8 28 1 +PREHOOK: query: insert overwrite table T2 partition (pval = '1') select * from T1 +PREHOOK: type: QUERY +PREHOOK: Input: default@t1 +PREHOOK: Output: default@t2@pval=1 +POSTHOOK: query: insert overwrite table T2 partition (pval = '1') select * from T1 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@t1 +POSTHOOK: Output: default@t2@pval=1 +POSTHOOK: Lineage: t2 PARTITION(pval=1).key SIMPLE [(t1)t1.FieldSchema(name:key, type:string, comment:null), ] +POSTHOOK: Lineage: t2 PARTITION(pval=1).val SIMPLE [(t1)t1.FieldSchema(name:val, type:string, comment:null), ] +POSTHOOK: Lineage: t2 PARTITION(pval=1).key SIMPLE [(t1)t1.FieldSchema(name:key, type:string, comment:null), ] +POSTHOOK: Lineage: t2 PARTITION(pval=1).val SIMPLE [(t1)t1.FieldSchema(name:val, type:string, comment:null), ] +PREHOOK: query: select * from T2 +PREHOOK: type: QUERY +PREHOOK: Input: default@t2 +PREHOOK: Input: default@t2@pval=1 +#### A masked pattern was here #### +POSTHOOK: query: select * from T2 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@t2 +POSTHOOK: Input: default@t2@pval=1 +#### A masked pattern was here #### +POSTHOOK: Lineage: t2 PARTITION(pval=1).key SIMPLE [(t1)t1.FieldSchema(name:key, type:string, comment:null), ] +POSTHOOK: Lineage: t2 PARTITION(pval=1).val SIMPLE [(t1)t1.FieldSchema(name:val, type:string, comment:null), ] +POSTHOOK: Lineage: t2 PARTITION(pval=1).key SIMPLE [(t1)t1.FieldSchema(name:key, type:string, comment:null), ] +POSTHOOK: Lineage: t2 PARTITION(pval=1).val SIMPLE [(t1)t1.FieldSchema(name:val, type:string, comment:null), ] +1 11 1 +2 12 1 +3 13 1 +7 17 1 +8 18 1 +8 28 1 +PREHOOK: query: drop table T1 +PREHOOK: type: DROPTABLE +PREHOOK: Input: default@t1 +PREHOOK: Output: default@t1 +POSTHOOK: query: drop table T1 +POSTHOOK: type: DROPTABLE +POSTHOOK: Input: default@t1 +POSTHOOK: Output: default@t1 +POSTHOOK: Lineage: t2 PARTITION(pval=1).key SIMPLE [(t1)t1.FieldSchema(name:key, type:string, comment:null), ] +POSTHOOK: Lineage: t2 PARTITION(pval=1).val SIMPLE [(t1)t1.FieldSchema(name:val, type:string, comment:null), ] +POSTHOOK: Lineage: t2 PARTITION(pval=1).key SIMPLE [(t1)t1.FieldSchema(name:key, type:string, comment:null), ] +POSTHOOK: Lineage: t2 PARTITION(pval=1).val SIMPLE [(t1)t1.FieldSchema(name:val, type:string, comment:null), ] +PREHOOK: query: drop table T2 +PREHOOK: type: DROPTABLE +PREHOOK: Input: default@t2 +PREHOOK: Output: default@t2 +POSTHOOK: query: drop table T2 +POSTHOOK: type: DROPTABLE +POSTHOOK: Input: default@t2 +POSTHOOK: Output: default@t2 +POSTHOOK: Lineage: t2 PARTITION(pval=1).key SIMPLE [(t1)t1.FieldSchema(name:key, type:string, comment:null), ] +POSTHOOK: Lineage: t2 PARTITION(pval=1).val SIMPLE [(t1)t1.FieldSchema(name:val, type:string, comment:null), ] +POSTHOOK: Lineage: t2 PARTITION(pval=1).key SIMPLE [(t1)t1.FieldSchema(name:key, type:string, comment:null), ] +POSTHOOK: Lineage: t2 PARTITION(pval=1).val SIMPLE [(t1)t1.FieldSchema(name:val, type:string, comment:null), ] diff --git ql/src/test/results/clientpositive/dbtxnmgr_query4.q.out ql/src/test/results/clientpositive/dbtxnmgr_query4.q.out new file mode 100644 index 0000000..75898f5 --- /dev/null +++ ql/src/test/results/clientpositive/dbtxnmgr_query4.q.out @@ -0,0 +1,115 @@ +PREHOOK: query: create table T1(key string, val string) stored as textfile +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +POSTHOOK: query: create table T1(key string, val string) stored as textfile +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@T1 +PREHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1 +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@t1 +POSTHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1 +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@t1 +PREHOOK: query: select * from T1 +PREHOOK: type: QUERY +PREHOOK: Input: default@t1 +#### A masked pattern was here #### +POSTHOOK: query: select * from T1 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@t1 +#### A masked pattern was here #### +1 11 +2 12 +3 13 +7 17 +8 18 +8 28 +PREHOOK: query: create table T2(key string) partitioned by (val string) stored as textfile +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +POSTHOOK: query: create table T2(key string) partitioned by (val string) stored as textfile +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@T2 +PREHOOK: query: insert overwrite table T2 partition (val) select key, val from T1 +PREHOOK: type: QUERY +PREHOOK: Input: default@t1 +PREHOOK: Output: default@t2 +POSTHOOK: query: insert overwrite table T2 partition (val) select key, val from T1 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@t1 +POSTHOOK: Output: default@t2@val=11 +POSTHOOK: Output: default@t2@val=12 +POSTHOOK: Output: default@t2@val=13 +POSTHOOK: Output: default@t2@val=17 +POSTHOOK: Output: default@t2@val=18 +POSTHOOK: Output: default@t2@val=28 +POSTHOOK: Lineage: t2 PARTITION(val=11).key SIMPLE [(t1)t1.FieldSchema(name:key, type:string, comment:null), ] +POSTHOOK: Lineage: t2 PARTITION(val=12).key SIMPLE [(t1)t1.FieldSchema(name:key, type:string, comment:null), ] +POSTHOOK: Lineage: t2 PARTITION(val=13).key SIMPLE [(t1)t1.FieldSchema(name:key, type:string, comment:null), ] +POSTHOOK: Lineage: t2 PARTITION(val=17).key SIMPLE [(t1)t1.FieldSchema(name:key, type:string, comment:null), ] +POSTHOOK: Lineage: t2 PARTITION(val=18).key SIMPLE [(t1)t1.FieldSchema(name:key, type:string, comment:null), ] +POSTHOOK: Lineage: t2 PARTITION(val=28).key SIMPLE [(t1)t1.FieldSchema(name:key, type:string, comment:null), ] +PREHOOK: query: select * from T2 +PREHOOK: type: QUERY +PREHOOK: Input: default@t2 +PREHOOK: Input: default@t2@val=11 +PREHOOK: Input: default@t2@val=12 +PREHOOK: Input: default@t2@val=13 +PREHOOK: Input: default@t2@val=17 +PREHOOK: Input: default@t2@val=18 +PREHOOK: Input: default@t2@val=28 +#### A masked pattern was here #### +POSTHOOK: query: select * from T2 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@t2 +POSTHOOK: Input: default@t2@val=11 +POSTHOOK: Input: default@t2@val=12 +POSTHOOK: Input: default@t2@val=13 +POSTHOOK: Input: default@t2@val=17 +POSTHOOK: Input: default@t2@val=18 +POSTHOOK: Input: default@t2@val=28 +#### A masked pattern was here #### +POSTHOOK: Lineage: t2 PARTITION(val=11).key SIMPLE [(t1)t1.FieldSchema(name:key, type:string, comment:null), ] +POSTHOOK: Lineage: t2 PARTITION(val=12).key SIMPLE [(t1)t1.FieldSchema(name:key, type:string, comment:null), ] +POSTHOOK: Lineage: t2 PARTITION(val=13).key SIMPLE [(t1)t1.FieldSchema(name:key, type:string, comment:null), ] +POSTHOOK: Lineage: t2 PARTITION(val=17).key SIMPLE [(t1)t1.FieldSchema(name:key, type:string, comment:null), ] +POSTHOOK: Lineage: t2 PARTITION(val=18).key SIMPLE [(t1)t1.FieldSchema(name:key, type:string, comment:null), ] +POSTHOOK: Lineage: t2 PARTITION(val=28).key SIMPLE [(t1)t1.FieldSchema(name:key, type:string, comment:null), ] +1 11 +2 12 +3 13 +7 17 +8 18 +8 28 +PREHOOK: query: drop table T1 +PREHOOK: type: DROPTABLE +PREHOOK: Input: default@t1 +PREHOOK: Output: default@t1 +POSTHOOK: query: drop table T1 +POSTHOOK: type: DROPTABLE +POSTHOOK: Input: default@t1 +POSTHOOK: Output: default@t1 +POSTHOOK: Lineage: t2 PARTITION(val=11).key SIMPLE [(t1)t1.FieldSchema(name:key, type:string, comment:null), ] +POSTHOOK: Lineage: t2 PARTITION(val=12).key SIMPLE [(t1)t1.FieldSchema(name:key, type:string, comment:null), ] +POSTHOOK: Lineage: t2 PARTITION(val=13).key SIMPLE [(t1)t1.FieldSchema(name:key, type:string, comment:null), ] +POSTHOOK: Lineage: t2 PARTITION(val=17).key SIMPLE [(t1)t1.FieldSchema(name:key, type:string, comment:null), ] +POSTHOOK: Lineage: t2 PARTITION(val=18).key SIMPLE [(t1)t1.FieldSchema(name:key, type:string, comment:null), ] +POSTHOOK: Lineage: t2 PARTITION(val=28).key SIMPLE [(t1)t1.FieldSchema(name:key, type:string, comment:null), ] +PREHOOK: query: drop table T2 +PREHOOK: type: DROPTABLE +PREHOOK: Input: default@t2 +PREHOOK: Output: default@t2 +POSTHOOK: query: drop table T2 +POSTHOOK: type: DROPTABLE +POSTHOOK: Input: default@t2 +POSTHOOK: Output: default@t2 +POSTHOOK: Lineage: t2 PARTITION(val=11).key SIMPLE [(t1)t1.FieldSchema(name:key, type:string, comment:null), ] +POSTHOOK: Lineage: t2 PARTITION(val=12).key SIMPLE [(t1)t1.FieldSchema(name:key, type:string, comment:null), ] +POSTHOOK: Lineage: t2 PARTITION(val=13).key SIMPLE [(t1)t1.FieldSchema(name:key, type:string, comment:null), ] +POSTHOOK: Lineage: t2 PARTITION(val=17).key SIMPLE [(t1)t1.FieldSchema(name:key, type:string, comment:null), ] +POSTHOOK: Lineage: t2 PARTITION(val=18).key SIMPLE [(t1)t1.FieldSchema(name:key, type:string, comment:null), ] +POSTHOOK: Lineage: t2 PARTITION(val=28).key SIMPLE [(t1)t1.FieldSchema(name:key, type:string, comment:null), ] diff --git ql/src/test/results/clientpositive/dbtxnmgr_query5.q.out ql/src/test/results/clientpositive/dbtxnmgr_query5.q.out new file mode 100644 index 0000000..4408256 --- /dev/null +++ ql/src/test/results/clientpositive/dbtxnmgr_query5.q.out @@ -0,0 +1,77 @@ +PREHOOK: query: create database foo +PREHOOK: type: CREATEDATABASE +POSTHOOK: query: create database foo +POSTHOOK: type: CREATEDATABASE +PREHOOK: query: use foo +PREHOOK: type: SWITCHDATABASE +POSTHOOK: query: use foo +POSTHOOK: type: SWITCHDATABASE +PREHOOK: query: create table T1(key string, val string) partitioned by (ds string) stored as textfile +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:foo +POSTHOOK: query: create table T1(key string, val string) partitioned by (ds string) stored as textfile +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:foo +POSTHOOK: Output: foo@T1 +PREHOOK: query: alter table T1 add partition (ds='today') +PREHOOK: type: ALTERTABLE_ADDPARTS +PREHOOK: Output: foo@t1 +POSTHOOK: query: alter table T1 add partition (ds='today') +POSTHOOK: type: ALTERTABLE_ADDPARTS +POSTHOOK: Output: foo@t1 +POSTHOOK: Output: foo@t1@ds=today +PREHOOK: query: create view V1 as select key from T1 +PREHOOK: type: CREATEVIEW +PREHOOK: Input: foo@t1 +POSTHOOK: query: create view V1 as select key from T1 +POSTHOOK: type: CREATEVIEW +POSTHOOK: Input: foo@t1 +POSTHOOK: Output: foo@V1 +PREHOOK: query: show tables +PREHOOK: type: SHOWTABLES +POSTHOOK: query: show tables +POSTHOOK: type: SHOWTABLES +t1 +v1 +PREHOOK: query: describe T1 +PREHOOK: type: DESCTABLE +POSTHOOK: query: describe T1 +POSTHOOK: type: DESCTABLE +key string None +val string None +ds string None + +# Partition Information +# col_name data_type comment + +ds string None +PREHOOK: query: drop view V1 +PREHOOK: type: DROPVIEW +PREHOOK: Input: foo@v1 +PREHOOK: Output: foo@v1 +POSTHOOK: query: drop view V1 +POSTHOOK: type: DROPVIEW +POSTHOOK: Input: foo@v1 +POSTHOOK: Output: foo@v1 +PREHOOK: query: drop table T1 +PREHOOK: type: DROPTABLE +PREHOOK: Input: foo@t1 +PREHOOK: Output: foo@t1 +POSTHOOK: query: drop table T1 +POSTHOOK: type: DROPTABLE +POSTHOOK: Input: foo@t1 +POSTHOOK: Output: foo@t1 +PREHOOK: query: show databases +PREHOOK: type: SHOWDATABASES +POSTHOOK: query: show databases +POSTHOOK: type: SHOWDATABASES +default +foo +PREHOOK: query: drop database foo +PREHOOK: type: DROPDATABASE +PREHOOK: Input: database:foo +PREHOOK: Output: database:foo +POSTHOOK: query: drop database foo +POSTHOOK: type: DROPDATABASE +POSTHOOK: Input: database:foo +POSTHOOK: Output: database:foo