From 0ccbba8c9b45a909d10b3f50b233ee7c98f83c0a Mon Sep 17 00:00:00 2001 From: rahulgidwani Date: Tue, 6 Jan 2015 17:09:30 -0800 Subject: [PATCH] maybe it works --- pom.xml | 2 +- .../hbase/client/replication/ReplicationAdmin.java | 21 +- .../hbase/replication/ReplicationZookeeper.java | 38 +- .../replication/regionserver/Replication.java | 12 +- .../replication/regionserver/ReplicationSink.java | 67 +- .../regionserver/ReplicationSource.java | 81 +- .../regionserver/ReplicationSyncUp.java | 2 +- .../thrift/ChainedTTransportFactory.java | 40 + .../hbase/replication/thrift/TFilterTransport.java | 99 + .../replication/thrift/TUGIAssumingTransport.java | 73 + .../thrift/TUGIAssumingTransportFactory.java | 55 + .../hbase/replication/thrift/ThriftAdaptors.java | 181 + .../hbase/replication/thrift/ThriftClient.java | 186 + .../hbase/replication/thrift/ThriftEditType.java | 58 + .../thrift/ThriftHBaseServiceHandler.java | 103 + .../hbase/replication/thrift/ThriftServer.java | 184 + .../hbase/replication/thrift/ThriftUtilities.java | 73 + .../replication/thrift/generated/TBatchEdit.java | 435 ++ .../replication/thrift/generated/TClusterId.java | 480 ++ .../replication/thrift/generated/TColumnValue.java | 910 +++ .../hbase/replication/thrift/generated/TEdit.java | 637 ++ .../thrift/generated/THBaseService.java | 2310 ++++++ .../replication/thrift/generated/THLogKey.java | 582 ++ .../replication/thrift/generated/TIOError.java | 391 + .../thrift/generated/TIllegalArgument.java | 390 + .../replication/thrift/generated/TWalLEdit.java | 439 ++ .../hadoop/hbase/thrift/ThriftServerRunner.java | 195 +- .../hbase/thrift/generated/AlreadyExists.java | 17 +- .../hbase/thrift/generated/BatchMutation.java | 26 +- .../hbase/thrift/generated/ColumnDescriptor.java | 91 +- .../hadoop/hbase/thrift/generated/Hbase.java | 7617 ++++++++++++++------ .../hadoop/hbase/thrift/generated/IOError.java | 16 +- .../hbase/thrift/generated/IllegalArgument.java | 16 +- .../hadoop/hbase/thrift/generated/Mutation.java | 47 +- .../hadoop/hbase/thrift/generated/TCell.java | 33 +- .../hadoop/hbase/thrift/generated/TColumn.java | 23 +- .../hadoop/hbase/thrift/generated/TIncrement.java | 41 +- .../hadoop/hbase/thrift/generated/TRegionInfo.java | 65 +- .../hadoop/hbase/thrift/generated/TRowResult.java | 40 +- .../hadoop/hbase/thrift/generated/TScan.java | 71 +- .../hadoop/hbase/thrift2/generated/TColumn.java | 37 +- .../hbase/thrift2/generated/TColumnIncrement.java | 37 +- .../hbase/thrift2/generated/TColumnValue.java | 41 +- .../hadoop/hbase/thrift2/generated/TDelete.java | 88 +- .../hbase/thrift2/generated/TDeleteType.java | 2 +- .../hbase/thrift2/generated/TDurability.java | 2 +- .../hadoop/hbase/thrift2/generated/TGet.java | 91 +- .../hbase/thrift2/generated/THBaseService.java | 1759 ++++- .../hadoop/hbase/thrift2/generated/TIOError.java | 16 +- .../hbase/thrift2/generated/TIllegalArgument.java | 16 +- .../hadoop/hbase/thrift2/generated/TIncrement.java | 75 +- .../hadoop/hbase/thrift2/generated/TMutation.java | 10 +- .../hadoop/hbase/thrift2/generated/TPut.java | 84 +- .../hadoop/hbase/thrift2/generated/TResult.java | 26 +- .../hbase/thrift2/generated/TRowMutations.java | 58 +- .../hadoop/hbase/thrift2/generated/TScan.java | 105 +- .../hadoop/hbase/thrift2/generated/TTimeRange.java | 39 +- .../hadoop/hbase/replication/thrift/hbase.thrift | 97 + src/main/ruby/hbase/replication_admin.rb | 10 +- src/main/ruby/shell/commands/add_peer.rb | 6 +- src/main/ruby/shell/commands/list_peers.rb | 5 +- .../client/replication/TestReplicationAdmin.java | 37 + .../hbase/replication/TestMasterReplication.java | 434 +- .../replication/TestMultiSlaveReplication.java | 8 +- .../hbase/replication/TestReplicationBase.java | 7 +- .../replication/TestReplicationSmallTests.java | 2 +- .../replication/TestReplicationZookeeper.java | 6 + .../regionserver/TestReplicationSink.java | 41 +- .../replication/thrift/ReplicationTestUtils.java | 167 + .../thrift/TestThriftCyclicReplication.java | 114 + .../thrift/TestThriftMasterMasterReplication.java | 94 + .../thrift/TestThriftMultiSlaveReplication.java | 123 + .../thrift/TestThriftReplicationBase.java | 14 + .../thrift/TestThriftReplicationSink.java | 275 + .../thrift/TestThriftReplicationSmallTests.java | 594 ++ .../replication/thrift/ThriftAdaptorsTest.java | 90 + 76 files changed, 16985 insertions(+), 3772 deletions(-) create mode 100644 src/main/java/org/apache/hadoop/hbase/replication/thrift/ChainedTTransportFactory.java create mode 100644 src/main/java/org/apache/hadoop/hbase/replication/thrift/TFilterTransport.java create mode 100644 src/main/java/org/apache/hadoop/hbase/replication/thrift/TUGIAssumingTransport.java create mode 100644 src/main/java/org/apache/hadoop/hbase/replication/thrift/TUGIAssumingTransportFactory.java create mode 100644 src/main/java/org/apache/hadoop/hbase/replication/thrift/ThriftAdaptors.java create mode 100644 src/main/java/org/apache/hadoop/hbase/replication/thrift/ThriftClient.java create mode 100644 src/main/java/org/apache/hadoop/hbase/replication/thrift/ThriftEditType.java create mode 100644 src/main/java/org/apache/hadoop/hbase/replication/thrift/ThriftHBaseServiceHandler.java create mode 100644 src/main/java/org/apache/hadoop/hbase/replication/thrift/ThriftServer.java create mode 100644 src/main/java/org/apache/hadoop/hbase/replication/thrift/ThriftUtilities.java create mode 100644 src/main/java/org/apache/hadoop/hbase/replication/thrift/generated/TBatchEdit.java create mode 100644 src/main/java/org/apache/hadoop/hbase/replication/thrift/generated/TClusterId.java create mode 100644 src/main/java/org/apache/hadoop/hbase/replication/thrift/generated/TColumnValue.java create mode 100644 src/main/java/org/apache/hadoop/hbase/replication/thrift/generated/TEdit.java create mode 100644 src/main/java/org/apache/hadoop/hbase/replication/thrift/generated/THBaseService.java create mode 100644 src/main/java/org/apache/hadoop/hbase/replication/thrift/generated/THLogKey.java create mode 100644 src/main/java/org/apache/hadoop/hbase/replication/thrift/generated/TIOError.java create mode 100644 src/main/java/org/apache/hadoop/hbase/replication/thrift/generated/TIllegalArgument.java create mode 100644 src/main/java/org/apache/hadoop/hbase/replication/thrift/generated/TWalLEdit.java create mode 100644 src/main/resources/org/apache/hadoop/hbase/replication/thrift/hbase.thrift create mode 100644 src/test/java/org/apache/hadoop/hbase/replication/thrift/ReplicationTestUtils.java create mode 100644 src/test/java/org/apache/hadoop/hbase/replication/thrift/TestThriftCyclicReplication.java create mode 100644 src/test/java/org/apache/hadoop/hbase/replication/thrift/TestThriftMasterMasterReplication.java create mode 100644 src/test/java/org/apache/hadoop/hbase/replication/thrift/TestThriftMultiSlaveReplication.java create mode 100644 src/test/java/org/apache/hadoop/hbase/replication/thrift/TestThriftReplicationBase.java create mode 100644 src/test/java/org/apache/hadoop/hbase/replication/thrift/TestThriftReplicationSink.java create mode 100644 src/test/java/org/apache/hadoop/hbase/replication/thrift/TestThriftReplicationSmallTests.java create mode 100644 src/test/java/org/apache/hadoop/hbase/replication/thrift/ThriftAdaptorsTest.java diff --git a/pom.xml b/pom.xml index 1216ec8..5a250b8 100644 --- a/pom.xml +++ b/pom.xml @@ -1058,7 +1058,7 @@ 1.8.5 2.4.0a 1.0.1 - 0.8.0 + 0.9.1 3.4.5 0.0.1-SNAPSHOT 2.6.3 diff --git a/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationAdmin.java b/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationAdmin.java index ac3bc81..92d6dd3 100644 --- a/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationAdmin.java +++ b/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationAdmin.java @@ -113,7 +113,12 @@ public class ReplicationAdmin implements Closeable { * multi-slave isn't supported yet. */ public void addPeer(String id, String clusterKey) throws IOException { - this.replicationZk.addPeer(id, clusterKey); + addPeer(id, clusterKey, ReplicationZookeeper.PeerProtocol.NATIVE.name()); + } + + public void addPeer(String id, String clusterKey, String protocol) + throws IOException { + this.replicationZk.addPeer(id, clusterKey, ReplicationZookeeper.PeerProtocol.valueOf(protocol)); } /** @@ -171,6 +176,20 @@ public class ReplicationAdmin implements Closeable { } /** + * Get the protocol for the peer + * + * @param id peer's identifier + * @return current protocol of the peer + */ + public String getPeerProtocol(String id) throws IOException { + try { + return this.replicationZk.getPeerProtocol(id).name(); + } catch (KeeperException e) { + throw new IOException("Couldn't get the protocol of the peer " + id, e); + } + } + + /** * Get the current status of the kill switch, if the cluster is replicating * or not. * @return true if the cluster is replicated, otherwise false diff --git a/src/main/java/org/apache/hadoop/hbase/replication/ReplicationZookeeper.java b/src/main/java/org/apache/hadoop/hbase/replication/ReplicationZookeeper.java index 6436f0b..52210cf 100644 --- a/src/main/java/org/apache/hadoop/hbase/replication/ReplicationZookeeper.java +++ b/src/main/java/org/apache/hadoop/hbase/replication/ReplicationZookeeper.java @@ -92,6 +92,11 @@ public class ReplicationZookeeper { ENABLED, DISABLED }; + // Values of znode which stores protocol of a peer + public static enum PeerProtocol { + NATIVE, THRIFT + } + // Our handle on zookeeper private final ZooKeeperWatcher zookeeper; // Map of peer clusters keyed by their id @@ -108,6 +113,8 @@ public class ReplicationZookeeper { private String replicationStateNodeName; // Name of zk node which stores peer state private String peerStateNodeName; + // Name of zk node which stores peer protocol + private String peerProtocolNodeName; private final Configuration conf; // Is this cluster replicating at the moment? private AtomicBoolean replicating; @@ -164,6 +171,8 @@ public class ReplicationZookeeper { conf.get("zookeeper.znode.replication.peers", "peers"); this.peerStateNodeName = conf.get( "zookeeper.znode.replication.peers.state", "peer-state"); + this.peerProtocolNodeName = conf.get( + "zookeeper.znode.replication.peers.protocol", "peer-protocol"); this.replicationStateNodeName = conf.get("zookeeper.znode.replication.state", "state"); String rsZNodeName = @@ -389,18 +398,24 @@ public class ReplicationZookeeper { * multi-slave isn't supported yet. */ public void addPeer(String id, String clusterKey) throws IOException { + addPeer(id, clusterKey, PeerProtocol.NATIVE); + } + + public void addPeer(String id, String clusterKey, PeerProtocol protocol) throws IOException { try { if (peerExists(id)) { throw new IllegalArgumentException("Cannot add existing peer"); } ZKUtil.createWithParents(this.zookeeper, this.peersZNode); ZKUtil.createAndWatch(this.zookeeper, ZKUtil.joinZNode(this.peersZNode, id), - Bytes.toBytes(clusterKey)); + Bytes.toBytes(clusterKey)); + ZKUtil.createNodeIfNotExistsAndWatch(this.zookeeper, getPeerProtocolNode(id), + Bytes.toBytes(protocol.name())); // There is a race b/w PeerWatcher and ReplicationZookeeper#add method to create the // peer-state znode. This happens while adding a peer. // The peer state data is set as "ENABLED" by default. ZKUtil.createNodeIfNotExistsAndWatch(this.zookeeper, getPeerStateNode(id), - Bytes.toBytes(PeerState.ENABLED.name())); + Bytes.toBytes(PeerState.ENABLED.name())); } catch (KeeperException e) { throw new IOException("Unable to add peer", e); } @@ -467,6 +482,20 @@ public class ReplicationZookeeper { } /** + * Get the transport protocol of the peer. This method checks the state by connecting to ZK. + * + * @param id peer's identifier + * @return current protocol of the peer + */ + public PeerProtocol getPeerProtocol(String id) throws KeeperException { + byte[] peerProtocolBytes = ZKUtil.getData(this.zookeeper, getPeerProtocolNode(id)); + if (peerProtocolBytes == null) { + return PeerProtocol.NATIVE; + } + return PeerProtocol.valueOf(Bytes.toString(peerProtocolBytes)); + } + + /** * Check whether the peer is enabled or not. This method checks the atomic * boolean of ReplicationPeer locally. * @@ -487,6 +516,11 @@ public class ReplicationZookeeper { ZKUtil.joinZNode(id, this.peerStateNodeName)); } + private String getPeerProtocolNode(String id) { + return ZKUtil.joinZNode(this.peersZNode, + ZKUtil.joinZNode(id, this.peerProtocolNodeName)); + } + /** * This reads the state znode for replication and sets the atomic boolean */ diff --git a/src/main/java/org/apache/hadoop/hbase/replication/regionserver/Replication.java b/src/main/java/org/apache/hadoop/hbase/replication/regionserver/Replication.java index 13a80bf..16b8a95 100644 --- a/src/main/java/org/apache/hadoop/hbase/replication/regionserver/Replication.java +++ b/src/main/java/org/apache/hadoop/hbase/replication/regionserver/Replication.java @@ -40,6 +40,7 @@ import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener; import org.apache.hadoop.hbase.replication.ReplicationZookeeper; import org.apache.hadoop.hbase.replication.master.ReplicationLogCleaner; import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.zookeeper.ClusterId; import org.apache.zookeeper.KeeperException; import static org.apache.hadoop.hbase.HConstants.HBASE_MASTER_LOGCLEANER_PLUGINS; @@ -151,7 +152,16 @@ public class Replication implements WALActionsListener, public void startReplicationService() throws IOException { if (this.replication) { this.replicationManager.init(); - this.replicationSink = new ReplicationSink(this.conf, this.server); + try { + this.replicationSink = new ReplicationSink( + this.conf, + this.server, + ClusterId.readClusterIdZNode(zkHelper.getZookeeperWatcher()) + ); + } catch (KeeperException e) { + throw new IOException(e); + } + } } diff --git a/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java b/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java index b11db60..858cbf9 100644 --- a/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java +++ b/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java @@ -22,9 +22,7 @@ package org.apache.hadoop.hbase.replication.regionserver; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.HBaseConfiguration; -import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.*; import org.apache.hadoop.hbase.client.Delete; import org.apache.hadoop.hbase.client.HConnection; import org.apache.hadoop.hbase.client.HConnectionManager; @@ -33,17 +31,17 @@ import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Row; import org.apache.hadoop.hbase.regionserver.wal.HLog; import org.apache.hadoop.hbase.regionserver.wal.WALEdit; +import org.apache.hadoop.hbase.replication.thrift.ThriftAdaptors; +import org.apache.hadoop.hbase.replication.thrift.ThriftServer; +import org.apache.hadoop.hbase.replication.thrift.generated.TBatchEdit; +import org.apache.hadoop.hbase.replication.thrift.generated.THBaseService; +import org.apache.hadoop.hbase.replication.thrift.generated.TIOError; import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.hbase.Stoppable; +import org.apache.hadoop.hbase.zookeeper.ClusterId; +import org.apache.thrift.TException; import java.io.IOException; -import java.util.ArrayList; -import java.util.Collection; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.TreeMap; -import java.util.UUID; +import java.util.*; /** * This class is responsible for replicating the edits coming @@ -56,31 +54,29 @@ import java.util.UUID; *

* This class uses the native HBase client in order to replicate entries. *

- * - * TODO make this class more like ReplicationSource wrt log handling */ -public class ReplicationSink { +public class ReplicationSink implements THBaseService.Iface { private static final Log LOG = LogFactory.getLog(ReplicationSink.class); - // Name of the HDFS directory that contains the temporary rep logs - public static final String REPLICATION_LOG_DIR = ".replogs"; + public static String CONF_KEY_REPLICATION_THRIFT = "hbase.replication.sink.enable.thrift"; private final Configuration conf; private final HConnection sharedHtableCon; private final ReplicationSinkMetrics metrics; + private final String clusterId; + private ThriftServer thriftServer; - /** - * Create a sink for replication - * - * @param conf conf object - * @param stopper boolean to tell this thread to stop - * @throws IOException thrown when HDFS goes bad or bad file name - */ - public ReplicationSink(Configuration conf, Stoppable stopper) + public ReplicationSink(Configuration conf, Server server, String clusterId) throws IOException { + this.clusterId = clusterId; this.conf = HBaseConfiguration.create(conf); decorateConf(); this.sharedHtableCon = HConnectionManager.createConnection(this.conf); this.metrics = new ReplicationSinkMetrics(); + if(conf.getBoolean(CONF_KEY_REPLICATION_THRIFT, false)) { + LOG.info("Starting up thrift server for replication"); + thriftServer = new ThriftServer(conf, this, server.getServerName()); + thriftServer.start(); + } } /** @@ -186,16 +182,20 @@ public class ReplicationSink { return values; } + /** * stop the thread pool executor. It is called when the regionserver is stopped. */ public void stopReplicationSinkServices() { try { + if (thriftServer != null) { + thriftServer.shutdown(); + } this.sharedHtableCon.close(); } catch (IOException e) { LOG.warn("IOException while closing the connection", e); // ignoring as we are closing. } - } + } /** * Do the changes and handle the pool @@ -222,4 +222,21 @@ public class ReplicationSink { } } } + + @Override public void replicate(TBatchEdit edits) throws TIOError, TException { + try { + replicateEntries(ThriftAdaptors.REPLICATION_BATCH_ADAPTOR.fromThrift(edits)); + } catch (IOException e) { + throw new TException("Failed to replicate", e); + } + } + + @Override public void ping() throws TException { + // maybe log at trace level for debugging purposes? + LOG.trace("PING CALLED"); + } + + @Override public String getClusterUUID() throws TException { + return clusterId; + } } diff --git a/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java b/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java index 2916ba7..66c5d6d 100644 --- a/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java +++ b/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java @@ -24,14 +24,7 @@ import java.io.FileNotFoundException; import java.io.IOException; import java.net.ConnectException; import java.net.SocketTimeoutException; -import java.util.ArrayList; -import java.util.Collections; -import java.util.Comparator; -import java.util.HashSet; -import java.util.List; -import java.util.Random; -import java.util.Set; -import java.util.UUID; +import java.util.*; import java.util.concurrent.CountDownLatch; import java.util.concurrent.PriorityBlockingQueue; import java.util.concurrent.TimeUnit; @@ -55,6 +48,8 @@ import org.apache.hadoop.hbase.regionserver.wal.HLog; import org.apache.hadoop.hbase.regionserver.wal.HLogKey; import org.apache.hadoop.hbase.regionserver.wal.WALEdit; import org.apache.hadoop.hbase.replication.ReplicationZookeeper; +import org.apache.hadoop.hbase.replication.thrift.ThriftClient; +import org.apache.hadoop.hbase.replication.thrift.ThriftUtilities; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Threads; import org.apache.hadoop.ipc.RemoteException; @@ -134,6 +129,9 @@ public class ReplicationSource extends Thread private ReplicationSourceMetrics metrics; // Handle on the log reader helper private ReplicationHLogReaderManager repLogReader; + // thrift client + private ThriftClient thriftClient; + /** @@ -184,9 +182,19 @@ public class ReplicationSource extends Thread } catch (KeeperException ke) { throw new IOException("Could not read cluster id", ke); } - // Finally look if this is a recovered queue this.checkIfQueueRecovered(peerClusterZnode); + + try { + if (zkHelper.getPeerProtocol(peerClusterZnode) + .equals(ReplicationZookeeper.PeerProtocol.THRIFT)) { + thriftClient = new ThriftClient(conf, peerId); + LOG.info("Starting new replication peer " + peerId + " with Thrift protocol over port: " + + "" + ThriftUtilities.getDestinationPeerPort(conf, peerId)); + } + } catch (KeeperException ke) { + throw new IOException("Could not read cluster id", ke); + } } // The passed znode will be either the id of the peer cluster or @@ -292,7 +300,7 @@ public class ReplicationSource extends Thread int sleepMultiplier = 1; // delay this until we are in an asynchronous thread while (this.isActive() && this.peerClusterId == null) { - this.peerClusterId = zkHelper.getPeerUUID(this.peerId); + this.peerClusterId = getPeerClusterUUID(); if (this.isActive() && this.peerClusterId == null) { if (sleepForRetries("Cannot contact the peer's zk ensemble", sleepMultiplier)) { sleepMultiplier++; @@ -744,10 +752,8 @@ public class ReplicationSource extends Thread continue; } try { - HRegionInterface rrs = getRS(); LOG.debug("Replicating " + entries.size() + ", " + this.currentSize + " bytes"); - // can't avoid the copy here, the replicateLogEntries RPC require an HLog.Entry[] - rrs.replicateLogEntries(entries.toArray(new HLog.Entry[entries.size()])); + shipIt(chooseSink(), entries.toArray(new HLog.Entry[entries.size()])); if (this.lastLoggedPosition != this.repLogReader.getPosition()) { this.manager.logPositionAndCleanOldLogs(this.currentPath, this.peerClusterZnode, this.repLogReader.getPosition(), queueRecovered, currentWALisBeingWrittenTo); @@ -810,6 +816,34 @@ public class ReplicationSource extends Thread } } + private void shipIt(ServerName address, HLog.Entry[] entries) throws IOException { + if(thriftClient != null) { + thriftClient.shipEdits(address, entries); + } else { + HRegionInterface rrs = getRS(address); + rrs.replicateLogEntries(entries); + } + } + + protected void pingRS(ServerName address) throws IOException { + if(thriftClient != null) { + thriftClient.ping(address); + } else { + HRegionInterface rrs = getRS(address); + rrs.getHServerInfo(); + } + } + + private UUID getPeerClusterUUID() { + if (thriftClient == null) { + return zkHelper.getPeerUUID(this.peerId); + } else { + return thriftClient.getPeerClusterUUID( + currentPeers.get(random.nextInt(this.currentPeers.size())) + ); + } + } + /** * check whether the peer is enabled or not * @@ -874,17 +908,22 @@ public class ReplicationSource extends Thread } } - /** - * Get a new region server at random from this peer - * @return - * @throws IOException - */ - private HRegionInterface getRS() throws IOException { + + private ServerName chooseSink() throws IOException { if (this.currentPeers.size() == 0) { throw new IOException(this.peerClusterZnode + " has 0 region servers"); } ServerName address = currentPeers.get(random.nextInt(this.currentPeers.size())); + return address; + } + + /** + * Get a new region server at random from this peer + * @return + * @throws IOException + */ + private HRegionInterface getRS(ServerName address) throws IOException { return this.conn.getHRegionConnection(address.getHostname(), address.getPort()); } @@ -898,9 +937,7 @@ public class ReplicationSource extends Thread Thread pingThread = new Thread() { public void run() { try { - HRegionInterface rrs = getRS(); - // Dummy call which should fail - rrs.getHServerInfo(); + pingRS(chooseSink()); latch.countDown(); } catch (IOException ex) { if (ex instanceof RemoteException) { diff --git a/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSyncUp.java b/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSyncUp.java index 351bc21..16045d7 100644 --- a/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSyncUp.java +++ b/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSyncUp.java @@ -39,7 +39,7 @@ import org.apache.hadoop.util.ToolRunner; /** * In a scenario of Replication based Disaster/Recovery, when hbase * Master-Cluster crashes, this tool is used to sync-up the delta from Master to - * Slave using the info from Zookeeper. The tool will run on Master-Cluser, and + * Slave using the info from Zookeeper. The tool will run on Master-Cluster, and * assume ZK, Filesystem and NetWork still available after hbase crashes * * hbase org.apache.hadoop.hbase.replication.regionserver.ReplicationSyncUp diff --git a/src/main/java/org/apache/hadoop/hbase/replication/thrift/ChainedTTransportFactory.java b/src/main/java/org/apache/hadoop/hbase/replication/thrift/ChainedTTransportFactory.java new file mode 100644 index 0000000..8748f16 --- /dev/null +++ b/src/main/java/org/apache/hadoop/hbase/replication/thrift/ChainedTTransportFactory.java @@ -0,0 +1,40 @@ +/* + * Copyright 2010 The Apache Software Foundation + * + * 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.hbase.replication.thrift; + +import org.apache.thrift.transport.TTransport; +import org.apache.thrift.transport.TTransportFactory; + +final class ChainedTTransportFactory extends TTransportFactory { + private final TTransportFactory parentTransFactory; + private final TTransportFactory childTransFactory; + + ChainedTTransportFactory( + TTransportFactory parentTransFactory, + TTransportFactory childTransFactory) { + this.parentTransFactory = parentTransFactory; + this.childTransFactory = childTransFactory; + } + + @Override + public TTransport getTransport(TTransport trans) { + return childTransFactory.getTransport(parentTransFactory.getTransport(trans)); + } +} diff --git a/src/main/java/org/apache/hadoop/hbase/replication/thrift/TFilterTransport.java b/src/main/java/org/apache/hadoop/hbase/replication/thrift/TFilterTransport.java new file mode 100644 index 0000000..328b261 --- /dev/null +++ b/src/main/java/org/apache/hadoop/hbase/replication/thrift/TFilterTransport.java @@ -0,0 +1,99 @@ +/** + * 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.hbase.replication.thrift; + +import org.apache.thrift.transport.TTransport; +import org.apache.thrift.transport.TTransportException; + +/** + * Transport that simply wraps another transport. + * This is the equivalent of FilterInputStream for Thrift transports. + */ + public class TFilterTransport extends TTransport { + protected final TTransport wrapped; + + public TFilterTransport(TTransport wrapped) { + this.wrapped = wrapped; + } + + @Override + public void open() throws TTransportException { + wrapped.open(); + } + + @Override + public boolean isOpen() { + return wrapped.isOpen(); + } + + @Override + public boolean peek() { + return wrapped.peek(); + } + + @Override + public void close() { + wrapped.close(); + } + + @Override + public int read(byte[] buf, int off, int len) throws TTransportException { + return wrapped.read(buf, off, len); + } + + @Override + public int readAll(byte[] buf, int off, int len) throws TTransportException { + return wrapped.readAll(buf, off, len); + } + + @Override + public void write(byte[] buf) throws TTransportException { + wrapped.write(buf); + } + + @Override + public void write(byte[] buf, int off, int len) throws TTransportException { + wrapped.write(buf, off, len); + } + + @Override + public void flush() throws TTransportException { + wrapped.flush(); + } + + @Override + public byte[] getBuffer() { + return wrapped.getBuffer(); + } + + @Override + public int getBufferPosition() { + return wrapped.getBufferPosition(); + } + + @Override + public int getBytesRemainingInBuffer() { + return wrapped.getBytesRemainingInBuffer(); + } + + @Override + public void consumeBuffer(int len) { + wrapped.consumeBuffer(len); + } + } diff --git a/src/main/java/org/apache/hadoop/hbase/replication/thrift/TUGIAssumingTransport.java b/src/main/java/org/apache/hadoop/hbase/replication/thrift/TUGIAssumingTransport.java new file mode 100644 index 0000000..c8bcee3 --- /dev/null +++ b/src/main/java/org/apache/hadoop/hbase/replication/thrift/TUGIAssumingTransport.java @@ -0,0 +1,73 @@ +/** + * 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.hbase.replication.thrift; + +import java.io.IOException; +import java.security.PrivilegedExceptionAction; + +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.thrift.transport.TTransport; +import org.apache.thrift.transport.TTransportException; + +/** + * The Thrift SASL transports call Sasl.createSaslServer and Sasl.createSaslClient + * inside open(). So, we need to assume the correct UGI when the transport is opened + * so that the SASL mechanisms have access to the right principal. This transport + * wraps the Sasl transports to set up the right UGI context for open(). + * + * This is used on the client side, where the API explicitly opens a transport to + * the server. + */ + public class TUGIAssumingTransport extends TFilterTransport { + protected UserGroupInformation ugi; + + public TUGIAssumingTransport(TTransport wrapped, UserGroupInformation ugi) { + super(wrapped); + this.ugi = ugi; + } + + @Override + public void open() throws TTransportException { + try { + ugi.doAs(new PrivilegedExceptionAction() { + public Void run() { + try { + wrapped.open(); + } catch (TTransportException tte) { + // Wrap the transport exception in an RTE, since UGI.doAs() then goes + // and unwraps this for us out of the doAs block. We then unwrap one + // more time in our catch clause to get back the TTE. (ugh) + throw new RuntimeException(tte); + } + return null; + } + }); + } catch (IOException ioe) { + throw new RuntimeException("Received an ioe we never threw!", ioe); + } catch (InterruptedException ie) { + throw new RuntimeException("Received an ie we never threw!", ie); + } catch (RuntimeException rte) { + if (rte.getCause() instanceof TTransportException) { + throw (TTransportException)rte.getCause(); + } else { + throw rte; + } + } + } + } diff --git a/src/main/java/org/apache/hadoop/hbase/replication/thrift/TUGIAssumingTransportFactory.java b/src/main/java/org/apache/hadoop/hbase/replication/thrift/TUGIAssumingTransportFactory.java new file mode 100644 index 0000000..daa32be --- /dev/null +++ b/src/main/java/org/apache/hadoop/hbase/replication/thrift/TUGIAssumingTransportFactory.java @@ -0,0 +1,55 @@ +/* + * Copyright 2010 The Apache Software Foundation + * + * 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.hbase.replication.thrift; + +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.thrift.transport.TTransport; +import org.apache.thrift.transport.TTransportFactory; + +import java.security.PrivilegedAction; + +/** + * A TransportFactory that wraps another one, but assumes a specified UGI + * before calling through. + * + * This is used on the server side to assume the server's Principal when accepting + * clients. + */ +public class TUGIAssumingTransportFactory extends TTransportFactory { + private final UserGroupInformation ugi; + private final TTransportFactory wrapped; + + public TUGIAssumingTransportFactory(TTransportFactory wrapped, UserGroupInformation ugi) { + assert wrapped != null; + assert ugi != null; + + this.wrapped = wrapped; + this.ugi = ugi; + } + + @Override + public TTransport getTransport(final TTransport trans) { + return ugi.doAs(new PrivilegedAction() { + public TTransport run() { + return wrapped.getTransport(trans); + } + }); + } +} diff --git a/src/main/java/org/apache/hadoop/hbase/replication/thrift/ThriftAdaptors.java b/src/main/java/org/apache/hadoop/hbase/replication/thrift/ThriftAdaptors.java new file mode 100644 index 0000000..6e892bf --- /dev/null +++ b/src/main/java/org/apache/hadoop/hbase/replication/thrift/ThriftAdaptors.java @@ -0,0 +1,181 @@ +package org.apache.hadoop.hbase.replication.thrift; + +import com.google.common.base.Function; +import com.google.common.collect.Lists; +import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.regionserver.wal.HLog; +import org.apache.hadoop.hbase.regionserver.wal.HLogKey; +import org.apache.hadoop.hbase.regionserver.wal.WALEdit; +import org.apache.hadoop.hbase.replication.thrift.generated.*; + +import java.util.List; +import java.util.UUID; + +public class ThriftAdaptors { + + interface Adaptor { + + K fromThrift(V vee); + + V toThrift(K kay); + + } + + public static final ReplicationBatchAdaptor REPLICATION_BATCH_ADAPTOR = new ReplicationBatchAdaptor(); + static final HLogKeyAdaptor H_LOG_KEY_ADAPTOR = new HLogKeyAdaptor(); + static final ClusterIdAdaptor CLUSTER_ID_ADAPTOR = new ClusterIdAdaptor(); + static final ColumnValueAdaptor COLUMN_VALUE_ADAPTOR = new ColumnValueAdaptor(); + static final WALEntryAdaptor WAL_ENTRY_ADAPTOR = new WALEntryAdaptor(); + + public static class ReplicationBatchAdaptor implements Adaptor { + + @Override public HLog.Entry[] fromThrift(TBatchEdit thriftBatch) { + HLog.Entry[] entries = new HLog.Entry[thriftBatch.getEditsSize()]; + int i = 0; + for (TEdit edit : thriftBatch.getEdits()) { + List uuids = CLUSTER_ID_ADAPTOR.fromThrift(edit.getClusterIds()); + entries[i] = new HLog.Entry( + H_LOG_KEY_ADAPTOR.fromThrift(new EntryWrapper(edit.getHLogKey(), uuids)), + WAL_ENTRY_ADAPTOR.fromThrift(new EntryWrapper(edit.getWalEdit(), uuids)) + ); + i++; + } + return entries; + } + + @Override public TBatchEdit toThrift(HLog.Entry[] hLogEntries) { + TBatchEdit batch = new TBatchEdit(); + + for (HLog.Entry hLogEntry : hLogEntries) { + List clusterIds = Lists.newArrayList(); + clusterIds.add(hLogEntry.getKey().getClusterId()); + clusterIds.addAll(hLogEntry.getEdit().getClusterIds()); + batch.addToEdits( + new TEdit( + H_LOG_KEY_ADAPTOR.toThrift(hLogEntry.getKey()).getEntryPart(), + WAL_ENTRY_ADAPTOR.toThrift(hLogEntry.getEdit()).getEntryPart(), + CLUSTER_ID_ADAPTOR.toThrift(clusterIds) + )); + } + return batch; + } + } + + // Maps HLogKey to Thrift: THLogKey + static class HLogKeyAdaptor implements Adaptor> { + + @Override public HLogKey fromThrift(EntryWrapper wrapper) { + return new HLogKey( + null, + wrapper.getEntryPart().getTableName(), + System.currentTimeMillis(), + wrapper.getEntryPart().getWriteTime(), + wrapper.getClusterIds().get(0) + ); + } + + + @Override public EntryWrapper toThrift(HLogKey hLogKey) { + THLogKey key = new THLogKey(); + key.setTableName(hLogKey.getTablename()); + key.setWriteTime(hLogKey.getWriteTime()); + return new EntryWrapper(key); + } + } + + // Maps WALEdit to Thrift: TWalEdit + static class WALEntryAdaptor implements Adaptor> { + + @Override public WALEdit fromThrift(EntryWrapper thriftEdit) { + WALEdit walEdit = new WALEdit(); + for (TColumnValue mutation : thriftEdit.getEntryPart().getMutations()) { + walEdit.add(COLUMN_VALUE_ADAPTOR.fromThrift(mutation)); + } + List clusterIds = thriftEdit.getClusterIds(); + if (clusterIds.size() > 1) { + walEdit.addClusterIds(clusterIds.subList(1, clusterIds.size())); + } + return walEdit; + } + + @Override public EntryWrapper toThrift(WALEdit walEdit) { + TWalLEdit thriftEdit = new TWalLEdit(); + thriftEdit.setMutations( + Lists.transform(walEdit.getKeyValues(), new Function() { + @Override public TColumnValue apply(KeyValue keyValue) { + return COLUMN_VALUE_ADAPTOR.toThrift(keyValue); + } + }) + ); + return new EntryWrapper(thriftEdit); + } + } + + // Maps Cluster's UUID to Thrift: TClusterId + static class ClusterIdAdaptor implements Adaptor, List> { + + @Override public List fromThrift(List clusterIds) { + return Lists.transform(clusterIds, new Function() { + @Override public UUID apply(TClusterId clusterId) { + return new UUID(clusterId.getUb(), clusterId.getLb()); + } + }); + } + + @Override public List toThrift(List uuids) { + return Lists.transform(uuids, new Function() { + @Override public TClusterId apply(UUID uuid) { + return new TClusterId(uuid.getLeastSignificantBits(), uuid.getMostSignificantBits()); + } + }); + } + } + // Maps KeyValue to Thrift: TColumnValue + static class ColumnValueAdaptor implements Adaptor { + + @Override public KeyValue fromThrift(TColumnValue tColumnValue) { + return new KeyValue( + tColumnValue.getRow(), + tColumnValue.getFamily(), + tColumnValue.getQualifier(), + tColumnValue.getTimestamp(), + ThriftEditType.codeToType(tColumnValue.getType()).getKvType(), + tColumnValue.getValue() + ); + } + + @Override public TColumnValue toThrift(KeyValue kv) { + TColumnValue col = new TColumnValue(); + col.setRow(kv.getRow()); + col.setFamily(kv.getFamily()); + col.setQualifier(kv.getQualifier()); + col.setValue(kv.getValue()); + col.setTimestamp(kv.getTimestamp()); + col.setType(ThriftEditType.keyValueToType(KeyValue.Type.codeToType(kv.getType())).getCode()); + return col; + } + } + + private static class EntryWrapper { + private final T EntryPart; + private final List clusterIds; + + private EntryWrapper(T entryPart) { + EntryPart = entryPart; + clusterIds = null; + } + + private EntryWrapper(T entryPart, List clusterIds) { + EntryPart = entryPart; + this.clusterIds = clusterIds; + } + + public T getEntryPart() { + return EntryPart; + } + + public List getClusterIds() { + return clusterIds; + } + } +} diff --git a/src/main/java/org/apache/hadoop/hbase/replication/thrift/ThriftClient.java b/src/main/java/org/apache/hadoop/hbase/replication/thrift/ThriftClient.java new file mode 100644 index 0000000..5588569 --- /dev/null +++ b/src/main/java/org/apache/hadoop/hbase/replication/thrift/ThriftClient.java @@ -0,0 +1,186 @@ +/** + * Copyright 2011 The Apache Software Foundation + * + * 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.hbase.replication.thrift; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.ServerName; +import org.apache.hadoop.hbase.regionserver.wal.HLog; +import org.apache.hadoop.hbase.replication.thrift.generated.*; +import org.apache.hadoop.hbase.security.User; +import org.apache.hadoop.security.SaslRpcServer; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.thrift.TException; +import org.apache.thrift.protocol.TBinaryProtocol; +import org.apache.thrift.protocol.TCompactProtocol; +import org.apache.thrift.protocol.TProtocol; +import org.apache.thrift.transport.TSaslClientTransport; +import org.apache.thrift.transport.TSocket; +import org.apache.thrift.transport.TTransport; +import org.apache.thrift.transport.TTransportException; + +import javax.security.sasl.Sasl; +import java.io.IOException; +import java.util.*; +import java.util.concurrent.ConcurrentSkipListMap; + +public class ThriftClient { + private static final Log LOG = LogFactory.getLog(ThriftClient.class); + private Configuration conf; + private final String peerId; + private boolean isSecure; + private ConcurrentSkipListMap clients = + new ConcurrentSkipListMap(); + + + public ThriftClient(Configuration conf, String peerId) throws IOException { + this.conf = conf; + this.peerId = peerId; + this.isSecure = User.isHBaseSecurityEnabled(conf); + } + + private THBaseService.Client createClient(String host, int port) throws IOException, + TTransportException { + boolean isCompact = + conf.getBoolean("hbase.replication.thrift.compact", true); + + String serverProtocol = UserGroupInformation.getCurrentUser().getUserName(); + String serverAddress = null; + if(User.isHBaseSecurityEnabled(conf)) { + String kerberosName = UserGroupInformation.getCurrentUser().getUserName(); + final String names[] = SaslRpcServer.splitKerberosName(kerberosName); + if (names.length != 3) { + throw new TTransportException("Kerberos principal should have 3 parts: " + kerberosName); + } + serverProtocol = names[0]; + serverAddress = names[1]; + } + + TTransport transport = new TSocket(host, port); + if(isSecure) { + Map saslProps = new HashMap(); + saslProps.put(Sasl.QOP, ThriftUtilities.getQOP(conf).getSaslQop()); + transport = new TUGIAssumingTransport( + new TSaslClientTransport( + ThriftUtilities.getAuthType(conf).getMechanismName(), + null, + serverProtocol, serverAddress, + saslProps, null, + transport), + User.getCurrent().getUGI()); + } + try { + transport.open(); + LOG.debug("Connected to "+host+":"+port); + } catch (TTransportException e) { + throw new IOException("Failed to open transport connection to : "+host+":"+port, e); + } + + TProtocol protocol; + if(isCompact) { + protocol = new TCompactProtocol(transport); + } else { + protocol = new TBinaryProtocol(transport); + } + + return new THBaseService.Client(protocol); + } + + public THBaseService.Client getClient(String host, int port) throws IOException, TTransportException { + String key = host+":"+port; + if(clients.containsKey(key)) { + return clients.get(key); + } + THBaseService.Client client = createClient(host, port); + clients.put(key, client); + return clients.get(key); + } + + public void removeClient(String host, int port) { + clients.remove(host+":"+port); + } + + public void ping(ServerName serverName) throws IOException{ + THBaseService.Client client = null; + try { + client = getClientFromServerName(serverName); + client.ping(); + } catch (TException e) { + removeClient(serverName.getHostname(), ThriftUtilities.getDestinationPeerPort(conf, peerId)); + try { + client.getOutputProtocol().getTransport().close(); + } catch(Exception e2) { + LOG.debug("Failed to gracefully close broken transport.", e2); + } + throw new IOException("Failed to ping replication client", e); + } + + } + + public void shipEdits(ServerName serverName, HLog.Entry[] entries) throws IOException { + THBaseService.Client client; + String host = serverName.getHostname(); + int port = ThriftUtilities.getDestinationPeerPort(conf, peerId); + try { + client = getClient(host, port); + } catch (TTransportException e) { + throw new IOException("Failed to create replication client", e); + } + try { + client.replicate(ThriftAdaptors.REPLICATION_BATCH_ADAPTOR.toThrift(entries)); + } catch (TTransportException e) { + removeClient(host, port); + try { + client.getOutputProtocol().getTransport().close(); + } catch(Exception e2) { + LOG.debug("Failed to gracefully close broken transport.", e2); + } + throw new IOException("Failed to ship edits", e); + } catch (TException e) { + throw new IOException("Failed to ship edits", e); + } + } + + public UUID getPeerClusterUUID(ServerName serverName) { + THBaseService.Client client; + String host = serverName.getHostname(); + int port = ThriftUtilities.getDestinationPeerPort(conf, peerId); + try { + client = getClient(host, port); + return UUID.fromString(client.getClusterUUID()); + } catch (Exception e) { + LOG.error("Error getting UUID from remote cluster", e); + return null; + } + } + + private THBaseService.Client getClientFromServerName(ServerName serverName) throws IOException { + THBaseService.Client client; + String host = serverName.getHostname(); + int port = ThriftUtilities.getDestinationPeerPort(conf, peerId); + try { + client = getClient(host, port); + } catch (TTransportException e) { + throw new IOException("Failed to create replication client", e); + } + return client; + } +} diff --git a/src/main/java/org/apache/hadoop/hbase/replication/thrift/ThriftEditType.java b/src/main/java/org/apache/hadoop/hbase/replication/thrift/ThriftEditType.java new file mode 100644 index 0000000..e1fcfd7 --- /dev/null +++ b/src/main/java/org/apache/hadoop/hbase/replication/thrift/ThriftEditType.java @@ -0,0 +1,58 @@ +package org.apache.hadoop.hbase.replication.thrift; + +import com.google.common.base.Function; +import com.google.common.collect.Maps; +import org.apache.hadoop.hbase.KeyValue; + +import java.util.Arrays; +import java.util.Map; + +public enum ThriftEditType { + PUT((byte)1, KeyValue.Type.Put), + DELCF((byte)2, KeyValue.Type.DeleteFamily), + DELCOLS((byte)3, KeyValue.Type.DeleteColumn), + DEL((byte)4, KeyValue.Type.Delete); + + private static Map toEditType = + Maps.uniqueIndex(Arrays.asList(ThriftEditType.values()), + new Function() { + @Override public Byte apply(ThriftEditType editType) { + return editType.getCode(); + } + } + ); + + private static Map keyValueToThrift = + Maps.uniqueIndex(Arrays.asList(ThriftEditType.values()), + new Function() { + @Override public KeyValue.Type apply(ThriftEditType editType) { + return editType.getKvType(); + } + } + ); + + private byte val; + private KeyValue.Type kvType; + + ThriftEditType(byte i, KeyValue.Type kvType) { + val = i; + this.kvType = kvType; + } + + public byte getCode() { + return val; + } + + public KeyValue.Type getKvType() { + return kvType; + } + + public static ThriftEditType codeToType(byte someByte) { + return toEditType.get(someByte); + } + + public static ThriftEditType keyValueToType(KeyValue.Type keyValueType) { + return keyValueToThrift.get(keyValueType); + } + +} diff --git a/src/main/java/org/apache/hadoop/hbase/replication/thrift/ThriftHBaseServiceHandler.java b/src/main/java/org/apache/hadoop/hbase/replication/thrift/ThriftHBaseServiceHandler.java new file mode 100644 index 0000000..5e5f6e3 --- /dev/null +++ b/src/main/java/org/apache/hadoop/hbase/replication/thrift/ThriftHBaseServiceHandler.java @@ -0,0 +1,103 @@ +/** + * Copyright 2011 The Apache Software Foundation + * + * 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.hbase.replication.thrift; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.replication.thrift.generated.TBatchEdit; +import org.apache.hadoop.hbase.replication.thrift.generated.THBaseService; +import org.apache.hadoop.hbase.replication.thrift.generated.TIOError; +import org.apache.hadoop.hbase.thrift.ThriftMetrics; +import org.apache.thrift.TException; + +import java.io.IOException; +import java.lang.reflect.InvocationHandler; +import java.lang.reflect.InvocationTargetException; +import java.lang.reflect.Method; +import java.lang.reflect.Proxy; + +public class ThriftHBaseServiceHandler implements THBaseService.Iface { + + private final THBaseService.Iface serviceHandler; + private static final Log LOG = LogFactory.getLog(ThriftHBaseServiceHandler.class); + + + public static THBaseService.Iface newInstance(Configuration conf, + THBaseService.Iface serviceHandler, + ThriftMetrics metrics) { + THBaseService.Iface handler = new ThriftHBaseServiceHandler(conf, serviceHandler); + return (THBaseService.Iface) Proxy.newProxyInstance(handler.getClass().getClassLoader(), + new Class[] { THBaseService.Iface.class }, new THBaseServiceMetricsProxy(handler, metrics)); + } + + private static class THBaseServiceMetricsProxy implements InvocationHandler { + private final THBaseService.Iface handler; + private final ThriftMetrics metrics; + + private THBaseServiceMetricsProxy(THBaseService.Iface handler, ThriftMetrics metrics) { + this.handler = handler; + this.metrics = metrics; + } + + @Override + public Object invoke(Object proxy, Method m, Object[] args) throws Throwable { + Object result; + try { + long start = now(); + result = m.invoke(handler, args); + int processTime = (int) (now() - start); + metrics.incMethodTime(m.getName(), processTime); + } catch (InvocationTargetException e) { + throw e.getTargetException(); + } catch (Exception e) { + throw new RuntimeException("unexpected invocation exception: " + e.getMessage()); + } + return result; + } + } + + private static long now() { + return System.nanoTime(); + } + + ThriftHBaseServiceHandler(Configuration conf, THBaseService.Iface serviceHandler) { + this.serviceHandler = serviceHandler; + } + + private TIOError getTIOError(IOException e) { + TIOError err = new TIOError(); + err.setMessage(e.getMessage()); + return err; + } + + @Override + public void replicate(TBatchEdit edits) throws TException { + serviceHandler.replicate(edits); + } + + @Override public void ping() throws TException { + serviceHandler.ping(); + } + + @Override public String getClusterUUID() throws TException { + return serviceHandler.getClusterUUID(); + } +} diff --git a/src/main/java/org/apache/hadoop/hbase/replication/thrift/ThriftServer.java b/src/main/java/org/apache/hadoop/hbase/replication/thrift/ThriftServer.java new file mode 100644 index 0000000..9da337e --- /dev/null +++ b/src/main/java/org/apache/hadoop/hbase/replication/thrift/ThriftServer.java @@ -0,0 +1,184 @@ +/** + * Copyright 2011 The Apache Software Foundation + * + * 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.hbase.replication.thrift; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.ServerName; +import org.apache.hadoop.hbase.security.User; +import org.apache.hadoop.hbase.thrift.ThriftMetrics; +import org.apache.hadoop.hbase.replication.thrift.generated.THBaseService; +import org.apache.hadoop.security.SaslRpcServer; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.thrift.protocol.TBinaryProtocol; +import org.apache.thrift.protocol.TCompactProtocol; +import org.apache.thrift.protocol.TProtocolFactory; +import org.apache.thrift.server.TServer; +import org.apache.thrift.server.TThreadPoolServer; +import org.apache.thrift.transport.TFramedTransport; +import org.apache.thrift.transport.TSaslServerTransport; +import org.apache.thrift.transport.TServerSocket; +import org.apache.thrift.transport.TServerTransport; +import org.apache.thrift.transport.TTransportException; +import org.apache.thrift.transport.TTransportFactory; + +import javax.security.sasl.Sasl; +import java.io.IOException; +import java.net.InetAddress; +import java.net.InetSocketAddress; +import java.net.UnknownHostException; +import java.util.HashMap; +import java.util.Map; + +public class ThriftServer extends Thread { + private static final Log LOG = LogFactory.getLog(ThriftServer.class); + private TServer server; + private Configuration conf; + private THBaseService.Iface sinkInterface; + private ServerName serverName; + private boolean useSecure; + + public ThriftServer(Configuration conf, THBaseService.Iface sinkInterface, + ServerName serverName) throws IOException { + this.conf = conf; + this.sinkInterface = sinkInterface; + this.serverName = serverName; + this.useSecure = User.isHBaseSecurityEnabled(conf); + try { + init(); + } catch (TTransportException e) { + throw new IOException(e); + } + } + + private static TProtocolFactory getTProtocolFactory(boolean isCompact) { + if (isCompact) { + LOG.debug("Using compact protocol"); + return new TCompactProtocol.Factory(); + } else { + LOG.debug("Using binary protocol"); + return new TBinaryProtocol.Factory(); + } + } + + private static TTransportFactory getTTransportFactory(boolean framed) { + if (framed) { + LOG.debug("Using framed transport"); + return new TFramedTransport.Factory(); + } else { + return new TTransportFactory(); + } + } + + /* + * If bindValue is null, we don't bind. + */ + private static InetSocketAddress bindToPort(String bindValue, int listenPort) + throws UnknownHostException { + try { + if (bindValue == null) { + return new InetSocketAddress(listenPort); + } else { + return new InetSocketAddress(InetAddress.getByName(bindValue), listenPort); + } + } catch (UnknownHostException e) { + throw new RuntimeException("Could not bind to provided ip address", e); + } + } + + private static TServer getTThreadPoolServer(TProtocolFactory protocolFactory, THBaseService.Processor processor, + TTransportFactory transportFactory, InetSocketAddress inetSocketAddress) throws TTransportException { + TServerTransport serverTransport = new TServerSocket(inetSocketAddress); + LOG.info("starting HBase ThreadPool Thrift server on " + inetSocketAddress.toString()); + TThreadPoolServer.Args serverArgs = new TThreadPoolServer.Args(serverTransport); + serverArgs.maxWorkerThreads(100); + serverArgs.processor(processor); + serverArgs.transportFactory(transportFactory); + serverArgs.protocolFactory(protocolFactory); + return new TThreadPoolServer(serverArgs); + } + + public void init() throws TTransportException, IOException { + int listenPort = ThriftUtilities.getThriftServerPort(conf); + + ThriftMetrics metrics = new ThriftMetrics( + listenPort, conf, THBaseService.Iface.class); + + boolean isCompact = + conf.getBoolean("hbase.replication.thrift.compact", true); + + // Construct correct ProtocolFactory + TProtocolFactory protocolFactory = getTProtocolFactory(isCompact); + THBaseService.Iface handler = + ThriftHBaseServiceHandler.newInstance(conf, sinkInterface, metrics); + THBaseService.Processor processor = new THBaseService.Processor(handler); + + + boolean isFramed = + conf.getBoolean("hbase.replication.thrift.framed", false); + TTransportFactory transportFactory = getTTransportFactory(isFramed); + + + String serverProtocol = UserGroupInformation.getCurrentUser().getUserName(); + String serverAddress = null; + if(User.isHBaseSecurityEnabled(conf)) { + String kerberosName = UserGroupInformation.getCurrentUser().getUserName(); + final String names[] = SaslRpcServer.splitKerberosName(kerberosName); + if (names.length != 3) { + throw new TTransportException("Kerberos principal should have 3 parts: " + kerberosName); + } + serverProtocol = names[0]; + serverAddress = names[1]; + } + + if(useSecure) { + Map saslProps = new HashMap(); + saslProps.put(Sasl.QOP, ThriftUtilities.getQOP(conf).getSaslQop()); + TSaslServerTransport.Factory saslTransFactory = new TSaslServerTransport.Factory(); + saslTransFactory.addServerDefinition( + ThriftUtilities.getAuthType(conf).getMechanismName(), + serverProtocol, serverAddress, // two parts of kerberos principal + saslProps, + new SaslRpcServer.SaslGssCallbackHandler()); + + transportFactory = new ChainedTTransportFactory(transportFactory, + new TUGIAssumingTransportFactory(saslTransFactory, + UserGroupInformation.getCurrentUser())); + } + + InetSocketAddress inetSocketAddress = + bindToPort(conf.get("hbase.replication.thrift.address"), listenPort); + LOG.info("Listening on "+inetSocketAddress.getHostName()+":"+inetSocketAddress.getPort()); + server = getTThreadPoolServer(protocolFactory, processor, transportFactory, inetSocketAddress); + } + + public void run() { + server.serve(); + } + + public void shutdown() { + try { + server.stop(); + } catch (Throwable e) { + LOG.error("Error stopping thrift server", e); + } + } +} diff --git a/src/main/java/org/apache/hadoop/hbase/replication/thrift/ThriftUtilities.java b/src/main/java/org/apache/hadoop/hbase/replication/thrift/ThriftUtilities.java new file mode 100644 index 0000000..1b39680 --- /dev/null +++ b/src/main/java/org/apache/hadoop/hbase/replication/thrift/ThriftUtilities.java @@ -0,0 +1,73 @@ +/** + * Copyright 2011 The Apache Software Foundation + * + * 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.hbase.replication.thrift; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.security.HBaseSaslRpcServer; +import org.apache.hadoop.hbase.security.HBaseSaslRpcServer.QualityOfProtection; +import org.apache.hadoop.hbase.security.User; + + +public class ThriftUtilities { + + private ThriftUtilities() { + throw new UnsupportedOperationException("Can't initialize class"); + } + + public static QualityOfProtection getQOP(Configuration conf) { + QualityOfProtection saslQOP = QualityOfProtection.AUTHENTICATION; + String rpcProtection = conf.get("hbase.replication.thrift.protection", + QualityOfProtection.AUTHENTICATION.name().toLowerCase()); + if (QualityOfProtection.INTEGRITY.name().toLowerCase() + .equals(rpcProtection)) { + saslQOP = QualityOfProtection.INTEGRITY; + } else if (QualityOfProtection.PRIVACY.name().toLowerCase().equals( + rpcProtection)) { + saslQOP = QualityOfProtection.PRIVACY; + } + return saslQOP; + } + + public static HBaseSaslRpcServer.AuthMethod getAuthType(Configuration conf) { + return HBaseSaslRpcServer.AuthMethod.valueOf(conf.get(User.HBASE_SECURITY_CONF_KEY, "SIMPLE")); + } + + public static int getThriftServerPort(Configuration conf) { + return conf.getInt("hbase.replication.thrift.server.port", -1); + } + + /* + * hbase.replication.thrift.peer..port is used for testing purposes only + * in your production cluster, which would be running on different instances, you do + * not need to set this field, simply provide the hbase.replication.thrift.server.port + * value and keep them the same across clusters. + */ + public static int getDestinationPeerPort(Configuration conf, String peerId) { + int port = conf.getInt("hbase.replication.thrift.peer." + peerId + ".port", -1); + // if there is no custom peer port set use the default server port. + if (port == -1) { + port = getThriftServerPort(conf); + } + return port; + } + +} + + diff --git a/src/main/java/org/apache/hadoop/hbase/replication/thrift/generated/TBatchEdit.java b/src/main/java/org/apache/hadoop/hbase/replication/thrift/generated/TBatchEdit.java new file mode 100644 index 0000000..815fd18 --- /dev/null +++ b/src/main/java/org/apache/hadoop/hbase/replication/thrift/generated/TBatchEdit.java @@ -0,0 +1,435 @@ +/** + * Autogenerated by Thrift Compiler (0.9.1) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hadoop.hbase.replication.thrift.generated; + +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 org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +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 TBatchEdit implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TBatchEdit"); + + private static final org.apache.thrift.protocol.TField EDITS_FIELD_DESC = new org.apache.thrift.protocol.TField("edits", org.apache.thrift.protocol.TType.LIST, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TBatchEditStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TBatchEditTupleSchemeFactory()); + } + + public List edits; // 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 { + EDITS((short)1, "edits"); + + 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: // EDITS + return EDITS; + 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.EDITS, new org.apache.thrift.meta_data.FieldMetaData("edits", 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, TEdit.class)))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TBatchEdit.class, metaDataMap); + } + + public TBatchEdit() { + } + + public TBatchEdit( + List edits) + { + this(); + this.edits = edits; + } + + /** + * Performs a deep copy on other. + */ + public TBatchEdit(TBatchEdit other) { + if (other.isSetEdits()) { + List __this__edits = new ArrayList(other.edits.size()); + for (TEdit other_element : other.edits) { + __this__edits.add(new TEdit(other_element)); + } + this.edits = __this__edits; + } + } + + public TBatchEdit deepCopy() { + return new TBatchEdit(this); + } + + @Override + public void clear() { + this.edits = null; + } + + public int getEditsSize() { + return (this.edits == null) ? 0 : this.edits.size(); + } + + public java.util.Iterator getEditsIterator() { + return (this.edits == null) ? null : this.edits.iterator(); + } + + public void addToEdits(TEdit elem) { + if (this.edits == null) { + this.edits = new ArrayList(); + } + this.edits.add(elem); + } + + public List getEdits() { + return this.edits; + } + + public TBatchEdit setEdits(List edits) { + this.edits = edits; + return this; + } + + public void unsetEdits() { + this.edits = null; + } + + /** Returns true if field edits is set (has been assigned a value) and false otherwise */ + public boolean isSetEdits() { + return this.edits != null; + } + + public void setEditsIsSet(boolean value) { + if (!value) { + this.edits = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case EDITS: + if (value == null) { + unsetEdits(); + } else { + setEdits((List)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case EDITS: + return getEdits(); + + } + 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 EDITS: + return isSetEdits(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TBatchEdit) + return this.equals((TBatchEdit)that); + return false; + } + + public boolean equals(TBatchEdit that) { + if (that == null) + return false; + + boolean this_present_edits = true && this.isSetEdits(); + boolean that_present_edits = true && that.isSetEdits(); + if (this_present_edits || that_present_edits) { + if (!(this_present_edits && that_present_edits)) + return false; + if (!this.edits.equals(that.edits)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + return 0; + } + + @Override + public int compareTo(TBatchEdit other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(isSetEdits()).compareTo(other.isSetEdits()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetEdits()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.edits, other.edits); + 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("TBatchEdit("); + boolean first = true; + + sb.append("edits:"); + if (this.edits == null) { + sb.append("null"); + } else { + sb.append(this.edits); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (edits == null) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'edits' was not present! 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 TBatchEditStandardSchemeFactory implements SchemeFactory { + public TBatchEditStandardScheme getScheme() { + return new TBatchEditStandardScheme(); + } + } + + private static class TBatchEditStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TBatchEdit 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: // EDITS + if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { + { + org.apache.thrift.protocol.TList _list16 = iprot.readListBegin(); + struct.edits = new ArrayList(_list16.size); + for (int _i17 = 0; _i17 < _list16.size; ++_i17) + { + TEdit _elem18; + _elem18 = new TEdit(); + _elem18.read(iprot); + struct.edits.add(_elem18); + } + iprot.readListEnd(); + } + struct.setEditsIsSet(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(); + + // check for required fields of primitive type, which can't be checked in the validate method + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TBatchEdit struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.edits != null) { + oprot.writeFieldBegin(EDITS_FIELD_DESC); + { + oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.edits.size())); + for (TEdit _iter19 : struct.edits) + { + _iter19.write(oprot); + } + oprot.writeListEnd(); + } + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TBatchEditTupleSchemeFactory implements SchemeFactory { + public TBatchEditTupleScheme getScheme() { + return new TBatchEditTupleScheme(); + } + } + + private static class TBatchEditTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TBatchEdit struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + { + oprot.writeI32(struct.edits.size()); + for (TEdit _iter20 : struct.edits) + { + _iter20.write(oprot); + } + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TBatchEdit struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + { + org.apache.thrift.protocol.TList _list21 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); + struct.edits = new ArrayList(_list21.size); + for (int _i22 = 0; _i22 < _list21.size; ++_i22) + { + TEdit _elem23; + _elem23 = new TEdit(); + _elem23.read(iprot); + struct.edits.add(_elem23); + } + } + struct.setEditsIsSet(true); + } + } + +} + diff --git a/src/main/java/org/apache/hadoop/hbase/replication/thrift/generated/TClusterId.java b/src/main/java/org/apache/hadoop/hbase/replication/thrift/generated/TClusterId.java new file mode 100644 index 0000000..61cf48f --- /dev/null +++ b/src/main/java/org/apache/hadoop/hbase/replication/thrift/generated/TClusterId.java @@ -0,0 +1,480 @@ +/** + * Autogenerated by Thrift Compiler (0.9.1) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hadoop.hbase.replication.thrift.generated; + +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 org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +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 TClusterId implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TClusterId"); + + private static final org.apache.thrift.protocol.TField LB_FIELD_DESC = new org.apache.thrift.protocol.TField("lb", org.apache.thrift.protocol.TType.I64, (short)1); + private static final org.apache.thrift.protocol.TField UB_FIELD_DESC = new org.apache.thrift.protocol.TField("ub", org.apache.thrift.protocol.TType.I64, (short)2); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TClusterIdStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TClusterIdTupleSchemeFactory()); + } + + public long lb; // required + public long ub; // 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 { + LB((short)1, "lb"), + UB((short)2, "ub"); + + 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: // LB + return LB; + case 2: // UB + return UB; + 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 __LB_ISSET_ID = 0; + private static final int __UB_ISSET_ID = 1; + 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.LB, new org.apache.thrift.meta_data.FieldMetaData("lb", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64))); + tmpMap.put(_Fields.UB, new org.apache.thrift.meta_data.FieldMetaData("ub", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TClusterId.class, metaDataMap); + } + + public TClusterId() { + } + + public TClusterId( + long lb, + long ub) + { + this(); + this.lb = lb; + setLbIsSet(true); + this.ub = ub; + setUbIsSet(true); + } + + /** + * Performs a deep copy on other. + */ + public TClusterId(TClusterId other) { + __isset_bitfield = other.__isset_bitfield; + this.lb = other.lb; + this.ub = other.ub; + } + + public TClusterId deepCopy() { + return new TClusterId(this); + } + + @Override + public void clear() { + setLbIsSet(false); + this.lb = 0; + setUbIsSet(false); + this.ub = 0; + } + + public long getLb() { + return this.lb; + } + + public TClusterId setLb(long lb) { + this.lb = lb; + setLbIsSet(true); + return this; + } + + public void unsetLb() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __LB_ISSET_ID); + } + + /** Returns true if field lb is set (has been assigned a value) and false otherwise */ + public boolean isSetLb() { + return EncodingUtils.testBit(__isset_bitfield, __LB_ISSET_ID); + } + + public void setLbIsSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __LB_ISSET_ID, value); + } + + public long getUb() { + return this.ub; + } + + public TClusterId setUb(long ub) { + this.ub = ub; + setUbIsSet(true); + return this; + } + + public void unsetUb() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __UB_ISSET_ID); + } + + /** Returns true if field ub is set (has been assigned a value) and false otherwise */ + public boolean isSetUb() { + return EncodingUtils.testBit(__isset_bitfield, __UB_ISSET_ID); + } + + public void setUbIsSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __UB_ISSET_ID, value); + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case LB: + if (value == null) { + unsetLb(); + } else { + setLb((Long)value); + } + break; + + case UB: + if (value == null) { + unsetUb(); + } else { + setUb((Long)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case LB: + return Long.valueOf(getLb()); + + case UB: + return Long.valueOf(getUb()); + + } + 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 LB: + return isSetLb(); + case UB: + return isSetUb(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TClusterId) + return this.equals((TClusterId)that); + return false; + } + + public boolean equals(TClusterId that) { + if (that == null) + return false; + + boolean this_present_lb = true; + boolean that_present_lb = true; + if (this_present_lb || that_present_lb) { + if (!(this_present_lb && that_present_lb)) + return false; + if (this.lb != that.lb) + return false; + } + + boolean this_present_ub = true; + boolean that_present_ub = true; + if (this_present_ub || that_present_ub) { + if (!(this_present_ub && that_present_ub)) + return false; + if (this.ub != that.ub) + return false; + } + + return true; + } + + @Override + public int hashCode() { + return 0; + } + + @Override + public int compareTo(TClusterId other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(isSetLb()).compareTo(other.isSetLb()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetLb()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.lb, other.lb); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetUb()).compareTo(other.isSetUb()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetUb()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.ub, other.ub); + 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("TClusterId("); + boolean first = true; + + sb.append("lb:"); + sb.append(this.lb); + first = false; + if (!first) sb.append(", "); + sb.append("ub:"); + sb.append(this.ub); + 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 TClusterIdStandardSchemeFactory implements SchemeFactory { + public TClusterIdStandardScheme getScheme() { + return new TClusterIdStandardScheme(); + } + } + + private static class TClusterIdStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TClusterId 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: // LB + if (schemeField.type == org.apache.thrift.protocol.TType.I64) { + struct.lb = iprot.readI64(); + struct.setLbIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // UB + if (schemeField.type == org.apache.thrift.protocol.TType.I64) { + struct.ub = iprot.readI64(); + struct.setUbIsSet(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(); + + // check for required fields of primitive type, which can't be checked in the validate method + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TClusterId struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + oprot.writeFieldBegin(LB_FIELD_DESC); + oprot.writeI64(struct.lb); + oprot.writeFieldEnd(); + oprot.writeFieldBegin(UB_FIELD_DESC); + oprot.writeI64(struct.ub); + oprot.writeFieldEnd(); + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TClusterIdTupleSchemeFactory implements SchemeFactory { + public TClusterIdTupleScheme getScheme() { + return new TClusterIdTupleScheme(); + } + } + + private static class TClusterIdTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TClusterId struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetLb()) { + optionals.set(0); + } + if (struct.isSetUb()) { + optionals.set(1); + } + oprot.writeBitSet(optionals, 2); + if (struct.isSetLb()) { + oprot.writeI64(struct.lb); + } + if (struct.isSetUb()) { + oprot.writeI64(struct.ub); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TClusterId struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(2); + if (incoming.get(0)) { + struct.lb = iprot.readI64(); + struct.setLbIsSet(true); + } + if (incoming.get(1)) { + struct.ub = iprot.readI64(); + struct.setUbIsSet(true); + } + } + } + +} + diff --git a/src/main/java/org/apache/hadoop/hbase/replication/thrift/generated/TColumnValue.java b/src/main/java/org/apache/hadoop/hbase/replication/thrift/generated/TColumnValue.java new file mode 100644 index 0000000..0dfdd26 --- /dev/null +++ b/src/main/java/org/apache/hadoop/hbase/replication/thrift/generated/TColumnValue.java @@ -0,0 +1,910 @@ +/** + * Autogenerated by Thrift Compiler (0.9.1) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hadoop.hbase.replication.thrift.generated; + +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 org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +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; + +/** + * Represents a single cell and its value. + */ +public class TColumnValue implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TColumnValue"); + + private static final org.apache.thrift.protocol.TField ROW_FIELD_DESC = new org.apache.thrift.protocol.TField("row", org.apache.thrift.protocol.TType.STRING, (short)1); + private static final org.apache.thrift.protocol.TField FAMILY_FIELD_DESC = new org.apache.thrift.protocol.TField("family", org.apache.thrift.protocol.TType.STRING, (short)2); + private static final org.apache.thrift.protocol.TField QUALIFIER_FIELD_DESC = new org.apache.thrift.protocol.TField("qualifier", org.apache.thrift.protocol.TType.STRING, (short)3); + private static final org.apache.thrift.protocol.TField VALUE_FIELD_DESC = new org.apache.thrift.protocol.TField("value", org.apache.thrift.protocol.TType.STRING, (short)4); + private static final org.apache.thrift.protocol.TField TIMESTAMP_FIELD_DESC = new org.apache.thrift.protocol.TField("timestamp", org.apache.thrift.protocol.TType.I64, (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.BYTE, (short)6); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TColumnValueStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TColumnValueTupleSchemeFactory()); + } + + public ByteBuffer row; // required + public ByteBuffer family; // required + public ByteBuffer qualifier; // required + public ByteBuffer value; // required + public long timestamp; // optional + public byte type; // 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 { + ROW((short)1, "row"), + FAMILY((short)2, "family"), + QUALIFIER((short)3, "qualifier"), + VALUE((short)4, "value"), + TIMESTAMP((short)5, "timestamp"), + TYPE((short)6, "type"); + + 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: // ROW + return ROW; + case 2: // FAMILY + return FAMILY; + case 3: // QUALIFIER + return QUALIFIER; + case 4: // VALUE + return VALUE; + case 5: // TIMESTAMP + return TIMESTAMP; + case 6: // TYPE + return TYPE; + 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 __TIMESTAMP_ISSET_ID = 0; + private static final int __TYPE_ISSET_ID = 1; + private byte __isset_bitfield = 0; + private _Fields optionals[] = {_Fields.TIMESTAMP}; + 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.ROW, new org.apache.thrift.meta_data.FieldMetaData("row", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , true))); + tmpMap.put(_Fields.FAMILY, new org.apache.thrift.meta_data.FieldMetaData("family", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , true))); + tmpMap.put(_Fields.QUALIFIER, new org.apache.thrift.meta_data.FieldMetaData("qualifier", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , true))); + tmpMap.put(_Fields.VALUE, new org.apache.thrift.meta_data.FieldMetaData("value", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , true))); + tmpMap.put(_Fields.TIMESTAMP, new org.apache.thrift.meta_data.FieldMetaData("timestamp", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64))); + tmpMap.put(_Fields.TYPE, new org.apache.thrift.meta_data.FieldMetaData("type", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BYTE))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TColumnValue.class, metaDataMap); + } + + public TColumnValue() { + } + + public TColumnValue( + ByteBuffer row, + ByteBuffer family, + ByteBuffer qualifier, + ByteBuffer value, + byte type) + { + this(); + this.row = row; + this.family = family; + this.qualifier = qualifier; + this.value = value; + this.type = type; + setTypeIsSet(true); + } + + /** + * Performs a deep copy on other. + */ + public TColumnValue(TColumnValue other) { + __isset_bitfield = other.__isset_bitfield; + if (other.isSetRow()) { + this.row = org.apache.thrift.TBaseHelper.copyBinary(other.row); +; + } + if (other.isSetFamily()) { + this.family = org.apache.thrift.TBaseHelper.copyBinary(other.family); +; + } + if (other.isSetQualifier()) { + this.qualifier = org.apache.thrift.TBaseHelper.copyBinary(other.qualifier); +; + } + if (other.isSetValue()) { + this.value = org.apache.thrift.TBaseHelper.copyBinary(other.value); +; + } + this.timestamp = other.timestamp; + this.type = other.type; + } + + public TColumnValue deepCopy() { + return new TColumnValue(this); + } + + @Override + public void clear() { + this.row = null; + this.family = null; + this.qualifier = null; + this.value = null; + setTimestampIsSet(false); + this.timestamp = 0; + setTypeIsSet(false); + this.type = 0; + } + + public byte[] getRow() { + setRow(org.apache.thrift.TBaseHelper.rightSize(row)); + return row == null ? null : row.array(); + } + + public ByteBuffer bufferForRow() { + return row; + } + + public TColumnValue setRow(byte[] row) { + setRow(row == null ? (ByteBuffer)null : ByteBuffer.wrap(row)); + return this; + } + + public TColumnValue setRow(ByteBuffer row) { + this.row = row; + return this; + } + + public void unsetRow() { + this.row = null; + } + + /** Returns true if field row is set (has been assigned a value) and false otherwise */ + public boolean isSetRow() { + return this.row != null; + } + + public void setRowIsSet(boolean value) { + if (!value) { + this.row = null; + } + } + + public byte[] getFamily() { + setFamily(org.apache.thrift.TBaseHelper.rightSize(family)); + return family == null ? null : family.array(); + } + + public ByteBuffer bufferForFamily() { + return family; + } + + public TColumnValue setFamily(byte[] family) { + setFamily(family == null ? (ByteBuffer)null : ByteBuffer.wrap(family)); + return this; + } + + public TColumnValue setFamily(ByteBuffer family) { + this.family = family; + return this; + } + + public void unsetFamily() { + this.family = null; + } + + /** Returns true if field family is set (has been assigned a value) and false otherwise */ + public boolean isSetFamily() { + return this.family != null; + } + + public void setFamilyIsSet(boolean value) { + if (!value) { + this.family = null; + } + } + + public byte[] getQualifier() { + setQualifier(org.apache.thrift.TBaseHelper.rightSize(qualifier)); + return qualifier == null ? null : qualifier.array(); + } + + public ByteBuffer bufferForQualifier() { + return qualifier; + } + + public TColumnValue setQualifier(byte[] qualifier) { + setQualifier(qualifier == null ? (ByteBuffer)null : ByteBuffer.wrap(qualifier)); + return this; + } + + public TColumnValue setQualifier(ByteBuffer qualifier) { + this.qualifier = qualifier; + return this; + } + + public void unsetQualifier() { + this.qualifier = null; + } + + /** Returns true if field qualifier is set (has been assigned a value) and false otherwise */ + public boolean isSetQualifier() { + return this.qualifier != null; + } + + public void setQualifierIsSet(boolean value) { + if (!value) { + this.qualifier = null; + } + } + + public byte[] getValue() { + setValue(org.apache.thrift.TBaseHelper.rightSize(value)); + return value == null ? null : value.array(); + } + + public ByteBuffer bufferForValue() { + return value; + } + + public TColumnValue setValue(byte[] value) { + setValue(value == null ? (ByteBuffer)null : ByteBuffer.wrap(value)); + return this; + } + + public TColumnValue setValue(ByteBuffer value) { + this.value = value; + return this; + } + + public void unsetValue() { + this.value = null; + } + + /** Returns true if field value is set (has been assigned a value) and false otherwise */ + public boolean isSetValue() { + return this.value != null; + } + + public void setValueIsSet(boolean value) { + if (!value) { + this.value = null; + } + } + + public long getTimestamp() { + return this.timestamp; + } + + public TColumnValue setTimestamp(long timestamp) { + this.timestamp = timestamp; + setTimestampIsSet(true); + return this; + } + + public void unsetTimestamp() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __TIMESTAMP_ISSET_ID); + } + + /** Returns true if field timestamp is set (has been assigned a value) and false otherwise */ + public boolean isSetTimestamp() { + return EncodingUtils.testBit(__isset_bitfield, __TIMESTAMP_ISSET_ID); + } + + public void setTimestampIsSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __TIMESTAMP_ISSET_ID, value); + } + + public byte getType() { + return this.type; + } + + public TColumnValue setType(byte type) { + this.type = type; + setTypeIsSet(true); + return this; + } + + public void unsetType() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __TYPE_ISSET_ID); + } + + /** Returns true if field type is set (has been assigned a value) and false otherwise */ + public boolean isSetType() { + return EncodingUtils.testBit(__isset_bitfield, __TYPE_ISSET_ID); + } + + public void setTypeIsSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __TYPE_ISSET_ID, value); + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case ROW: + if (value == null) { + unsetRow(); + } else { + setRow((ByteBuffer)value); + } + break; + + case FAMILY: + if (value == null) { + unsetFamily(); + } else { + setFamily((ByteBuffer)value); + } + break; + + case QUALIFIER: + if (value == null) { + unsetQualifier(); + } else { + setQualifier((ByteBuffer)value); + } + break; + + case VALUE: + if (value == null) { + unsetValue(); + } else { + setValue((ByteBuffer)value); + } + break; + + case TIMESTAMP: + if (value == null) { + unsetTimestamp(); + } else { + setTimestamp((Long)value); + } + break; + + case TYPE: + if (value == null) { + unsetType(); + } else { + setType((Byte)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case ROW: + return getRow(); + + case FAMILY: + return getFamily(); + + case QUALIFIER: + return getQualifier(); + + case VALUE: + return getValue(); + + case TIMESTAMP: + return Long.valueOf(getTimestamp()); + + case TYPE: + return Byte.valueOf(getType()); + + } + 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 ROW: + return isSetRow(); + case FAMILY: + return isSetFamily(); + case QUALIFIER: + return isSetQualifier(); + case VALUE: + return isSetValue(); + case TIMESTAMP: + return isSetTimestamp(); + case TYPE: + return isSetType(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TColumnValue) + return this.equals((TColumnValue)that); + return false; + } + + public boolean equals(TColumnValue that) { + if (that == null) + return false; + + boolean this_present_row = true && this.isSetRow(); + boolean that_present_row = true && that.isSetRow(); + if (this_present_row || that_present_row) { + if (!(this_present_row && that_present_row)) + return false; + if (!this.row.equals(that.row)) + return false; + } + + boolean this_present_family = true && this.isSetFamily(); + boolean that_present_family = true && that.isSetFamily(); + if (this_present_family || that_present_family) { + if (!(this_present_family && that_present_family)) + return false; + if (!this.family.equals(that.family)) + return false; + } + + boolean this_present_qualifier = true && this.isSetQualifier(); + boolean that_present_qualifier = true && that.isSetQualifier(); + if (this_present_qualifier || that_present_qualifier) { + if (!(this_present_qualifier && that_present_qualifier)) + return false; + if (!this.qualifier.equals(that.qualifier)) + return false; + } + + boolean this_present_value = true && this.isSetValue(); + boolean that_present_value = true && that.isSetValue(); + if (this_present_value || that_present_value) { + if (!(this_present_value && that_present_value)) + return false; + if (!this.value.equals(that.value)) + return false; + } + + boolean this_present_timestamp = true && this.isSetTimestamp(); + boolean that_present_timestamp = true && that.isSetTimestamp(); + if (this_present_timestamp || that_present_timestamp) { + if (!(this_present_timestamp && that_present_timestamp)) + return false; + if (this.timestamp != that.timestamp) + return false; + } + + boolean this_present_type = true; + boolean that_present_type = true; + if (this_present_type || that_present_type) { + if (!(this_present_type && that_present_type)) + return false; + if (this.type != that.type) + return false; + } + + return true; + } + + @Override + public int hashCode() { + return 0; + } + + @Override + public int compareTo(TColumnValue other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(isSetRow()).compareTo(other.isSetRow()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetRow()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.row, other.row); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetFamily()).compareTo(other.isSetFamily()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetFamily()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.family, other.family); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetQualifier()).compareTo(other.isSetQualifier()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetQualifier()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.qualifier, other.qualifier); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetValue()).compareTo(other.isSetValue()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetValue()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.value, other.value); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetTimestamp()).compareTo(other.isSetTimestamp()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetTimestamp()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.timestamp, other.timestamp); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetType()).compareTo(other.isSetType()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetType()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.type, other.type); + 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("TColumnValue("); + boolean first = true; + + sb.append("row:"); + if (this.row == null) { + sb.append("null"); + } else { + org.apache.thrift.TBaseHelper.toString(this.row, sb); + } + first = false; + if (!first) sb.append(", "); + sb.append("family:"); + if (this.family == null) { + sb.append("null"); + } else { + org.apache.thrift.TBaseHelper.toString(this.family, sb); + } + first = false; + if (!first) sb.append(", "); + sb.append("qualifier:"); + if (this.qualifier == null) { + sb.append("null"); + } else { + org.apache.thrift.TBaseHelper.toString(this.qualifier, sb); + } + first = false; + if (!first) sb.append(", "); + sb.append("value:"); + if (this.value == null) { + sb.append("null"); + } else { + org.apache.thrift.TBaseHelper.toString(this.value, sb); + } + first = false; + if (isSetTimestamp()) { + if (!first) sb.append(", "); + sb.append("timestamp:"); + sb.append(this.timestamp); + first = false; + } + if (!first) sb.append(", "); + sb.append("type:"); + sb.append(this.type); + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (row == null) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'row' was not present! Struct: " + toString()); + } + if (family == null) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'family' was not present! Struct: " + toString()); + } + if (qualifier == null) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'qualifier' was not present! Struct: " + toString()); + } + if (value == null) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'value' was not present! Struct: " + toString()); + } + // alas, we cannot check 'type' because it's a primitive and you chose the non-beans generator. + // 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 TColumnValueStandardSchemeFactory implements SchemeFactory { + public TColumnValueStandardScheme getScheme() { + return new TColumnValueStandardScheme(); + } + } + + private static class TColumnValueStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TColumnValue 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: // ROW + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.row = iprot.readBinary(); + struct.setRowIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // FAMILY + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.family = iprot.readBinary(); + struct.setFamilyIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 3: // QUALIFIER + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.qualifier = iprot.readBinary(); + struct.setQualifierIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 4: // VALUE + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.value = iprot.readBinary(); + struct.setValueIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 5: // TIMESTAMP + if (schemeField.type == org.apache.thrift.protocol.TType.I64) { + struct.timestamp = iprot.readI64(); + struct.setTimestampIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 6: // TYPE + if (schemeField.type == org.apache.thrift.protocol.TType.BYTE) { + struct.type = iprot.readByte(); + struct.setTypeIsSet(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(); + + // check for required fields of primitive type, which can't be checked in the validate method + if (!struct.isSetType()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'type' was not found in serialized data! Struct: " + toString()); + } + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TColumnValue struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.row != null) { + oprot.writeFieldBegin(ROW_FIELD_DESC); + oprot.writeBinary(struct.row); + oprot.writeFieldEnd(); + } + if (struct.family != null) { + oprot.writeFieldBegin(FAMILY_FIELD_DESC); + oprot.writeBinary(struct.family); + oprot.writeFieldEnd(); + } + if (struct.qualifier != null) { + oprot.writeFieldBegin(QUALIFIER_FIELD_DESC); + oprot.writeBinary(struct.qualifier); + oprot.writeFieldEnd(); + } + if (struct.value != null) { + oprot.writeFieldBegin(VALUE_FIELD_DESC); + oprot.writeBinary(struct.value); + oprot.writeFieldEnd(); + } + if (struct.isSetTimestamp()) { + oprot.writeFieldBegin(TIMESTAMP_FIELD_DESC); + oprot.writeI64(struct.timestamp); + oprot.writeFieldEnd(); + } + oprot.writeFieldBegin(TYPE_FIELD_DESC); + oprot.writeByte(struct.type); + oprot.writeFieldEnd(); + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TColumnValueTupleSchemeFactory implements SchemeFactory { + public TColumnValueTupleScheme getScheme() { + return new TColumnValueTupleScheme(); + } + } + + private static class TColumnValueTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TColumnValue struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + oprot.writeBinary(struct.row); + oprot.writeBinary(struct.family); + oprot.writeBinary(struct.qualifier); + oprot.writeBinary(struct.value); + oprot.writeByte(struct.type); + BitSet optionals = new BitSet(); + if (struct.isSetTimestamp()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetTimestamp()) { + oprot.writeI64(struct.timestamp); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TColumnValue struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.row = iprot.readBinary(); + struct.setRowIsSet(true); + struct.family = iprot.readBinary(); + struct.setFamilyIsSet(true); + struct.qualifier = iprot.readBinary(); + struct.setQualifierIsSet(true); + struct.value = iprot.readBinary(); + struct.setValueIsSet(true); + struct.type = iprot.readByte(); + struct.setTypeIsSet(true); + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.timestamp = iprot.readI64(); + struct.setTimestampIsSet(true); + } + } + } + +} + diff --git a/src/main/java/org/apache/hadoop/hbase/replication/thrift/generated/TEdit.java b/src/main/java/org/apache/hadoop/hbase/replication/thrift/generated/TEdit.java new file mode 100644 index 0000000..6c3e25d --- /dev/null +++ b/src/main/java/org/apache/hadoop/hbase/replication/thrift/generated/TEdit.java @@ -0,0 +1,637 @@ +/** + * Autogenerated by Thrift Compiler (0.9.1) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hadoop.hbase.replication.thrift.generated; + +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 org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +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 TEdit implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TEdit"); + + private static final org.apache.thrift.protocol.TField H_LOG_KEY_FIELD_DESC = new org.apache.thrift.protocol.TField("hLogKey", org.apache.thrift.protocol.TType.STRUCT, (short)1); + private static final org.apache.thrift.protocol.TField WAL_EDIT_FIELD_DESC = new org.apache.thrift.protocol.TField("walEdit", org.apache.thrift.protocol.TType.STRUCT, (short)2); + private static final org.apache.thrift.protocol.TField CLUSTER_IDS_FIELD_DESC = new org.apache.thrift.protocol.TField("clusterIds", org.apache.thrift.protocol.TType.LIST, (short)3); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TEditStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TEditTupleSchemeFactory()); + } + + public THLogKey hLogKey; // required + public TWalLEdit walEdit; // required + public List clusterIds; // 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 { + H_LOG_KEY((short)1, "hLogKey"), + WAL_EDIT((short)2, "walEdit"), + CLUSTER_IDS((short)3, "clusterIds"); + + 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: // H_LOG_KEY + return H_LOG_KEY; + case 2: // WAL_EDIT + return WAL_EDIT; + case 3: // CLUSTER_IDS + return CLUSTER_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.H_LOG_KEY, new org.apache.thrift.meta_data.FieldMetaData("hLogKey", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, THLogKey.class))); + tmpMap.put(_Fields.WAL_EDIT, new org.apache.thrift.meta_data.FieldMetaData("walEdit", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TWalLEdit.class))); + tmpMap.put(_Fields.CLUSTER_IDS, new org.apache.thrift.meta_data.FieldMetaData("clusterIds", 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, TClusterId.class)))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TEdit.class, metaDataMap); + } + + public TEdit() { + } + + public TEdit( + THLogKey hLogKey, + TWalLEdit walEdit, + List clusterIds) + { + this(); + this.hLogKey = hLogKey; + this.walEdit = walEdit; + this.clusterIds = clusterIds; + } + + /** + * Performs a deep copy on other. + */ + public TEdit(TEdit other) { + if (other.isSetHLogKey()) { + this.hLogKey = new THLogKey(other.hLogKey); + } + if (other.isSetWalEdit()) { + this.walEdit = new TWalLEdit(other.walEdit); + } + if (other.isSetClusterIds()) { + List __this__clusterIds = new ArrayList(other.clusterIds.size()); + for (TClusterId other_element : other.clusterIds) { + __this__clusterIds.add(new TClusterId(other_element)); + } + this.clusterIds = __this__clusterIds; + } + } + + public TEdit deepCopy() { + return new TEdit(this); + } + + @Override + public void clear() { + this.hLogKey = null; + this.walEdit = null; + this.clusterIds = null; + } + + public THLogKey getHLogKey() { + return this.hLogKey; + } + + public TEdit setHLogKey(THLogKey hLogKey) { + this.hLogKey = hLogKey; + return this; + } + + public void unsetHLogKey() { + this.hLogKey = null; + } + + /** Returns true if field hLogKey is set (has been assigned a value) and false otherwise */ + public boolean isSetHLogKey() { + return this.hLogKey != null; + } + + public void setHLogKeyIsSet(boolean value) { + if (!value) { + this.hLogKey = null; + } + } + + public TWalLEdit getWalEdit() { + return this.walEdit; + } + + public TEdit setWalEdit(TWalLEdit walEdit) { + this.walEdit = walEdit; + return this; + } + + public void unsetWalEdit() { + this.walEdit = null; + } + + /** Returns true if field walEdit is set (has been assigned a value) and false otherwise */ + public boolean isSetWalEdit() { + return this.walEdit != null; + } + + public void setWalEditIsSet(boolean value) { + if (!value) { + this.walEdit = null; + } + } + + public int getClusterIdsSize() { + return (this.clusterIds == null) ? 0 : this.clusterIds.size(); + } + + public java.util.Iterator getClusterIdsIterator() { + return (this.clusterIds == null) ? null : this.clusterIds.iterator(); + } + + public void addToClusterIds(TClusterId elem) { + if (this.clusterIds == null) { + this.clusterIds = new ArrayList(); + } + this.clusterIds.add(elem); + } + + public List getClusterIds() { + return this.clusterIds; + } + + public TEdit setClusterIds(List clusterIds) { + this.clusterIds = clusterIds; + return this; + } + + public void unsetClusterIds() { + this.clusterIds = null; + } + + /** Returns true if field clusterIds is set (has been assigned a value) and false otherwise */ + public boolean isSetClusterIds() { + return this.clusterIds != null; + } + + public void setClusterIdsIsSet(boolean value) { + if (!value) { + this.clusterIds = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case H_LOG_KEY: + if (value == null) { + unsetHLogKey(); + } else { + setHLogKey((THLogKey)value); + } + break; + + case WAL_EDIT: + if (value == null) { + unsetWalEdit(); + } else { + setWalEdit((TWalLEdit)value); + } + break; + + case CLUSTER_IDS: + if (value == null) { + unsetClusterIds(); + } else { + setClusterIds((List)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case H_LOG_KEY: + return getHLogKey(); + + case WAL_EDIT: + return getWalEdit(); + + case CLUSTER_IDS: + return getClusterIds(); + + } + 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 H_LOG_KEY: + return isSetHLogKey(); + case WAL_EDIT: + return isSetWalEdit(); + case CLUSTER_IDS: + return isSetClusterIds(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TEdit) + return this.equals((TEdit)that); + return false; + } + + public boolean equals(TEdit that) { + if (that == null) + return false; + + boolean this_present_hLogKey = true && this.isSetHLogKey(); + boolean that_present_hLogKey = true && that.isSetHLogKey(); + if (this_present_hLogKey || that_present_hLogKey) { + if (!(this_present_hLogKey && that_present_hLogKey)) + return false; + if (!this.hLogKey.equals(that.hLogKey)) + return false; + } + + boolean this_present_walEdit = true && this.isSetWalEdit(); + boolean that_present_walEdit = true && that.isSetWalEdit(); + if (this_present_walEdit || that_present_walEdit) { + if (!(this_present_walEdit && that_present_walEdit)) + return false; + if (!this.walEdit.equals(that.walEdit)) + return false; + } + + boolean this_present_clusterIds = true && this.isSetClusterIds(); + boolean that_present_clusterIds = true && that.isSetClusterIds(); + if (this_present_clusterIds || that_present_clusterIds) { + if (!(this_present_clusterIds && that_present_clusterIds)) + return false; + if (!this.clusterIds.equals(that.clusterIds)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + return 0; + } + + @Override + public int compareTo(TEdit other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(isSetHLogKey()).compareTo(other.isSetHLogKey()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetHLogKey()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.hLogKey, other.hLogKey); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetWalEdit()).compareTo(other.isSetWalEdit()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetWalEdit()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.walEdit, other.walEdit); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetClusterIds()).compareTo(other.isSetClusterIds()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetClusterIds()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.clusterIds, other.clusterIds); + 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("TEdit("); + boolean first = true; + + sb.append("hLogKey:"); + if (this.hLogKey == null) { + sb.append("null"); + } else { + sb.append(this.hLogKey); + } + first = false; + if (!first) sb.append(", "); + sb.append("walEdit:"); + if (this.walEdit == null) { + sb.append("null"); + } else { + sb.append(this.walEdit); + } + first = false; + if (!first) sb.append(", "); + sb.append("clusterIds:"); + if (this.clusterIds == null) { + sb.append("null"); + } else { + sb.append(this.clusterIds); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (hLogKey == null) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'hLogKey' was not present! Struct: " + toString()); + } + if (walEdit == null) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'walEdit' was not present! Struct: " + toString()); + } + if (clusterIds == null) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'clusterIds' was not present! Struct: " + toString()); + } + // check for sub-struct validity + if (hLogKey != null) { + hLogKey.validate(); + } + if (walEdit != null) { + walEdit.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 TEditStandardSchemeFactory implements SchemeFactory { + public TEditStandardScheme getScheme() { + return new TEditStandardScheme(); + } + } + + private static class TEditStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TEdit 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: // H_LOG_KEY + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.hLogKey = new THLogKey(); + struct.hLogKey.read(iprot); + struct.setHLogKeyIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // WAL_EDIT + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.walEdit = new TWalLEdit(); + struct.walEdit.read(iprot); + struct.setWalEditIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 3: // CLUSTER_IDS + if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { + { + org.apache.thrift.protocol.TList _list8 = iprot.readListBegin(); + struct.clusterIds = new ArrayList(_list8.size); + for (int _i9 = 0; _i9 < _list8.size; ++_i9) + { + TClusterId _elem10; + _elem10 = new TClusterId(); + _elem10.read(iprot); + struct.clusterIds.add(_elem10); + } + iprot.readListEnd(); + } + struct.setClusterIdsIsSet(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(); + + // check for required fields of primitive type, which can't be checked in the validate method + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TEdit struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.hLogKey != null) { + oprot.writeFieldBegin(H_LOG_KEY_FIELD_DESC); + struct.hLogKey.write(oprot); + oprot.writeFieldEnd(); + } + if (struct.walEdit != null) { + oprot.writeFieldBegin(WAL_EDIT_FIELD_DESC); + struct.walEdit.write(oprot); + oprot.writeFieldEnd(); + } + if (struct.clusterIds != null) { + oprot.writeFieldBegin(CLUSTER_IDS_FIELD_DESC); + { + oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.clusterIds.size())); + for (TClusterId _iter11 : struct.clusterIds) + { + _iter11.write(oprot); + } + oprot.writeListEnd(); + } + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TEditTupleSchemeFactory implements SchemeFactory { + public TEditTupleScheme getScheme() { + return new TEditTupleScheme(); + } + } + + private static class TEditTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TEdit struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + struct.hLogKey.write(oprot); + struct.walEdit.write(oprot); + { + oprot.writeI32(struct.clusterIds.size()); + for (TClusterId _iter12 : struct.clusterIds) + { + _iter12.write(oprot); + } + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TEdit struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.hLogKey = new THLogKey(); + struct.hLogKey.read(iprot); + struct.setHLogKeyIsSet(true); + struct.walEdit = new TWalLEdit(); + struct.walEdit.read(iprot); + struct.setWalEditIsSet(true); + { + org.apache.thrift.protocol.TList _list13 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); + struct.clusterIds = new ArrayList(_list13.size); + for (int _i14 = 0; _i14 < _list13.size; ++_i14) + { + TClusterId _elem15; + _elem15 = new TClusterId(); + _elem15.read(iprot); + struct.clusterIds.add(_elem15); + } + } + struct.setClusterIdsIsSet(true); + } + } + +} + diff --git a/src/main/java/org/apache/hadoop/hbase/replication/thrift/generated/THBaseService.java b/src/main/java/org/apache/hadoop/hbase/replication/thrift/generated/THBaseService.java new file mode 100644 index 0000000..6e4f633 --- /dev/null +++ b/src/main/java/org/apache/hadoop/hbase/replication/thrift/generated/THBaseService.java @@ -0,0 +1,2310 @@ +/** + * Autogenerated by Thrift Compiler (0.9.1) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hadoop.hbase.replication.thrift.generated; + +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 org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +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 THBaseService { + + public interface Iface { + + public void replicate(TBatchEdit edits) throws TIOError, org.apache.thrift.TException; + + public void ping() throws org.apache.thrift.TException; + + public String getClusterUUID() throws org.apache.thrift.TException; + + } + + public interface AsyncIface { + + public void replicate(TBatchEdit edits, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + + public void ping(org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + + public void getClusterUUID(org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + + } + + public static class Client extends org.apache.thrift.TServiceClient implements Iface { + public static class Factory implements org.apache.thrift.TServiceClientFactory { + public Factory() {} + public Client getClient(org.apache.thrift.protocol.TProtocol prot) { + return new Client(prot); + } + public Client getClient(org.apache.thrift.protocol.TProtocol iprot, org.apache.thrift.protocol.TProtocol oprot) { + return new Client(iprot, oprot); + } + } + + public Client(org.apache.thrift.protocol.TProtocol prot) + { + super(prot, prot); + } + + public Client(org.apache.thrift.protocol.TProtocol iprot, org.apache.thrift.protocol.TProtocol oprot) { + super(iprot, oprot); + } + + public void replicate(TBatchEdit edits) throws TIOError, org.apache.thrift.TException + { + send_replicate(edits); + recv_replicate(); + } + + public void send_replicate(TBatchEdit edits) throws org.apache.thrift.TException + { + replicate_args args = new replicate_args(); + args.setEdits(edits); + sendBase("replicate", args); + } + + public void recv_replicate() throws TIOError, org.apache.thrift.TException + { + replicate_result result = new replicate_result(); + receiveBase(result, "replicate"); + if (result.io != null) { + throw result.io; + } + return; + } + + public void ping() throws org.apache.thrift.TException + { + send_ping(); + recv_ping(); + } + + public void send_ping() throws org.apache.thrift.TException + { + ping_args args = new ping_args(); + sendBase("ping", args); + } + + public void recv_ping() throws org.apache.thrift.TException + { + ping_result result = new ping_result(); + receiveBase(result, "ping"); + return; + } + + public String getClusterUUID() throws org.apache.thrift.TException + { + send_getClusterUUID(); + return recv_getClusterUUID(); + } + + public void send_getClusterUUID() throws org.apache.thrift.TException + { + getClusterUUID_args args = new getClusterUUID_args(); + sendBase("getClusterUUID", args); + } + + public String recv_getClusterUUID() throws org.apache.thrift.TException + { + getClusterUUID_result result = new getClusterUUID_result(); + receiveBase(result, "getClusterUUID"); + if (result.isSetSuccess()) { + return result.success; + } + throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "getClusterUUID failed: unknown result"); + } + + } + public static class AsyncClient extends org.apache.thrift.async.TAsyncClient implements AsyncIface { + public static class Factory implements org.apache.thrift.async.TAsyncClientFactory { + private org.apache.thrift.async.TAsyncClientManager clientManager; + private org.apache.thrift.protocol.TProtocolFactory protocolFactory; + public Factory(org.apache.thrift.async.TAsyncClientManager clientManager, org.apache.thrift.protocol.TProtocolFactory protocolFactory) { + this.clientManager = clientManager; + this.protocolFactory = protocolFactory; + } + public AsyncClient getAsyncClient(org.apache.thrift.transport.TNonblockingTransport transport) { + return new AsyncClient(protocolFactory, clientManager, transport); + } + } + + public AsyncClient(org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.async.TAsyncClientManager clientManager, org.apache.thrift.transport.TNonblockingTransport transport) { + super(protocolFactory, clientManager, transport); + } + + public void replicate(TBatchEdit edits, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + checkReady(); + replicate_call method_call = new replicate_call(edits, resultHandler, this, ___protocolFactory, ___transport); + this.___currentMethod = method_call; + ___manager.call(method_call); + } + + public static class replicate_call extends org.apache.thrift.async.TAsyncMethodCall { + private TBatchEdit edits; + public replicate_call(TBatchEdit edits, 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.edits = edits; + } + + public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { + prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("replicate", org.apache.thrift.protocol.TMessageType.CALL, 0)); + replicate_args args = new replicate_args(); + args.setEdits(edits); + args.write(prot); + prot.writeMessageEnd(); + } + + public void getResult() throws TIOError, 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_replicate(); + } + } + + public void ping(org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + checkReady(); + ping_call method_call = new ping_call(resultHandler, this, ___protocolFactory, ___transport); + this.___currentMethod = method_call; + ___manager.call(method_call); + } + + public static class ping_call extends org.apache.thrift.async.TAsyncMethodCall { + public ping_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("ping", org.apache.thrift.protocol.TMessageType.CALL, 0)); + ping_args args = new ping_args(); + 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_ping(); + } + } + + public void getClusterUUID(org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + checkReady(); + getClusterUUID_call method_call = new getClusterUUID_call(resultHandler, this, ___protocolFactory, ___transport); + this.___currentMethod = method_call; + ___manager.call(method_call); + } + + public static class getClusterUUID_call extends org.apache.thrift.async.TAsyncMethodCall { + public getClusterUUID_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("getClusterUUID", org.apache.thrift.protocol.TMessageType.CALL, 0)); + getClusterUUID_args args = new getClusterUUID_args(); + args.write(prot); + prot.writeMessageEnd(); + } + + public String 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_getClusterUUID(); + } + } + + } + + public static class Processor extends org.apache.thrift.TBaseProcessor implements org.apache.thrift.TProcessor { + private static final Logger LOGGER = LoggerFactory.getLogger(Processor.class.getName()); + public Processor(I iface) { + super(iface, getProcessMap(new HashMap>())); + } + + protected Processor(I iface, Map> processMap) { + super(iface, getProcessMap(processMap)); + } + + private static Map> getProcessMap(Map> processMap) { + processMap.put("replicate", new replicate()); + processMap.put("ping", new ping()); + processMap.put("getClusterUUID", new getClusterUUID()); + return processMap; + } + + public static class replicate extends org.apache.thrift.ProcessFunction { + public replicate() { + super("replicate"); + } + + public replicate_args getEmptyArgsInstance() { + return new replicate_args(); + } + + protected boolean isOneway() { + return false; + } + + public replicate_result getResult(I iface, replicate_args args) throws org.apache.thrift.TException { + replicate_result result = new replicate_result(); + try { + iface.replicate(args.edits); + } catch (TIOError io) { + result.io = io; + } + return result; + } + } + + public static class ping extends org.apache.thrift.ProcessFunction { + public ping() { + super("ping"); + } + + public ping_args getEmptyArgsInstance() { + return new ping_args(); + } + + protected boolean isOneway() { + return false; + } + + public ping_result getResult(I iface, ping_args args) throws org.apache.thrift.TException { + ping_result result = new ping_result(); + iface.ping(); + return result; + } + } + + public static class getClusterUUID extends org.apache.thrift.ProcessFunction { + public getClusterUUID() { + super("getClusterUUID"); + } + + public getClusterUUID_args getEmptyArgsInstance() { + return new getClusterUUID_args(); + } + + protected boolean isOneway() { + return false; + } + + public getClusterUUID_result getResult(I iface, getClusterUUID_args args) throws org.apache.thrift.TException { + getClusterUUID_result result = new getClusterUUID_result(); + result.success = iface.getClusterUUID(); + return result; + } + } + + } + + public static class AsyncProcessor extends org.apache.thrift.TBaseAsyncProcessor { + private static final Logger LOGGER = LoggerFactory.getLogger(AsyncProcessor.class.getName()); + public AsyncProcessor(I iface) { + super(iface, getProcessMap(new HashMap>())); + } + + protected AsyncProcessor(I iface, Map> processMap) { + super(iface, getProcessMap(processMap)); + } + + private static Map> getProcessMap(Map> processMap) { + processMap.put("replicate", new replicate()); + processMap.put("ping", new ping()); + processMap.put("getClusterUUID", new getClusterUUID()); + return processMap; + } + + public static class replicate extends org.apache.thrift.AsyncProcessFunction { + public replicate() { + super("replicate"); + } + + public replicate_args getEmptyArgsInstance() { + return new replicate_args(); + } + + public AsyncMethodCallback getResultHandler(final AsyncFrameBuffer fb, final int seqid) { + final org.apache.thrift.AsyncProcessFunction fcall = this; + return new AsyncMethodCallback() { + public void onComplete(Void o) { + replicate_result result = new replicate_result(); + try { + fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); + return; + } catch (Exception e) { + LOGGER.error("Exception writing to internal frame buffer", e); + } + fb.close(); + } + public void onError(Exception e) { + byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; + org.apache.thrift.TBase msg; + replicate_result result = new replicate_result(); + if (e instanceof TIOError) { + result.io = (TIOError) e; + result.setIoIsSet(true); + msg = result; + } + else + { + msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION; + msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage()); + } + try { + fcall.sendResponse(fb,msg,msgType,seqid); + return; + } catch (Exception ex) { + LOGGER.error("Exception writing to internal frame buffer", ex); + } + fb.close(); + } + }; + } + + protected boolean isOneway() { + return false; + } + + public void start(I iface, replicate_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws TException { + iface.replicate(args.edits,resultHandler); + } + } + + public static class ping extends org.apache.thrift.AsyncProcessFunction { + public ping() { + super("ping"); + } + + public ping_args getEmptyArgsInstance() { + return new ping_args(); + } + + public AsyncMethodCallback getResultHandler(final AsyncFrameBuffer fb, final int seqid) { + final org.apache.thrift.AsyncProcessFunction fcall = this; + return new AsyncMethodCallback() { + public void onComplete(Void o) { + ping_result result = new ping_result(); + try { + fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); + return; + } catch (Exception e) { + LOGGER.error("Exception writing to internal frame buffer", e); + } + fb.close(); + } + public void onError(Exception e) { + byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; + org.apache.thrift.TBase msg; + ping_result result = new ping_result(); + { + msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION; + msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage()); + } + try { + fcall.sendResponse(fb,msg,msgType,seqid); + return; + } catch (Exception ex) { + LOGGER.error("Exception writing to internal frame buffer", ex); + } + fb.close(); + } + }; + } + + protected boolean isOneway() { + return false; + } + + public void start(I iface, ping_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws TException { + iface.ping(resultHandler); + } + } + + public static class getClusterUUID extends org.apache.thrift.AsyncProcessFunction { + public getClusterUUID() { + super("getClusterUUID"); + } + + public getClusterUUID_args getEmptyArgsInstance() { + return new getClusterUUID_args(); + } + + public AsyncMethodCallback getResultHandler(final AsyncFrameBuffer fb, final int seqid) { + final org.apache.thrift.AsyncProcessFunction fcall = this; + return new AsyncMethodCallback() { + public void onComplete(String o) { + getClusterUUID_result result = new getClusterUUID_result(); + result.success = o; + try { + fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); + return; + } catch (Exception e) { + LOGGER.error("Exception writing to internal frame buffer", e); + } + fb.close(); + } + public void onError(Exception e) { + byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; + org.apache.thrift.TBase msg; + getClusterUUID_result result = new getClusterUUID_result(); + { + msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION; + msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage()); + } + try { + fcall.sendResponse(fb,msg,msgType,seqid); + return; + } catch (Exception ex) { + LOGGER.error("Exception writing to internal frame buffer", ex); + } + fb.close(); + } + }; + } + + protected boolean isOneway() { + return false; + } + + public void start(I iface, getClusterUUID_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws TException { + iface.getClusterUUID(resultHandler); + } + } + + } + + public static class replicate_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("replicate_args"); + + private static final org.apache.thrift.protocol.TField EDITS_FIELD_DESC = new org.apache.thrift.protocol.TField("edits", org.apache.thrift.protocol.TType.STRUCT, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new replicate_argsStandardSchemeFactory()); + schemes.put(TupleScheme.class, new replicate_argsTupleSchemeFactory()); + } + + public TBatchEdit edits; // 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 { + EDITS((short)1, "edits"); + + 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: // EDITS + return EDITS; + 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.EDITS, new org.apache.thrift.meta_data.FieldMetaData("edits", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TBatchEdit.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(replicate_args.class, metaDataMap); + } + + public replicate_args() { + } + + public replicate_args( + TBatchEdit edits) + { + this(); + this.edits = edits; + } + + /** + * Performs a deep copy on other. + */ + public replicate_args(replicate_args other) { + if (other.isSetEdits()) { + this.edits = new TBatchEdit(other.edits); + } + } + + public replicate_args deepCopy() { + return new replicate_args(this); + } + + @Override + public void clear() { + this.edits = null; + } + + public TBatchEdit getEdits() { + return this.edits; + } + + public replicate_args setEdits(TBatchEdit edits) { + this.edits = edits; + return this; + } + + public void unsetEdits() { + this.edits = null; + } + + /** Returns true if field edits is set (has been assigned a value) and false otherwise */ + public boolean isSetEdits() { + return this.edits != null; + } + + public void setEditsIsSet(boolean value) { + if (!value) { + this.edits = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case EDITS: + if (value == null) { + unsetEdits(); + } else { + setEdits((TBatchEdit)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case EDITS: + return getEdits(); + + } + 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 EDITS: + return isSetEdits(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof replicate_args) + return this.equals((replicate_args)that); + return false; + } + + public boolean equals(replicate_args that) { + if (that == null) + return false; + + boolean this_present_edits = true && this.isSetEdits(); + boolean that_present_edits = true && that.isSetEdits(); + if (this_present_edits || that_present_edits) { + if (!(this_present_edits && that_present_edits)) + return false; + if (!this.edits.equals(that.edits)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + return 0; + } + + @Override + public int compareTo(replicate_args other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(isSetEdits()).compareTo(other.isSetEdits()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetEdits()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.edits, other.edits); + 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("replicate_args("); + boolean first = true; + + sb.append("edits:"); + if (this.edits == null) { + sb.append("null"); + } else { + sb.append(this.edits); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (edits == null) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'edits' was not present! Struct: " + toString()); + } + // check for sub-struct validity + if (edits != null) { + edits.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 replicate_argsStandardSchemeFactory implements SchemeFactory { + public replicate_argsStandardScheme getScheme() { + return new replicate_argsStandardScheme(); + } + } + + private static class replicate_argsStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, replicate_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: // EDITS + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.edits = new TBatchEdit(); + struct.edits.read(iprot); + struct.setEditsIsSet(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(); + + // check for required fields of primitive type, which can't be checked in the validate method + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, replicate_args struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.edits != null) { + oprot.writeFieldBegin(EDITS_FIELD_DESC); + struct.edits.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class replicate_argsTupleSchemeFactory implements SchemeFactory { + public replicate_argsTupleScheme getScheme() { + return new replicate_argsTupleScheme(); + } + } + + private static class replicate_argsTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, replicate_args struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + struct.edits.write(oprot); + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, replicate_args struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.edits = new TBatchEdit(); + struct.edits.read(iprot); + struct.setEditsIsSet(true); + } + } + + } + + public static class replicate_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("replicate_result"); + + private static final org.apache.thrift.protocol.TField IO_FIELD_DESC = new org.apache.thrift.protocol.TField("io", org.apache.thrift.protocol.TType.STRUCT, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new replicate_resultStandardSchemeFactory()); + schemes.put(TupleScheme.class, new replicate_resultTupleSchemeFactory()); + } + + public TIOError io; // 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 { + IO((short)1, "io"); + + 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: // IO + return IO; + 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.IO, new org.apache.thrift.meta_data.FieldMetaData("io", 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(replicate_result.class, metaDataMap); + } + + public replicate_result() { + } + + public replicate_result( + TIOError io) + { + this(); + this.io = io; + } + + /** + * Performs a deep copy on other. + */ + public replicate_result(replicate_result other) { + if (other.isSetIo()) { + this.io = new TIOError(other.io); + } + } + + public replicate_result deepCopy() { + return new replicate_result(this); + } + + @Override + public void clear() { + this.io = null; + } + + public TIOError getIo() { + return this.io; + } + + public replicate_result setIo(TIOError io) { + this.io = io; + return this; + } + + public void unsetIo() { + this.io = null; + } + + /** Returns true if field io is set (has been assigned a value) and false otherwise */ + public boolean isSetIo() { + return this.io != null; + } + + public void setIoIsSet(boolean value) { + if (!value) { + this.io = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case IO: + if (value == null) { + unsetIo(); + } else { + setIo((TIOError)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case IO: + return getIo(); + + } + 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 IO: + return isSetIo(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof replicate_result) + return this.equals((replicate_result)that); + return false; + } + + public boolean equals(replicate_result that) { + if (that == null) + return false; + + boolean this_present_io = true && this.isSetIo(); + boolean that_present_io = true && that.isSetIo(); + if (this_present_io || that_present_io) { + if (!(this_present_io && that_present_io)) + return false; + if (!this.io.equals(that.io)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + return 0; + } + + @Override + public int compareTo(replicate_result other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(isSetIo()).compareTo(other.isSetIo()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetIo()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.io, other.io); + 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("replicate_result("); + boolean first = true; + + sb.append("io:"); + if (this.io == null) { + sb.append("null"); + } else { + sb.append(this.io); + } + 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 replicate_resultStandardSchemeFactory implements SchemeFactory { + public replicate_resultStandardScheme getScheme() { + return new replicate_resultStandardScheme(); + } + } + + private static class replicate_resultStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, replicate_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: // IO + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.io = new TIOError(); + struct.io.read(iprot); + struct.setIoIsSet(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(); + + // check for required fields of primitive type, which can't be checked in the validate method + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, replicate_result struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.io != null) { + oprot.writeFieldBegin(IO_FIELD_DESC); + struct.io.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class replicate_resultTupleSchemeFactory implements SchemeFactory { + public replicate_resultTupleScheme getScheme() { + return new replicate_resultTupleScheme(); + } + } + + private static class replicate_resultTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, replicate_result struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetIo()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetIo()) { + struct.io.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, replicate_result struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.io = new TIOError(); + struct.io.read(iprot); + struct.setIoIsSet(true); + } + } + } + + } + + public static class ping_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("ping_args"); + + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new ping_argsStandardSchemeFactory()); + schemes.put(TupleScheme.class, new ping_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(ping_args.class, metaDataMap); + } + + public ping_args() { + } + + /** + * Performs a deep copy on other. + */ + public ping_args(ping_args other) { + } + + public ping_args deepCopy() { + return new ping_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 ping_args) + return this.equals((ping_args)that); + return false; + } + + public boolean equals(ping_args that) { + if (that == null) + return false; + + return true; + } + + @Override + public int hashCode() { + return 0; + } + + @Override + public int compareTo(ping_args other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + 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("ping_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 ping_argsStandardSchemeFactory implements SchemeFactory { + public ping_argsStandardScheme getScheme() { + return new ping_argsStandardScheme(); + } + } + + private static class ping_argsStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, ping_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(); + + // check for required fields of primitive type, which can't be checked in the validate method + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, ping_args struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class ping_argsTupleSchemeFactory implements SchemeFactory { + public ping_argsTupleScheme getScheme() { + return new ping_argsTupleScheme(); + } + } + + private static class ping_argsTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, ping_args struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, ping_args struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + } + } + + } + + public static class ping_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("ping_result"); + + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new ping_resultStandardSchemeFactory()); + schemes.put(TupleScheme.class, new ping_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(ping_result.class, metaDataMap); + } + + public ping_result() { + } + + /** + * Performs a deep copy on other. + */ + public ping_result(ping_result other) { + } + + public ping_result deepCopy() { + return new ping_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 ping_result) + return this.equals((ping_result)that); + return false; + } + + public boolean equals(ping_result that) { + if (that == null) + return false; + + return true; + } + + @Override + public int hashCode() { + return 0; + } + + @Override + public int compareTo(ping_result other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + 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("ping_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 ping_resultStandardSchemeFactory implements SchemeFactory { + public ping_resultStandardScheme getScheme() { + return new ping_resultStandardScheme(); + } + } + + private static class ping_resultStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, ping_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(); + + // check for required fields of primitive type, which can't be checked in the validate method + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, ping_result struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class ping_resultTupleSchemeFactory implements SchemeFactory { + public ping_resultTupleScheme getScheme() { + return new ping_resultTupleScheme(); + } + } + + private static class ping_resultTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, ping_result struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, ping_result struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + } + } + + } + + public static class getClusterUUID_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getClusterUUID_args"); + + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new getClusterUUID_argsStandardSchemeFactory()); + schemes.put(TupleScheme.class, new getClusterUUID_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(getClusterUUID_args.class, metaDataMap); + } + + public getClusterUUID_args() { + } + + /** + * Performs a deep copy on other. + */ + public getClusterUUID_args(getClusterUUID_args other) { + } + + public getClusterUUID_args deepCopy() { + return new getClusterUUID_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 getClusterUUID_args) + return this.equals((getClusterUUID_args)that); + return false; + } + + public boolean equals(getClusterUUID_args that) { + if (that == null) + return false; + + return true; + } + + @Override + public int hashCode() { + return 0; + } + + @Override + public int compareTo(getClusterUUID_args other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + 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("getClusterUUID_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 getClusterUUID_argsStandardSchemeFactory implements SchemeFactory { + public getClusterUUID_argsStandardScheme getScheme() { + return new getClusterUUID_argsStandardScheme(); + } + } + + private static class getClusterUUID_argsStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, getClusterUUID_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(); + + // check for required fields of primitive type, which can't be checked in the validate method + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, getClusterUUID_args struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class getClusterUUID_argsTupleSchemeFactory implements SchemeFactory { + public getClusterUUID_argsTupleScheme getScheme() { + return new getClusterUUID_argsTupleScheme(); + } + } + + private static class getClusterUUID_argsTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, getClusterUUID_args struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, getClusterUUID_args struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + } + } + + } + + public static class getClusterUUID_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getClusterUUID_result"); + + private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRING, (short)0); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new getClusterUUID_resultStandardSchemeFactory()); + schemes.put(TupleScheme.class, new getClusterUUID_resultTupleSchemeFactory()); + } + + public String 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.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getClusterUUID_result.class, metaDataMap); + } + + public getClusterUUID_result() { + } + + public getClusterUUID_result( + String success) + { + this(); + this.success = success; + } + + /** + * Performs a deep copy on other. + */ + public getClusterUUID_result(getClusterUUID_result other) { + if (other.isSetSuccess()) { + this.success = other.success; + } + } + + public getClusterUUID_result deepCopy() { + return new getClusterUUID_result(this); + } + + @Override + public void clear() { + this.success = null; + } + + public String getSuccess() { + return this.success; + } + + public getClusterUUID_result setSuccess(String success) { + this.success = success; + return this; + } + + 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((String)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 getClusterUUID_result) + return this.equals((getClusterUUID_result)that); + return false; + } + + public boolean equals(getClusterUUID_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() { + return 0; + } + + @Override + public int compareTo(getClusterUUID_result other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(other.isSetSuccess()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetSuccess()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.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("getClusterUUID_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 + } + + 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 getClusterUUID_resultStandardSchemeFactory implements SchemeFactory { + public getClusterUUID_resultStandardScheme getScheme() { + return new getClusterUUID_resultStandardScheme(); + } + } + + private static class getClusterUUID_resultStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, getClusterUUID_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.STRING) { + struct.success = iprot.readString(); + 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(); + + // check for required fields of primitive type, which can't be checked in the validate method + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, getClusterUUID_result struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.success != null) { + oprot.writeFieldBegin(SUCCESS_FIELD_DESC); + oprot.writeString(struct.success); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class getClusterUUID_resultTupleSchemeFactory implements SchemeFactory { + public getClusterUUID_resultTupleScheme getScheme() { + return new getClusterUUID_resultTupleScheme(); + } + } + + private static class getClusterUUID_resultTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, getClusterUUID_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()) { + oprot.writeString(struct.success); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, getClusterUUID_result struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.success = iprot.readString(); + struct.setSuccessIsSet(true); + } + } + } + + } + +} diff --git a/src/main/java/org/apache/hadoop/hbase/replication/thrift/generated/THLogKey.java b/src/main/java/org/apache/hadoop/hbase/replication/thrift/generated/THLogKey.java new file mode 100644 index 0000000..93615a8 --- /dev/null +++ b/src/main/java/org/apache/hadoop/hbase/replication/thrift/generated/THLogKey.java @@ -0,0 +1,582 @@ +/** + * Autogenerated by Thrift Compiler (0.9.1) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hadoop.hbase.replication.thrift.generated; + +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 org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +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; + +/** + * Mapping for HLogKey + * + */ +public class THLogKey implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("THLogKey"); + + private static final org.apache.thrift.protocol.TField TABLE_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("tableName", org.apache.thrift.protocol.TType.STRING, (short)1); + private static final org.apache.thrift.protocol.TField WRITE_TIME_FIELD_DESC = new org.apache.thrift.protocol.TField("writeTime", org.apache.thrift.protocol.TType.I64, (short)2); + private static final org.apache.thrift.protocol.TField SEQ_NUM_FIELD_DESC = new org.apache.thrift.protocol.TField("seqNum", org.apache.thrift.protocol.TType.I64, (short)3); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new THLogKeyStandardSchemeFactory()); + schemes.put(TupleScheme.class, new THLogKeyTupleSchemeFactory()); + } + + public ByteBuffer tableName; // required + public long writeTime; // required + public long seqNum; // 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 { + TABLE_NAME((short)1, "tableName"), + WRITE_TIME((short)2, "writeTime"), + SEQ_NUM((short)3, "seqNum"); + + 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: // TABLE_NAME + return TABLE_NAME; + case 2: // WRITE_TIME + return WRITE_TIME; + case 3: // SEQ_NUM + return SEQ_NUM; + 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 __WRITETIME_ISSET_ID = 0; + private static final int __SEQNUM_ISSET_ID = 1; + 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.TABLE_NAME, 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 , true))); + tmpMap.put(_Fields.WRITE_TIME, new org.apache.thrift.meta_data.FieldMetaData("writeTime", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64))); + tmpMap.put(_Fields.SEQ_NUM, new org.apache.thrift.meta_data.FieldMetaData("seqNum", 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(THLogKey.class, metaDataMap); + } + + public THLogKey() { + } + + public THLogKey( + ByteBuffer tableName, + long writeTime, + long seqNum) + { + this(); + this.tableName = tableName; + this.writeTime = writeTime; + setWriteTimeIsSet(true); + this.seqNum = seqNum; + setSeqNumIsSet(true); + } + + /** + * Performs a deep copy on other. + */ + public THLogKey(THLogKey other) { + __isset_bitfield = other.__isset_bitfield; + if (other.isSetTableName()) { + this.tableName = org.apache.thrift.TBaseHelper.copyBinary(other.tableName); +; + } + this.writeTime = other.writeTime; + this.seqNum = other.seqNum; + } + + public THLogKey deepCopy() { + return new THLogKey(this); + } + + @Override + public void clear() { + this.tableName = null; + setWriteTimeIsSet(false); + this.writeTime = 0; + setSeqNumIsSet(false); + this.seqNum = 0; + } + + public byte[] getTableName() { + setTableName(org.apache.thrift.TBaseHelper.rightSize(tableName)); + return tableName == null ? null : tableName.array(); + } + + public ByteBuffer bufferForTableName() { + return tableName; + } + + public THLogKey setTableName(byte[] tableName) { + setTableName(tableName == null ? (ByteBuffer)null : ByteBuffer.wrap(tableName)); + return this; + } + + public THLogKey setTableName(ByteBuffer tableName) { + this.tableName = tableName; + return this; + } + + 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 long getWriteTime() { + return this.writeTime; + } + + public THLogKey setWriteTime(long writeTime) { + this.writeTime = writeTime; + setWriteTimeIsSet(true); + return this; + } + + public void unsetWriteTime() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __WRITETIME_ISSET_ID); + } + + /** Returns true if field writeTime is set (has been assigned a value) and false otherwise */ + public boolean isSetWriteTime() { + return EncodingUtils.testBit(__isset_bitfield, __WRITETIME_ISSET_ID); + } + + public void setWriteTimeIsSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __WRITETIME_ISSET_ID, value); + } + + public long getSeqNum() { + return this.seqNum; + } + + public THLogKey setSeqNum(long seqNum) { + this.seqNum = seqNum; + setSeqNumIsSet(true); + return this; + } + + public void unsetSeqNum() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __SEQNUM_ISSET_ID); + } + + /** Returns true if field seqNum is set (has been assigned a value) and false otherwise */ + public boolean isSetSeqNum() { + return EncodingUtils.testBit(__isset_bitfield, __SEQNUM_ISSET_ID); + } + + public void setSeqNumIsSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __SEQNUM_ISSET_ID, value); + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case TABLE_NAME: + if (value == null) { + unsetTableName(); + } else { + setTableName((ByteBuffer)value); + } + break; + + case WRITE_TIME: + if (value == null) { + unsetWriteTime(); + } else { + setWriteTime((Long)value); + } + break; + + case SEQ_NUM: + if (value == null) { + unsetSeqNum(); + } else { + setSeqNum((Long)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case TABLE_NAME: + return getTableName(); + + case WRITE_TIME: + return Long.valueOf(getWriteTime()); + + case SEQ_NUM: + return Long.valueOf(getSeqNum()); + + } + 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 TABLE_NAME: + return isSetTableName(); + case WRITE_TIME: + return isSetWriteTime(); + case SEQ_NUM: + return isSetSeqNum(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof THLogKey) + return this.equals((THLogKey)that); + return false; + } + + public boolean equals(THLogKey that) { + if (that == null) + 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_writeTime = true; + boolean that_present_writeTime = true; + if (this_present_writeTime || that_present_writeTime) { + if (!(this_present_writeTime && that_present_writeTime)) + return false; + if (this.writeTime != that.writeTime) + return false; + } + + boolean this_present_seqNum = true; + boolean that_present_seqNum = true; + if (this_present_seqNum || that_present_seqNum) { + if (!(this_present_seqNum && that_present_seqNum)) + return false; + if (this.seqNum != that.seqNum) + return false; + } + + return true; + } + + @Override + public int hashCode() { + return 0; + } + + @Override + public int compareTo(THLogKey other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(isSetTableName()).compareTo(other.isSetTableName()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetTableName()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tableName, other.tableName); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetWriteTime()).compareTo(other.isSetWriteTime()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetWriteTime()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.writeTime, other.writeTime); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetSeqNum()).compareTo(other.isSetSeqNum()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetSeqNum()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.seqNum, other.seqNum); + 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("THLogKey("); + boolean first = true; + + sb.append("tableName:"); + if (this.tableName == null) { + sb.append("null"); + } else { + org.apache.thrift.TBaseHelper.toString(this.tableName, sb); + } + first = false; + if (!first) sb.append(", "); + sb.append("writeTime:"); + sb.append(this.writeTime); + first = false; + if (!first) sb.append(", "); + sb.append("seqNum:"); + sb.append(this.seqNum); + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (tableName == null) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'tableName' was not present! Struct: " + toString()); + } + // alas, we cannot check 'writeTime' because it's a primitive and you chose the non-beans generator. + // alas, we cannot check 'seqNum' because it's a primitive and you chose the non-beans generator. + // 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 THLogKeyStandardSchemeFactory implements SchemeFactory { + public THLogKeyStandardScheme getScheme() { + return new THLogKeyStandardScheme(); + } + } + + private static class THLogKeyStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, THLogKey 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: // TABLE_NAME + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.tableName = iprot.readBinary(); + struct.setTableNameIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // WRITE_TIME + if (schemeField.type == org.apache.thrift.protocol.TType.I64) { + struct.writeTime = iprot.readI64(); + struct.setWriteTimeIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 3: // SEQ_NUM + if (schemeField.type == org.apache.thrift.protocol.TType.I64) { + struct.seqNum = iprot.readI64(); + struct.setSeqNumIsSet(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(); + + // check for required fields of primitive type, which can't be checked in the validate method + if (!struct.isSetWriteTime()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'writeTime' was not found in serialized data! Struct: " + toString()); + } + if (!struct.isSetSeqNum()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'seqNum' was not found in serialized data! Struct: " + toString()); + } + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, THLogKey struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.tableName != null) { + oprot.writeFieldBegin(TABLE_NAME_FIELD_DESC); + oprot.writeBinary(struct.tableName); + oprot.writeFieldEnd(); + } + oprot.writeFieldBegin(WRITE_TIME_FIELD_DESC); + oprot.writeI64(struct.writeTime); + oprot.writeFieldEnd(); + oprot.writeFieldBegin(SEQ_NUM_FIELD_DESC); + oprot.writeI64(struct.seqNum); + oprot.writeFieldEnd(); + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class THLogKeyTupleSchemeFactory implements SchemeFactory { + public THLogKeyTupleScheme getScheme() { + return new THLogKeyTupleScheme(); + } + } + + private static class THLogKeyTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, THLogKey struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + oprot.writeBinary(struct.tableName); + oprot.writeI64(struct.writeTime); + oprot.writeI64(struct.seqNum); + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, THLogKey struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.tableName = iprot.readBinary(); + struct.setTableNameIsSet(true); + struct.writeTime = iprot.readI64(); + struct.setWriteTimeIsSet(true); + struct.seqNum = iprot.readI64(); + struct.setSeqNumIsSet(true); + } + } + +} + diff --git a/src/main/java/org/apache/hadoop/hbase/replication/thrift/generated/TIOError.java b/src/main/java/org/apache/hadoop/hbase/replication/thrift/generated/TIOError.java new file mode 100644 index 0000000..c2fcca6 --- /dev/null +++ b/src/main/java/org/apache/hadoop/hbase/replication/thrift/generated/TIOError.java @@ -0,0 +1,391 @@ +/** + * Autogenerated by Thrift Compiler (0.9.1) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hadoop.hbase.replication.thrift.generated; + +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 org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +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; + +/** + * A TIOError exception signals that an error occurred communicating + * to the HBase master or a HBase region server. Also used to return + * more general HBase error conditions. + */ +public class TIOError extends TException implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TIOError"); + + 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 TIOErrorStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TIOErrorTupleSchemeFactory()); + } + + public String message; // 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 { + 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 + private _Fields optionals[] = {_Fields.MESSAGE}; + 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.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(TIOError.class, metaDataMap); + } + + public TIOError() { + } + + /** + * Performs a deep copy on other. + */ + public TIOError(TIOError other) { + if (other.isSetMessage()) { + this.message = other.message; + } + } + + public TIOError deepCopy() { + return new TIOError(this); + } + + @Override + public void clear() { + this.message = null; + } + + public String getMessage() { + return this.message; + } + + public TIOError setMessage(String message) { + this.message = message; + return this; + } + + 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 TIOError) + return this.equals((TIOError)that); + return false; + } + + public boolean equals(TIOError 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() { + return 0; + } + + @Override + public int compareTo(TIOError other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(isSetMessage()).compareTo(other.isSetMessage()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetMessage()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.message, other.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("TIOError("); + boolean first = true; + + if (isSetMessage()) { + 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 TIOErrorStandardSchemeFactory implements SchemeFactory { + public TIOErrorStandardScheme getScheme() { + return new TIOErrorStandardScheme(); + } + } + + private static class TIOErrorStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TIOError 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(); + + // check for required fields of primitive type, which can't be checked in the validate method + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TIOError struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.message != null) { + if (struct.isSetMessage()) { + oprot.writeFieldBegin(MESSAGE_FIELD_DESC); + oprot.writeString(struct.message); + oprot.writeFieldEnd(); + } + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TIOErrorTupleSchemeFactory implements SchemeFactory { + public TIOErrorTupleScheme getScheme() { + return new TIOErrorTupleScheme(); + } + } + + private static class TIOErrorTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TIOError 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, TIOError 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 a/src/main/java/org/apache/hadoop/hbase/replication/thrift/generated/TIllegalArgument.java b/src/main/java/org/apache/hadoop/hbase/replication/thrift/generated/TIllegalArgument.java new file mode 100644 index 0000000..ade826f --- /dev/null +++ b/src/main/java/org/apache/hadoop/hbase/replication/thrift/generated/TIllegalArgument.java @@ -0,0 +1,390 @@ +/** + * Autogenerated by Thrift Compiler (0.9.1) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hadoop.hbase.replication.thrift.generated; + +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 org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +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; + +/** + * A TIllegalArgument exception indicates an illegal or invalid + * argument was passed into a procedure. + */ +public class TIllegalArgument extends TException implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TIllegalArgument"); + + 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 TIllegalArgumentStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TIllegalArgumentTupleSchemeFactory()); + } + + public String message; // 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 { + 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 + private _Fields optionals[] = {_Fields.MESSAGE}; + 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.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(TIllegalArgument.class, metaDataMap); + } + + public TIllegalArgument() { + } + + /** + * Performs a deep copy on other. + */ + public TIllegalArgument(TIllegalArgument other) { + if (other.isSetMessage()) { + this.message = other.message; + } + } + + public TIllegalArgument deepCopy() { + return new TIllegalArgument(this); + } + + @Override + public void clear() { + this.message = null; + } + + public String getMessage() { + return this.message; + } + + public TIllegalArgument setMessage(String message) { + this.message = message; + return this; + } + + 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 TIllegalArgument) + return this.equals((TIllegalArgument)that); + return false; + } + + public boolean equals(TIllegalArgument 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() { + return 0; + } + + @Override + public int compareTo(TIllegalArgument other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(isSetMessage()).compareTo(other.isSetMessage()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetMessage()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.message, other.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("TIllegalArgument("); + boolean first = true; + + if (isSetMessage()) { + 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 TIllegalArgumentStandardSchemeFactory implements SchemeFactory { + public TIllegalArgumentStandardScheme getScheme() { + return new TIllegalArgumentStandardScheme(); + } + } + + private static class TIllegalArgumentStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TIllegalArgument 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(); + + // check for required fields of primitive type, which can't be checked in the validate method + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TIllegalArgument struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.message != null) { + if (struct.isSetMessage()) { + oprot.writeFieldBegin(MESSAGE_FIELD_DESC); + oprot.writeString(struct.message); + oprot.writeFieldEnd(); + } + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TIllegalArgumentTupleSchemeFactory implements SchemeFactory { + public TIllegalArgumentTupleScheme getScheme() { + return new TIllegalArgumentTupleScheme(); + } + } + + private static class TIllegalArgumentTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TIllegalArgument 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, TIllegalArgument 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 a/src/main/java/org/apache/hadoop/hbase/replication/thrift/generated/TWalLEdit.java b/src/main/java/org/apache/hadoop/hbase/replication/thrift/generated/TWalLEdit.java new file mode 100644 index 0000000..e47e036 --- /dev/null +++ b/src/main/java/org/apache/hadoop/hbase/replication/thrift/generated/TWalLEdit.java @@ -0,0 +1,439 @@ +/** + * Autogenerated by Thrift Compiler (0.9.1) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hadoop.hbase.replication.thrift.generated; + +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 org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +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; + +/** + * Mapping for WALEdit + * + */ +public class TWalLEdit implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TWalLEdit"); + + private static final org.apache.thrift.protocol.TField MUTATIONS_FIELD_DESC = new org.apache.thrift.protocol.TField("mutations", org.apache.thrift.protocol.TType.LIST, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TWalLEditStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TWalLEditTupleSchemeFactory()); + } + + public List mutations; // 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 { + MUTATIONS((short)1, "mutations"); + + 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: // MUTATIONS + return MUTATIONS; + 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.MUTATIONS, new org.apache.thrift.meta_data.FieldMetaData("mutations", 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, TColumnValue.class)))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TWalLEdit.class, metaDataMap); + } + + public TWalLEdit() { + } + + public TWalLEdit( + List mutations) + { + this(); + this.mutations = mutations; + } + + /** + * Performs a deep copy on other. + */ + public TWalLEdit(TWalLEdit other) { + if (other.isSetMutations()) { + List __this__mutations = new ArrayList(other.mutations.size()); + for (TColumnValue other_element : other.mutations) { + __this__mutations.add(new TColumnValue(other_element)); + } + this.mutations = __this__mutations; + } + } + + public TWalLEdit deepCopy() { + return new TWalLEdit(this); + } + + @Override + public void clear() { + this.mutations = null; + } + + public int getMutationsSize() { + return (this.mutations == null) ? 0 : this.mutations.size(); + } + + public java.util.Iterator getMutationsIterator() { + return (this.mutations == null) ? null : this.mutations.iterator(); + } + + public void addToMutations(TColumnValue elem) { + if (this.mutations == null) { + this.mutations = new ArrayList(); + } + this.mutations.add(elem); + } + + public List getMutations() { + return this.mutations; + } + + public TWalLEdit setMutations(List mutations) { + this.mutations = mutations; + return this; + } + + public void unsetMutations() { + this.mutations = null; + } + + /** Returns true if field mutations is set (has been assigned a value) and false otherwise */ + public boolean isSetMutations() { + return this.mutations != null; + } + + public void setMutationsIsSet(boolean value) { + if (!value) { + this.mutations = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case MUTATIONS: + if (value == null) { + unsetMutations(); + } else { + setMutations((List)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case MUTATIONS: + return getMutations(); + + } + 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 MUTATIONS: + return isSetMutations(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TWalLEdit) + return this.equals((TWalLEdit)that); + return false; + } + + public boolean equals(TWalLEdit that) { + if (that == null) + return false; + + boolean this_present_mutations = true && this.isSetMutations(); + boolean that_present_mutations = true && that.isSetMutations(); + if (this_present_mutations || that_present_mutations) { + if (!(this_present_mutations && that_present_mutations)) + return false; + if (!this.mutations.equals(that.mutations)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + return 0; + } + + @Override + public int compareTo(TWalLEdit other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(isSetMutations()).compareTo(other.isSetMutations()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetMutations()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.mutations, other.mutations); + 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("TWalLEdit("); + boolean first = true; + + sb.append("mutations:"); + if (this.mutations == null) { + sb.append("null"); + } else { + sb.append(this.mutations); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (mutations == null) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'mutations' was not present! 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 TWalLEditStandardSchemeFactory implements SchemeFactory { + public TWalLEditStandardScheme getScheme() { + return new TWalLEditStandardScheme(); + } + } + + private static class TWalLEditStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TWalLEdit 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: // MUTATIONS + if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { + { + org.apache.thrift.protocol.TList _list0 = iprot.readListBegin(); + struct.mutations = new ArrayList(_list0.size); + for (int _i1 = 0; _i1 < _list0.size; ++_i1) + { + TColumnValue _elem2; + _elem2 = new TColumnValue(); + _elem2.read(iprot); + struct.mutations.add(_elem2); + } + iprot.readListEnd(); + } + struct.setMutationsIsSet(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(); + + // check for required fields of primitive type, which can't be checked in the validate method + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TWalLEdit struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.mutations != null) { + oprot.writeFieldBegin(MUTATIONS_FIELD_DESC); + { + oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.mutations.size())); + for (TColumnValue _iter3 : struct.mutations) + { + _iter3.write(oprot); + } + oprot.writeListEnd(); + } + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TWalLEditTupleSchemeFactory implements SchemeFactory { + public TWalLEditTupleScheme getScheme() { + return new TWalLEditTupleScheme(); + } + } + + private static class TWalLEditTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TWalLEdit struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + { + oprot.writeI32(struct.mutations.size()); + for (TColumnValue _iter4 : struct.mutations) + { + _iter4.write(oprot); + } + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TWalLEdit struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + { + org.apache.thrift.protocol.TList _list5 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); + struct.mutations = new ArrayList(_list5.size); + for (int _i6 = 0; _i6 < _list5.size; ++_i6) + { + TColumnValue _elem7; + _elem7 = new TColumnValue(); + _elem7.read(iprot); + struct.mutations.add(_elem7); + } + } + struct.setMutationsIsSet(true); + } + } + +} + diff --git a/src/main/java/org/apache/hadoop/hbase/thrift/ThriftServerRunner.java b/src/main/java/org/apache/hadoop/hbase/thrift/ThriftServerRunner.java index f8317d3..a6fd3ac 100644 --- a/src/main/java/org/apache/hadoop/hbase/thrift/ThriftServerRunner.java +++ b/src/main/java/org/apache/hadoop/hbase/thrift/ThriftServerRunner.java @@ -115,9 +115,6 @@ public class ThriftServerRunner implements Runnable { static final String COMPACT_CONF_KEY = "hbase.regionserver.thrift.compact"; static final String FRAMED_CONF_KEY = "hbase.regionserver.thrift.framed"; static final String PORT_CONF_KEY = "hbase.regionserver.thrift.port"; - //The max length of an individual thrift message and frames in MB - static final String MAX_MESSAGE_LENGTH_CONF_KEY = "hbase.regionserver.thrift.binary.max_message_length_in_mb"; - static final String MAX_FRAME_SIZE_CONF_KEY = "hbase.regionserver.thrift.framed.max_frame_size_in_mb"; static final String COALESCE_INC_KEY = "hbase.regionserver.thrift.coalesceIncrement"; private static final String DEFAULT_BIND_ADDR = "0.0.0.0"; @@ -205,7 +202,7 @@ public class ThriftServerRunner implements Runnable { chosenType = DEFAULT; } else if (numChosen > 1) { throw new AssertionError("Exactly one option out of " + - Arrays.toString(values()) + " has to be specified"); + Arrays.toString(values()) + " has to be specified"); } LOG.info("Using thrift server type " + chosenType.option); conf.set(SERVER_TYPE_CONF_KEY, chosenType.option); @@ -272,8 +269,7 @@ public class ThriftServerRunner implements Runnable { protocolFactory = new TCompactProtocol.Factory(); } else { LOG.debug("Using binary protocol"); - int maxMessageLength = conf.getInt(MAX_MESSAGE_LENGTH_CONF_KEY, 2) * 1024 * 1024; - protocolFactory = new TBinaryProtocol.Factory(false, true, maxMessageLength); + protocolFactory = new TBinaryProtocol.Factory(); } Hbase.Processor processor = @@ -283,8 +279,7 @@ public class ThriftServerRunner implements Runnable { // Construct correct TransportFactory TTransportFactory transportFactory; if (conf.getBoolean(FRAMED_CONF_KEY, false) || implType.isAlwaysFramed) { - int maxFrameSize = conf.getInt(MAX_FRAME_SIZE_CONF_KEY, 2) * 1024 * 1024; - transportFactory = new TFramedTransport.Factory(maxFrameSize); + transportFactory = new TFramedTransport.Factory(); LOG.debug("Using framed transport"); } else { transportFactory = new TTransportFactory(); @@ -309,8 +304,8 @@ public class ThriftServerRunner implements Runnable { TNonblockingServer.Args serverArgs = new TNonblockingServer.Args(serverTransport); serverArgs.processor(processor) - .transportFactory(transportFactory) - .protocolFactory(protocolFactory); + .transportFactory(transportFactory) + .protocolFactory(protocolFactory); tserver = new TNonblockingServer(serverArgs); } else if (implType == ImplType.HS_HA) { THsHaServer.Args serverArgs = new THsHaServer.Args(serverTransport); @@ -319,9 +314,9 @@ public class ThriftServerRunner implements Runnable { ExecutorService executorService = createExecutor( callQueue, serverArgs.getWorkerThreads()); serverArgs.executorService(executorService) - .processor(processor) - .transportFactory(transportFactory) - .protocolFactory(protocolFactory); + .processor(processor) + .transportFactory(transportFactory) + .protocolFactory(protocolFactory); tserver = new THsHaServer(serverArgs); } else { // THREADED_SELECTOR TThreadedSelectorServer.Args serverArgs = @@ -331,9 +326,9 @@ public class ThriftServerRunner implements Runnable { ExecutorService executorService = createExecutor( callQueue, serverArgs.getWorkerThreads()); serverArgs.executorService(executorService) - .processor(processor) - .transportFactory(transportFactory) - .protocolFactory(protocolFactory); + .processor(processor) + .transportFactory(transportFactory) + .protocolFactory(protocolFactory); tserver = new TThreadedSelectorServer(serverArgs); } LOG.info("starting HBase " + implType.simpleClassName() + @@ -348,8 +343,8 @@ public class ThriftServerRunner implements Runnable { TBoundedThreadPoolServer.Args serverArgs = new TBoundedThreadPoolServer.Args(serverTransport, conf); serverArgs.processor(processor) - .transportFactory(transportFactory) - .protocolFactory(protocolFactory); + .transportFactory(transportFactory) + .protocolFactory(protocolFactory); LOG.info("starting " + ImplType.THREAD_POOL.simpleClassName() + " on " + listenAddress + ":" + Integer.toString(listenPort) + "; " + serverArgs); @@ -372,12 +367,12 @@ public class ThriftServerRunner implements Runnable { } ExecutorService createExecutor(BlockingQueue callQueue, - int workerThreads) { + int workerThreads) { ThreadFactoryBuilder tfb = new ThreadFactoryBuilder(); tfb.setDaemon(true); tfb.setNameFormat("thrift-worker-%d"); return new ThreadPoolExecutor(workerThreads, workerThreads, - Long.MAX_VALUE, TimeUnit.SECONDS, callQueue, tfb.build()); + Long.MAX_VALUE, TimeUnit.SECONDS, callQueue, tfb.build()); } private InetAddress getBindAddress(Configuration conf) @@ -386,25 +381,6 @@ public class ThriftServerRunner implements Runnable { return InetAddress.getByName(bindAddressStr); } - protected static class ResultScannerWrapper { - - private final ResultScanner scanner; - private final boolean sortColumns; - public ResultScannerWrapper(ResultScanner resultScanner, - boolean sortResultColumns) { - scanner = resultScanner; - sortColumns = sortResultColumns; - } - - public ResultScanner getScanner() { - return scanner; - } - - public boolean isColumnSorted() { - return sortColumns; - } - } - /** * The HBaseHandler is a glue object that connects Thrift RPC calls to the * HBase client API primarily defined in the HBaseAdmin and HTable objects. @@ -416,16 +392,16 @@ public class ThriftServerRunner implements Runnable { // nextScannerId and scannerMap are used to manage scanner state protected int nextScannerId = 0; - protected HashMap scannerMap = null; + protected HashMap scannerMap = null; private ThriftMetrics metrics = null; private static ThreadLocal> threadLocalTables = new ThreadLocal>() { - @Override - protected Map initialValue() { - return new TreeMap(); - } - }; + @Override + protected Map initialValue() { + return new TreeMap(); + } + }; IncrementCoalescer coalescer = null; @@ -476,10 +452,9 @@ public class ThriftServerRunner implements Runnable { * @param scanner * @return integer scanner id */ - protected synchronized int addScanner(ResultScanner scanner,boolean sortColumns) { + protected synchronized int addScanner(ResultScanner scanner) { int id = nextScannerId++; - ResultScannerWrapper resultScannerWrapper = new ResultScannerWrapper(scanner, sortColumns); - scannerMap.put(id, resultScannerWrapper); + scannerMap.put(id, scanner); return id; } @@ -489,7 +464,7 @@ public class ThriftServerRunner implements Runnable { * @param id * @return a Scanner, or null if ID was invalid. */ - protected synchronized ResultScannerWrapper getScanner(int id) { + protected synchronized ResultScanner getScanner(int id) { return scannerMap.get(id); } @@ -500,7 +475,7 @@ public class ThriftServerRunner implements Runnable { * @param id * @return a Scanner, or null if ID was invalid. */ - protected synchronized ResultScannerWrapper removeScanner(int id) { + protected synchronized ResultScanner removeScanner(int id) { return scannerMap.remove(id); } @@ -509,14 +484,14 @@ public class ThriftServerRunner implements Runnable { * @throws IOException */ protected HBaseHandler() - throws IOException { + throws IOException { this(HBaseConfiguration.create()); } protected HBaseHandler(final Configuration c) throws IOException { this.conf = c; admin = new HBaseAdmin(conf); - scannerMap = new HashMap(); + scannerMap = new HashMap(); this.coalescer = new IncrementCoalescer(this); } @@ -578,10 +553,10 @@ public class ThriftServerRunner implements Runnable { @Override public List getTableNames() throws IOError { try { - String[] tableNames = this.admin.getTableNames(); - ArrayList list = new ArrayList(tableNames.length); - for (int i = 0; i < tableNames.length; i++) { - list.add(ByteBuffer.wrap(Bytes.toBytes(tableNames[i]))); + HTableDescriptor[] tables = this.admin.listTables(); + ArrayList list = new ArrayList(tables.length); + for (int i = 0; i < tables.length; i++) { + list.add(ByteBuffer.wrap(tables[i].getName())); } return list; } catch (IOException e) { @@ -592,7 +567,7 @@ public class ThriftServerRunner implements Runnable { @Override public List getTableRegions(ByteBuffer tableName) - throws IOError { + throws IOError { try { HTable table = getTable(tableName); Map regionLocations = @@ -649,10 +624,10 @@ public class ThriftServerRunner implements Runnable { } protected List get(ByteBuffer tableName, - ByteBuffer row, - byte[] family, - byte[] qualifier, - Map attributes) throws IOError { + ByteBuffer row, + byte[] family, + byte[] qualifier, + Map attributes) throws IOError { try { HTable table = getTable(tableName); Get get = new Get(getBytes(row)); @@ -685,7 +660,7 @@ public class ThriftServerRunner implements Runnable { } public List getVer(ByteBuffer tableName, ByteBuffer row, - byte[] family, + byte[] family, byte[] qualifier, int numVersions, Map attributes) throws IOError { try { @@ -705,7 +680,7 @@ public class ThriftServerRunner implements Runnable { @Deprecated @Override public List getVerTs(ByteBuffer tableName, - ByteBuffer row, + ByteBuffer row, ByteBuffer column, long timestamp, int numVersions, @@ -720,7 +695,7 @@ public class ThriftServerRunner implements Runnable { } protected List getVerTs(ByteBuffer tableName, - ByteBuffer row, byte [] family, + ByteBuffer row, byte [] family, byte [] qualifier, long timestamp, int numVersions, Map attributes) throws IOError { try { @@ -742,25 +717,25 @@ public class ThriftServerRunner implements Runnable { public List getRow(ByteBuffer tableName, ByteBuffer row, Map attributes) throws IOError { return getRowWithColumnsTs(tableName, row, null, - HConstants.LATEST_TIMESTAMP, - attributes); + HConstants.LATEST_TIMESTAMP, + attributes); } @Override public List getRowWithColumns(ByteBuffer tableName, - ByteBuffer row, + ByteBuffer row, List columns, Map attributes) throws IOError { return getRowWithColumnsTs(tableName, row, columns, - HConstants.LATEST_TIMESTAMP, - attributes); + HConstants.LATEST_TIMESTAMP, + attributes); } @Override public List getRowTs(ByteBuffer tableName, ByteBuffer row, long timestamp, Map attributes) throws IOError { return getRowWithColumnsTs(tableName, row, null, - timestamp, attributes); + timestamp, attributes); } @Override @@ -781,9 +756,9 @@ public class ThriftServerRunner implements Runnable { for(ByteBuffer column : columns) { byte [][] famAndQf = KeyValue.parseColumn(getBytes(column)); if (famAndQf.length == 1) { - get.addFamily(famAndQf[0]); + get.addFamily(famAndQf[0]); } else { - get.addColumn(famAndQf[0], famAndQf[1]); + get.addColumn(famAndQf[0], famAndQf[1]); } } get.setTimeRange(Long.MIN_VALUE, timestamp); @@ -797,36 +772,36 @@ public class ThriftServerRunner implements Runnable { @Override public List getRows(ByteBuffer tableName, - List rows, + List rows, Map attributes) throws IOError { return getRowsWithColumnsTs(tableName, rows, null, - HConstants.LATEST_TIMESTAMP, - attributes); + HConstants.LATEST_TIMESTAMP, + attributes); } @Override public List getRowsWithColumns(ByteBuffer tableName, - List rows, + List rows, List columns, Map attributes) throws IOError { return getRowsWithColumnsTs(tableName, rows, columns, - HConstants.LATEST_TIMESTAMP, - attributes); + HConstants.LATEST_TIMESTAMP, + attributes); } @Override public List getRowsTs(ByteBuffer tableName, - List rows, + List rows, long timestamp, Map attributes) throws IOError { return getRowsWithColumnsTs(tableName, rows, null, - timestamp, attributes); + timestamp, attributes); } @Override public List getRowsWithColumnsTs(ByteBuffer tableName, - List rows, + List rows, List columns, long timestamp, Map attributes) throws IOError { try { @@ -866,13 +841,13 @@ public class ThriftServerRunner implements Runnable { Map attributes) throws IOError { deleteAllTs(tableName, row, column, HConstants.LATEST_TIMESTAMP, - attributes); + attributes); } @Override public void deleteAllTs(ByteBuffer tableName, - ByteBuffer row, - ByteBuffer column, + ByteBuffer row, + ByteBuffer column, long timestamp, Map attributes) throws IOError { try { HTable table = getTable(tableName); @@ -960,7 +935,7 @@ public class ThriftServerRunner implements Runnable { List mutations, Map attributes) throws IOError, IllegalArgument { mutateRowTs(tableName, row, mutations, HConstants.LATEST_TIMESTAMP, - attributes); + attributes); } @Override @@ -1086,7 +1061,7 @@ public class ThriftServerRunner implements Runnable { @Override public long atomicIncrement( ByteBuffer tableName, ByteBuffer row, ByteBuffer column, long amount) - throws IOError, IllegalArgument, TException { + throws IOError, IllegalArgument, TException { byte [][] famAndQf = KeyValue.parseColumn(getBytes(column)); if(famAndQf.length == 1) { return atomicIncrement(tableName, row, famAndQf[0], new byte[0], @@ -1111,13 +1086,13 @@ public class ThriftServerRunner implements Runnable { public void scannerClose(int id) throws IOError, IllegalArgument { LOG.debug("scannerClose: id=" + id); - ResultScannerWrapper resultScannerWrapper = getScanner(id); - if (resultScannerWrapper == null) { + ResultScanner scanner = getScanner(id); + if (scanner == null) { String message = "scanner ID is invalid"; LOG.warn(message); throw new IllegalArgument("scanner ID is invalid"); } - resultScannerWrapper.getScanner().close(); + scanner.close(); removeScanner(id); } @@ -1125,8 +1100,8 @@ public class ThriftServerRunner implements Runnable { public List scannerGetList(int id,int nbRows) throws IllegalArgument, IOError { LOG.debug("scannerGetList: id=" + id); - ResultScannerWrapper resultScannerWrapper = getScanner(id); - if (null == resultScannerWrapper) { + ResultScanner scanner = getScanner(id); + if (null == scanner) { String message = "scanner ID is invalid"; LOG.warn(message); throw new IllegalArgument("scanner ID is invalid"); @@ -1134,7 +1109,7 @@ public class ThriftServerRunner implements Runnable { Result [] results = null; try { - results = resultScannerWrapper.getScanner().next(nbRows); + results = scanner.next(nbRows); if (null == results) { return new ArrayList(); } @@ -1142,7 +1117,7 @@ public class ThriftServerRunner implements Runnable { LOG.warn(e.getMessage(), e); throw new IOError(e.getMessage()); } - return ThriftUtilities.rowResultFromHBase(results, resultScannerWrapper.isColumnSorted()); + return ThriftUtilities.rowResultFromHBase(results); } @Override @@ -1184,7 +1159,7 @@ public class ThriftServerRunner implements Runnable { scan.setFilter( parseFilter.parseFilterString(tScan.getFilterString())); } - return addScanner(table.getScanner(scan), tScan.sortColumns); + return addScanner(table.getScanner(scan)); } catch (IOException e) { LOG.warn(e.getMessage(), e); throw new IOError(e.getMessage()); @@ -1209,7 +1184,7 @@ public class ThriftServerRunner implements Runnable { } } } - return addScanner(table.getScanner(scan), false); + return addScanner(table.getScanner(scan)); } catch (IOException e) { LOG.warn(e.getMessage(), e); throw new IOError(e.getMessage()); @@ -1235,7 +1210,7 @@ public class ThriftServerRunner implements Runnable { } } } - return addScanner(table.getScanner(scan), false); + return addScanner(table.getScanner(scan)); } catch (IOException e) { LOG.warn(e.getMessage(), e); throw new IOError(e.getMessage()); @@ -1244,8 +1219,8 @@ public class ThriftServerRunner implements Runnable { @Override public int scannerOpenWithPrefix(ByteBuffer tableName, - ByteBuffer startAndPrefix, - List columns, + ByteBuffer startAndPrefix, + List columns, Map attributes) throws IOError, TException { try { @@ -1265,7 +1240,7 @@ public class ThriftServerRunner implements Runnable { } } } - return addScanner(table.getScanner(scan), false); + return addScanner(table.getScanner(scan)); } catch (IOException e) { LOG.warn(e.getMessage(), e); throw new IOError(e.getMessage()); @@ -1291,7 +1266,7 @@ public class ThriftServerRunner implements Runnable { } } } - return addScanner(table.getScanner(scan), false); + return addScanner(table.getScanner(scan)); } catch (IOException e) { LOG.warn(e.getMessage(), e); throw new IOError(e.getMessage()); @@ -1319,7 +1294,7 @@ public class ThriftServerRunner implements Runnable { } } scan.setTimeRange(Long.MIN_VALUE, timestamp); - return addScanner(table.getScanner(scan), false); + return addScanner(table.getScanner(scan)); } catch (IOException e) { LOG.warn(e.getMessage(), e); throw new IOError(e.getMessage()); @@ -1331,7 +1306,7 @@ public class ThriftServerRunner implements Runnable { ByteBuffer tableName) throws IOError, TException { try { TreeMap columns = - new TreeMap(); + new TreeMap(); HTable table = getTable(tableName); HTableDescriptor desc = table.getTableDescriptor(); @@ -1366,20 +1341,20 @@ public class ThriftServerRunner implements Runnable { HTable table = getTable(HConstants.META_TABLE_NAME); byte[] row = toBytes(searchRow); Result startRowResult = table.getRowOrBefore( - row, HConstants.CATALOG_FAMILY); + row, HConstants.CATALOG_FAMILY); if (startRowResult == null) { - throw new IOException("Cannot find row in .META., row=" - + Bytes.toString(searchRow.array())); + throw new IOException("Cannot find row in "+Bytes.toString(HConstants.META_TABLE_NAME)+", row=" + + Bytes.toString(searchRow.array())); } // find region start and end keys byte[] value = startRowResult.getValue(HConstants.CATALOG_FAMILY, - HConstants.REGIONINFO_QUALIFIER); + HConstants.REGIONINFO_QUALIFIER); if (value == null || value.length == 0) { throw new IOException("HRegionInfo REGIONINFO was null or " + - " empty in Meta for row=" - + Bytes.toString(row)); + " empty in Meta for row=" + + Bytes.toString(row)); } HRegionInfo regionInfo = Writables.getHRegionInfo(value); TRegionInfo region = new TRegionInfo(); @@ -1391,7 +1366,7 @@ public class ThriftServerRunner implements Runnable { // find region assignment to server value = startRowResult.getValue(HConstants.CATALOG_FAMILY, - HConstants.SERVER_QUALIFIER); + HConstants.SERVER_QUALIFIER); if (value != null && value.length > 0) { String hostAndPort = Bytes.toString(value); region.setServerName(Bytes.toBytes( @@ -1449,7 +1424,7 @@ public class ThriftServerRunner implements Runnable { * Adds all the attributes into the Operation object */ private static void addAttributes(OperationWithAttributes op, - Map attributes) { + Map attributes) { if (attributes == null || attributes.size() == 0) { return; } diff --git a/src/main/java/org/apache/hadoop/hbase/thrift/generated/AlreadyExists.java b/src/main/java/org/apache/hadoop/hbase/thrift/generated/AlreadyExists.java index a5b81f5..f0f1f27 100644 --- a/src/main/java/org/apache/hadoop/hbase/thrift/generated/AlreadyExists.java +++ b/src/main/java/org/apache/hadoop/hbase/thrift/generated/AlreadyExists.java @@ -1,5 +1,5 @@ /** - * Autogenerated by Thrift Compiler (0.8.0) + * Autogenerated by Thrift Compiler (0.9.1) * * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING * @generated @@ -12,6 +12,11 @@ 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 org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; import java.util.List; import java.util.ArrayList; import java.util.Map; @@ -31,7 +36,7 @@ import org.slf4j.LoggerFactory; * An AlreadyExists exceptions signals that a table with the specified * name already exists */ -public class AlreadyExists extends Exception implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { +public class AlreadyExists extends TException implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("AlreadyExists"); 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); @@ -48,6 +53,7 @@ public class AlreadyExists extends Exception implements org.apache.thrift.TBase< public enum _Fields implements org.apache.thrift.TFieldIdEnum { MESSAGE((short)1, "message"); + private static final Map byName = new HashMap(); static { @@ -229,20 +235,20 @@ public class AlreadyExists extends Exception implements org.apache.thrift.TBase< return 0; } + @Override public int compareTo(AlreadyExists other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; - AlreadyExists typedOther = (AlreadyExists)other; - lastComparison = Boolean.valueOf(isSetMessage()).compareTo(typedOther.isSetMessage()); + lastComparison = Boolean.valueOf(isSetMessage()).compareTo(other.isSetMessage()); if (lastComparison != 0) { return lastComparison; } if (isSetMessage()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.message, typedOther.message); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.message, other.message); if (lastComparison != 0) { return lastComparison; } @@ -280,6 +286,7 @@ public class AlreadyExists extends Exception implements org.apache.thrift.TBase< 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 { diff --git a/src/main/java/org/apache/hadoop/hbase/thrift/generated/BatchMutation.java b/src/main/java/org/apache/hadoop/hbase/thrift/generated/BatchMutation.java index d5df940..5f1d01c 100644 --- a/src/main/java/org/apache/hadoop/hbase/thrift/generated/BatchMutation.java +++ b/src/main/java/org/apache/hadoop/hbase/thrift/generated/BatchMutation.java @@ -1,5 +1,5 @@ /** - * Autogenerated by Thrift Compiler (0.8.0) + * Autogenerated by Thrift Compiler (0.9.1) * * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING * @generated @@ -12,6 +12,11 @@ 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 org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; import java.util.List; import java.util.ArrayList; import java.util.Map; @@ -30,7 +35,7 @@ import org.slf4j.LoggerFactory; /** * A BatchMutation object is used to apply a number of Mutations to a single row. */ -public class BatchMutation implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { +public class BatchMutation implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("BatchMutation"); private static final org.apache.thrift.protocol.TField ROW_FIELD_DESC = new org.apache.thrift.protocol.TField("row", org.apache.thrift.protocol.TType.STRING, (short)1); @@ -139,7 +144,7 @@ public class BatchMutation implements org.apache.thrift.TBase __this__mutations = new ArrayList(); + List __this__mutations = new ArrayList(other.mutations.size()); for (Mutation other_element : other.mutations) { __this__mutations.add(new Mutation(other_element)); } @@ -317,30 +322,30 @@ public class BatchMutation implements org.apache.thrift.TBase(_list0.size); for (int _i1 = 0; _i1 < _list0.size; ++_i1) { - Mutation _elem2; // optional + Mutation _elem2; _elem2 = new Mutation(); _elem2.read(iprot); struct.mutations.add(_elem2); @@ -534,7 +540,7 @@ public class BatchMutation implements org.apache.thrift.TBase(_list5.size); for (int _i6 = 0; _i6 < _list5.size; ++_i6) { - Mutation _elem7; // optional + Mutation _elem7; _elem7 = new Mutation(); _elem7.read(iprot); struct.mutations.add(_elem7); diff --git a/src/main/java/org/apache/hadoop/hbase/thrift/generated/ColumnDescriptor.java b/src/main/java/org/apache/hadoop/hbase/thrift/generated/ColumnDescriptor.java index 4ce85e7..a659ff4 100644 --- a/src/main/java/org/apache/hadoop/hbase/thrift/generated/ColumnDescriptor.java +++ b/src/main/java/org/apache/hadoop/hbase/thrift/generated/ColumnDescriptor.java @@ -1,5 +1,5 @@ /** - * Autogenerated by Thrift Compiler (0.8.0) + * Autogenerated by Thrift Compiler (0.9.1) * * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING * @generated @@ -12,6 +12,11 @@ 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 org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; import java.util.List; import java.util.ArrayList; import java.util.Map; @@ -32,7 +37,7 @@ import org.slf4j.LoggerFactory; * such as the number of versions, compression settings, etc. It is * used as input when creating a table or adding a column. */ -public class ColumnDescriptor implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { +public class ColumnDescriptor implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("ColumnDescriptor"); private static final org.apache.thrift.protocol.TField NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("name", org.apache.thrift.protocol.TType.STRING, (short)1); @@ -150,7 +155,7 @@ public class ColumnDescriptor implements org.apache.thrift.TBase metaDataMap; static { Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); @@ -228,8 +233,7 @@ public class ColumnDescriptor implements org.apache.thrift.TBaseother. */ public ColumnDescriptor(ColumnDescriptor other) { - __isset_bit_vector.clear(); - __isset_bit_vector.or(other.__isset_bit_vector); + __isset_bitfield = other.__isset_bitfield; if (other.isSetName()) { this.name = other.name; } @@ -317,16 +321,16 @@ public class ColumnDescriptor implements org.apache.thrift.TBase resultHandler) throws org.apache.thrift.TException; + public void enableTable(ByteBuffer tableName, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; - public void disableTable(ByteBuffer tableName, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + public void disableTable(ByteBuffer tableName, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; - public void isTableEnabled(ByteBuffer tableName, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + public void isTableEnabled(ByteBuffer tableName, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; - public void compact(ByteBuffer tableNameOrRegionName, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + public void compact(ByteBuffer tableNameOrRegionName, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; - public void majorCompact(ByteBuffer tableNameOrRegionName, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + public void majorCompact(ByteBuffer tableNameOrRegionName, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; - public void getTableNames(org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + public void getTableNames(org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; - public void getColumnDescriptors(ByteBuffer tableName, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + public void getColumnDescriptors(ByteBuffer tableName, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; - public void getTableRegions(ByteBuffer tableName, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + public void getTableRegions(ByteBuffer tableName, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; - public void createTable(ByteBuffer tableName, List columnFamilies, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + public void createTable(ByteBuffer tableName, List columnFamilies, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; - public void deleteTable(ByteBuffer tableName, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + public void deleteTable(ByteBuffer tableName, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; - public void get(ByteBuffer tableName, ByteBuffer row, ByteBuffer column, Map attributes, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + public void get(ByteBuffer tableName, ByteBuffer row, ByteBuffer column, Map attributes, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; - public void getVer(ByteBuffer tableName, ByteBuffer row, ByteBuffer column, int numVersions, Map attributes, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + public void getVer(ByteBuffer tableName, ByteBuffer row, ByteBuffer column, int numVersions, Map attributes, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; - public void getVerTs(ByteBuffer tableName, ByteBuffer row, ByteBuffer column, long timestamp, int numVersions, Map attributes, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + public void getVerTs(ByteBuffer tableName, ByteBuffer row, ByteBuffer column, long timestamp, int numVersions, Map attributes, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; - public void getRow(ByteBuffer tableName, ByteBuffer row, Map attributes, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + public void getRow(ByteBuffer tableName, ByteBuffer row, Map attributes, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; - public void getRowWithColumns(ByteBuffer tableName, ByteBuffer row, List columns, Map attributes, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + public void getRowWithColumns(ByteBuffer tableName, ByteBuffer row, List columns, Map attributes, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; - public void getRowTs(ByteBuffer tableName, ByteBuffer row, long timestamp, Map attributes, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + public void getRowTs(ByteBuffer tableName, ByteBuffer row, long timestamp, Map attributes, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; - public void getRowWithColumnsTs(ByteBuffer tableName, ByteBuffer row, List columns, long timestamp, Map attributes, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + public void getRowWithColumnsTs(ByteBuffer tableName, ByteBuffer row, List columns, long timestamp, Map attributes, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; - public void getRows(ByteBuffer tableName, List rows, Map attributes, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + public void getRows(ByteBuffer tableName, List rows, Map attributes, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; - public void getRowsWithColumns(ByteBuffer tableName, List rows, List columns, Map attributes, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + public void getRowsWithColumns(ByteBuffer tableName, List rows, List columns, Map attributes, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; - public void getRowsTs(ByteBuffer tableName, List rows, long timestamp, Map attributes, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + public void getRowsTs(ByteBuffer tableName, List rows, long timestamp, Map attributes, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; - public void getRowsWithColumnsTs(ByteBuffer tableName, List rows, List columns, long timestamp, Map attributes, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + public void getRowsWithColumnsTs(ByteBuffer tableName, List rows, List columns, long timestamp, Map attributes, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; - public void mutateRow(ByteBuffer tableName, ByteBuffer row, List mutations, Map attributes, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + public void mutateRow(ByteBuffer tableName, ByteBuffer row, List mutations, Map attributes, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; - public void mutateRowTs(ByteBuffer tableName, ByteBuffer row, List mutations, long timestamp, Map attributes, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + public void mutateRowTs(ByteBuffer tableName, ByteBuffer row, List mutations, long timestamp, Map attributes, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; - public void mutateRows(ByteBuffer tableName, List rowBatches, Map attributes, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + public void mutateRows(ByteBuffer tableName, List rowBatches, Map attributes, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; - public void mutateRowsTs(ByteBuffer tableName, List rowBatches, long timestamp, Map attributes, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + public void mutateRowsTs(ByteBuffer tableName, List rowBatches, long timestamp, Map attributes, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; - public void atomicIncrement(ByteBuffer tableName, ByteBuffer row, ByteBuffer column, long value, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + public void atomicIncrement(ByteBuffer tableName, ByteBuffer row, ByteBuffer column, long value, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; - public void deleteAll(ByteBuffer tableName, ByteBuffer row, ByteBuffer column, Map attributes, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + public void deleteAll(ByteBuffer tableName, ByteBuffer row, ByteBuffer column, Map attributes, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; - public void deleteAllTs(ByteBuffer tableName, ByteBuffer row, ByteBuffer column, long timestamp, Map attributes, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + public void deleteAllTs(ByteBuffer tableName, ByteBuffer row, ByteBuffer column, long timestamp, Map attributes, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; - public void deleteAllRow(ByteBuffer tableName, ByteBuffer row, Map attributes, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + public void deleteAllRow(ByteBuffer tableName, ByteBuffer row, Map attributes, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; - public void increment(TIncrement increment, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + public void increment(TIncrement increment, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; - public void incrementRows(List increments, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + public void incrementRows(List increments, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; - public void deleteAllRowTs(ByteBuffer tableName, ByteBuffer row, long timestamp, Map attributes, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + public void deleteAllRowTs(ByteBuffer tableName, ByteBuffer row, long timestamp, Map attributes, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; - public void scannerOpenWithScan(ByteBuffer tableName, TScan scan, Map attributes, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + public void scannerOpenWithScan(ByteBuffer tableName, TScan scan, Map attributes, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; - public void scannerOpen(ByteBuffer tableName, ByteBuffer startRow, List columns, Map attributes, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + public void scannerOpen(ByteBuffer tableName, ByteBuffer startRow, List columns, Map attributes, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; - public void scannerOpenWithStop(ByteBuffer tableName, ByteBuffer startRow, ByteBuffer stopRow, List columns, Map attributes, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + public void scannerOpenWithStop(ByteBuffer tableName, ByteBuffer startRow, ByteBuffer stopRow, List columns, Map attributes, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; - public void scannerOpenWithPrefix(ByteBuffer tableName, ByteBuffer startAndPrefix, List columns, Map attributes, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + public void scannerOpenWithPrefix(ByteBuffer tableName, ByteBuffer startAndPrefix, List columns, Map attributes, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; - public void scannerOpenTs(ByteBuffer tableName, ByteBuffer startRow, List columns, long timestamp, Map attributes, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + public void scannerOpenTs(ByteBuffer tableName, ByteBuffer startRow, List columns, long timestamp, Map attributes, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; - public void scannerOpenWithStopTs(ByteBuffer tableName, ByteBuffer startRow, ByteBuffer stopRow, List columns, long timestamp, Map attributes, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + public void scannerOpenWithStopTs(ByteBuffer tableName, ByteBuffer startRow, ByteBuffer stopRow, List columns, long timestamp, Map attributes, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; - public void scannerGet(int id, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + public void scannerGet(int id, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; - public void scannerGetList(int id, int nbRows, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + public void scannerGetList(int id, int nbRows, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; - public void scannerClose(int id, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + public void scannerClose(int id, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; - public void getRowOrBefore(ByteBuffer tableName, ByteBuffer row, ByteBuffer family, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + public void getRowOrBefore(ByteBuffer tableName, ByteBuffer row, ByteBuffer family, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; - public void getRegionInfo(ByteBuffer row, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + public void getRegionInfo(ByteBuffer row, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; } @@ -1930,7 +1935,7 @@ public class Hbase { super(protocolFactory, clientManager, transport); } - public void enableTable(ByteBuffer tableName, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + public void enableTable(ByteBuffer tableName, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { checkReady(); enableTable_call method_call = new enableTable_call(tableName, resultHandler, this, ___protocolFactory, ___transport); this.___currentMethod = method_call; @@ -1939,7 +1944,7 @@ public class Hbase { public static class enableTable_call extends org.apache.thrift.async.TAsyncMethodCall { private ByteBuffer tableName; - public enableTable_call(ByteBuffer tableName, 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 { + public enableTable_call(ByteBuffer tableName, 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.tableName = tableName; } @@ -1962,7 +1967,7 @@ public class Hbase { } } - public void disableTable(ByteBuffer tableName, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + public void disableTable(ByteBuffer tableName, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { checkReady(); disableTable_call method_call = new disableTable_call(tableName, resultHandler, this, ___protocolFactory, ___transport); this.___currentMethod = method_call; @@ -1971,7 +1976,7 @@ public class Hbase { public static class disableTable_call extends org.apache.thrift.async.TAsyncMethodCall { private ByteBuffer tableName; - public disableTable_call(ByteBuffer tableName, 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 { + public disableTable_call(ByteBuffer tableName, 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.tableName = tableName; } @@ -1994,7 +1999,7 @@ public class Hbase { } } - public void isTableEnabled(ByteBuffer tableName, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + public void isTableEnabled(ByteBuffer tableName, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { checkReady(); isTableEnabled_call method_call = new isTableEnabled_call(tableName, resultHandler, this, ___protocolFactory, ___transport); this.___currentMethod = method_call; @@ -2003,7 +2008,7 @@ public class Hbase { public static class isTableEnabled_call extends org.apache.thrift.async.TAsyncMethodCall { private ByteBuffer tableName; - public isTableEnabled_call(ByteBuffer tableName, 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 { + public isTableEnabled_call(ByteBuffer tableName, 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.tableName = tableName; } @@ -2026,7 +2031,7 @@ public class Hbase { } } - public void compact(ByteBuffer tableNameOrRegionName, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + public void compact(ByteBuffer tableNameOrRegionName, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { checkReady(); compact_call method_call = new compact_call(tableNameOrRegionName, resultHandler, this, ___protocolFactory, ___transport); this.___currentMethod = method_call; @@ -2035,7 +2040,7 @@ public class Hbase { public static class compact_call extends org.apache.thrift.async.TAsyncMethodCall { private ByteBuffer tableNameOrRegionName; - public compact_call(ByteBuffer tableNameOrRegionName, 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 { + public compact_call(ByteBuffer tableNameOrRegionName, 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.tableNameOrRegionName = tableNameOrRegionName; } @@ -2058,7 +2063,7 @@ public class Hbase { } } - public void majorCompact(ByteBuffer tableNameOrRegionName, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + public void majorCompact(ByteBuffer tableNameOrRegionName, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { checkReady(); majorCompact_call method_call = new majorCompact_call(tableNameOrRegionName, resultHandler, this, ___protocolFactory, ___transport); this.___currentMethod = method_call; @@ -2067,7 +2072,7 @@ public class Hbase { public static class majorCompact_call extends org.apache.thrift.async.TAsyncMethodCall { private ByteBuffer tableNameOrRegionName; - public majorCompact_call(ByteBuffer tableNameOrRegionName, 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 { + public majorCompact_call(ByteBuffer tableNameOrRegionName, 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.tableNameOrRegionName = tableNameOrRegionName; } @@ -2090,7 +2095,7 @@ public class Hbase { } } - public void getTableNames(org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + public void getTableNames(org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { checkReady(); getTableNames_call method_call = new getTableNames_call(resultHandler, this, ___protocolFactory, ___transport); this.___currentMethod = method_call; @@ -2098,7 +2103,7 @@ public class Hbase { } public static class getTableNames_call extends org.apache.thrift.async.TAsyncMethodCall { - public getTableNames_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 { + public getTableNames_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); } @@ -2119,7 +2124,7 @@ public class Hbase { } } - public void getColumnDescriptors(ByteBuffer tableName, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + public void getColumnDescriptors(ByteBuffer tableName, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { checkReady(); getColumnDescriptors_call method_call = new getColumnDescriptors_call(tableName, resultHandler, this, ___protocolFactory, ___transport); this.___currentMethod = method_call; @@ -2128,7 +2133,7 @@ public class Hbase { public static class getColumnDescriptors_call extends org.apache.thrift.async.TAsyncMethodCall { private ByteBuffer tableName; - public getColumnDescriptors_call(ByteBuffer tableName, 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 { + public getColumnDescriptors_call(ByteBuffer tableName, 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.tableName = tableName; } @@ -2151,7 +2156,7 @@ public class Hbase { } } - public void getTableRegions(ByteBuffer tableName, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + public void getTableRegions(ByteBuffer tableName, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { checkReady(); getTableRegions_call method_call = new getTableRegions_call(tableName, resultHandler, this, ___protocolFactory, ___transport); this.___currentMethod = method_call; @@ -2160,7 +2165,7 @@ public class Hbase { public static class getTableRegions_call extends org.apache.thrift.async.TAsyncMethodCall { private ByteBuffer tableName; - public getTableRegions_call(ByteBuffer tableName, 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 { + public getTableRegions_call(ByteBuffer tableName, 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.tableName = tableName; } @@ -2183,7 +2188,7 @@ public class Hbase { } } - public void createTable(ByteBuffer tableName, List columnFamilies, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + public void createTable(ByteBuffer tableName, List columnFamilies, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { checkReady(); createTable_call method_call = new createTable_call(tableName, columnFamilies, resultHandler, this, ___protocolFactory, ___transport); this.___currentMethod = method_call; @@ -2193,7 +2198,7 @@ public class Hbase { public static class createTable_call extends org.apache.thrift.async.TAsyncMethodCall { private ByteBuffer tableName; private List columnFamilies; - public createTable_call(ByteBuffer tableName, List columnFamilies, 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 { + public createTable_call(ByteBuffer tableName, List columnFamilies, 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.tableName = tableName; this.columnFamilies = columnFamilies; @@ -2218,7 +2223,7 @@ public class Hbase { } } - public void deleteTable(ByteBuffer tableName, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + public void deleteTable(ByteBuffer tableName, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { checkReady(); deleteTable_call method_call = new deleteTable_call(tableName, resultHandler, this, ___protocolFactory, ___transport); this.___currentMethod = method_call; @@ -2227,7 +2232,7 @@ public class Hbase { public static class deleteTable_call extends org.apache.thrift.async.TAsyncMethodCall { private ByteBuffer tableName; - public deleteTable_call(ByteBuffer tableName, 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 { + public deleteTable_call(ByteBuffer tableName, 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.tableName = tableName; } @@ -2250,7 +2255,7 @@ public class Hbase { } } - public void get(ByteBuffer tableName, ByteBuffer row, ByteBuffer column, Map attributes, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + public void get(ByteBuffer tableName, ByteBuffer row, ByteBuffer column, Map attributes, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { checkReady(); get_call method_call = new get_call(tableName, row, column, attributes, resultHandler, this, ___protocolFactory, ___transport); this.___currentMethod = method_call; @@ -2262,7 +2267,7 @@ public class Hbase { private ByteBuffer row; private ByteBuffer column; private Map attributes; - public get_call(ByteBuffer tableName, ByteBuffer row, ByteBuffer column, Map attributes, 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 { + public get_call(ByteBuffer tableName, ByteBuffer row, ByteBuffer column, Map attributes, 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.tableName = tableName; this.row = row; @@ -2291,7 +2296,7 @@ public class Hbase { } } - public void getVer(ByteBuffer tableName, ByteBuffer row, ByteBuffer column, int numVersions, Map attributes, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + public void getVer(ByteBuffer tableName, ByteBuffer row, ByteBuffer column, int numVersions, Map attributes, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { checkReady(); getVer_call method_call = new getVer_call(tableName, row, column, numVersions, attributes, resultHandler, this, ___protocolFactory, ___transport); this.___currentMethod = method_call; @@ -2304,7 +2309,7 @@ public class Hbase { private ByteBuffer column; private int numVersions; private Map attributes; - public getVer_call(ByteBuffer tableName, ByteBuffer row, ByteBuffer column, int numVersions, Map attributes, 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 { + public getVer_call(ByteBuffer tableName, ByteBuffer row, ByteBuffer column, int numVersions, Map attributes, 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.tableName = tableName; this.row = row; @@ -2335,7 +2340,7 @@ public class Hbase { } } - public void getVerTs(ByteBuffer tableName, ByteBuffer row, ByteBuffer column, long timestamp, int numVersions, Map attributes, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + public void getVerTs(ByteBuffer tableName, ByteBuffer row, ByteBuffer column, long timestamp, int numVersions, Map attributes, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { checkReady(); getVerTs_call method_call = new getVerTs_call(tableName, row, column, timestamp, numVersions, attributes, resultHandler, this, ___protocolFactory, ___transport); this.___currentMethod = method_call; @@ -2349,7 +2354,7 @@ public class Hbase { private long timestamp; private int numVersions; private Map attributes; - public getVerTs_call(ByteBuffer tableName, ByteBuffer row, ByteBuffer column, long timestamp, int numVersions, Map attributes, 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 { + public getVerTs_call(ByteBuffer tableName, ByteBuffer row, ByteBuffer column, long timestamp, int numVersions, Map attributes, 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.tableName = tableName; this.row = row; @@ -2382,7 +2387,7 @@ public class Hbase { } } - public void getRow(ByteBuffer tableName, ByteBuffer row, Map attributes, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + public void getRow(ByteBuffer tableName, ByteBuffer row, Map attributes, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { checkReady(); getRow_call method_call = new getRow_call(tableName, row, attributes, resultHandler, this, ___protocolFactory, ___transport); this.___currentMethod = method_call; @@ -2393,7 +2398,7 @@ public class Hbase { private ByteBuffer tableName; private ByteBuffer row; private Map attributes; - public getRow_call(ByteBuffer tableName, ByteBuffer row, Map attributes, 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 { + public getRow_call(ByteBuffer tableName, ByteBuffer row, Map attributes, 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.tableName = tableName; this.row = row; @@ -2420,7 +2425,7 @@ public class Hbase { } } - public void getRowWithColumns(ByteBuffer tableName, ByteBuffer row, List columns, Map attributes, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + public void getRowWithColumns(ByteBuffer tableName, ByteBuffer row, List columns, Map attributes, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { checkReady(); getRowWithColumns_call method_call = new getRowWithColumns_call(tableName, row, columns, attributes, resultHandler, this, ___protocolFactory, ___transport); this.___currentMethod = method_call; @@ -2432,7 +2437,7 @@ public class Hbase { private ByteBuffer row; private List columns; private Map attributes; - public getRowWithColumns_call(ByteBuffer tableName, ByteBuffer row, List columns, Map attributes, 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 { + public getRowWithColumns_call(ByteBuffer tableName, ByteBuffer row, List columns, Map attributes, 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.tableName = tableName; this.row = row; @@ -2461,7 +2466,7 @@ public class Hbase { } } - public void getRowTs(ByteBuffer tableName, ByteBuffer row, long timestamp, Map attributes, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + public void getRowTs(ByteBuffer tableName, ByteBuffer row, long timestamp, Map attributes, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { checkReady(); getRowTs_call method_call = new getRowTs_call(tableName, row, timestamp, attributes, resultHandler, this, ___protocolFactory, ___transport); this.___currentMethod = method_call; @@ -2473,7 +2478,7 @@ public class Hbase { private ByteBuffer row; private long timestamp; private Map attributes; - public getRowTs_call(ByteBuffer tableName, ByteBuffer row, long timestamp, Map attributes, 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 { + public getRowTs_call(ByteBuffer tableName, ByteBuffer row, long timestamp, Map attributes, 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.tableName = tableName; this.row = row; @@ -2502,7 +2507,7 @@ public class Hbase { } } - public void getRowWithColumnsTs(ByteBuffer tableName, ByteBuffer row, List columns, long timestamp, Map attributes, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + public void getRowWithColumnsTs(ByteBuffer tableName, ByteBuffer row, List columns, long timestamp, Map attributes, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { checkReady(); getRowWithColumnsTs_call method_call = new getRowWithColumnsTs_call(tableName, row, columns, timestamp, attributes, resultHandler, this, ___protocolFactory, ___transport); this.___currentMethod = method_call; @@ -2515,7 +2520,7 @@ public class Hbase { private List columns; private long timestamp; private Map attributes; - public getRowWithColumnsTs_call(ByteBuffer tableName, ByteBuffer row, List columns, long timestamp, Map attributes, 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 { + public getRowWithColumnsTs_call(ByteBuffer tableName, ByteBuffer row, List columns, long timestamp, Map attributes, 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.tableName = tableName; this.row = row; @@ -2546,7 +2551,7 @@ public class Hbase { } } - public void getRows(ByteBuffer tableName, List rows, Map attributes, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + public void getRows(ByteBuffer tableName, List rows, Map attributes, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { checkReady(); getRows_call method_call = new getRows_call(tableName, rows, attributes, resultHandler, this, ___protocolFactory, ___transport); this.___currentMethod = method_call; @@ -2557,7 +2562,7 @@ public class Hbase { private ByteBuffer tableName; private List rows; private Map attributes; - public getRows_call(ByteBuffer tableName, List rows, Map attributes, 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 { + public getRows_call(ByteBuffer tableName, List rows, Map attributes, 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.tableName = tableName; this.rows = rows; @@ -2584,7 +2589,7 @@ public class Hbase { } } - public void getRowsWithColumns(ByteBuffer tableName, List rows, List columns, Map attributes, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + public void getRowsWithColumns(ByteBuffer tableName, List rows, List columns, Map attributes, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { checkReady(); getRowsWithColumns_call method_call = new getRowsWithColumns_call(tableName, rows, columns, attributes, resultHandler, this, ___protocolFactory, ___transport); this.___currentMethod = method_call; @@ -2596,7 +2601,7 @@ public class Hbase { private List rows; private List columns; private Map attributes; - public getRowsWithColumns_call(ByteBuffer tableName, List rows, List columns, Map attributes, 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 { + public getRowsWithColumns_call(ByteBuffer tableName, List rows, List columns, Map attributes, 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.tableName = tableName; this.rows = rows; @@ -2625,7 +2630,7 @@ public class Hbase { } } - public void getRowsTs(ByteBuffer tableName, List rows, long timestamp, Map attributes, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + public void getRowsTs(ByteBuffer tableName, List rows, long timestamp, Map attributes, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { checkReady(); getRowsTs_call method_call = new getRowsTs_call(tableName, rows, timestamp, attributes, resultHandler, this, ___protocolFactory, ___transport); this.___currentMethod = method_call; @@ -2637,7 +2642,7 @@ public class Hbase { private List rows; private long timestamp; private Map attributes; - public getRowsTs_call(ByteBuffer tableName, List rows, long timestamp, Map attributes, 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 { + public getRowsTs_call(ByteBuffer tableName, List rows, long timestamp, Map attributes, 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.tableName = tableName; this.rows = rows; @@ -2666,7 +2671,7 @@ public class Hbase { } } - public void getRowsWithColumnsTs(ByteBuffer tableName, List rows, List columns, long timestamp, Map attributes, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + public void getRowsWithColumnsTs(ByteBuffer tableName, List rows, List columns, long timestamp, Map attributes, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { checkReady(); getRowsWithColumnsTs_call method_call = new getRowsWithColumnsTs_call(tableName, rows, columns, timestamp, attributes, resultHandler, this, ___protocolFactory, ___transport); this.___currentMethod = method_call; @@ -2679,7 +2684,7 @@ public class Hbase { private List columns; private long timestamp; private Map attributes; - public getRowsWithColumnsTs_call(ByteBuffer tableName, List rows, List columns, long timestamp, Map attributes, 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 { + public getRowsWithColumnsTs_call(ByteBuffer tableName, List rows, List columns, long timestamp, Map attributes, 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.tableName = tableName; this.rows = rows; @@ -2710,7 +2715,7 @@ public class Hbase { } } - public void mutateRow(ByteBuffer tableName, ByteBuffer row, List mutations, Map attributes, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + public void mutateRow(ByteBuffer tableName, ByteBuffer row, List mutations, Map attributes, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { checkReady(); mutateRow_call method_call = new mutateRow_call(tableName, row, mutations, attributes, resultHandler, this, ___protocolFactory, ___transport); this.___currentMethod = method_call; @@ -2722,7 +2727,7 @@ public class Hbase { private ByteBuffer row; private List mutations; private Map attributes; - public mutateRow_call(ByteBuffer tableName, ByteBuffer row, List mutations, Map attributes, 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 { + public mutateRow_call(ByteBuffer tableName, ByteBuffer row, List mutations, Map attributes, 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.tableName = tableName; this.row = row; @@ -2751,7 +2756,7 @@ public class Hbase { } } - public void mutateRowTs(ByteBuffer tableName, ByteBuffer row, List mutations, long timestamp, Map attributes, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + public void mutateRowTs(ByteBuffer tableName, ByteBuffer row, List mutations, long timestamp, Map attributes, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { checkReady(); mutateRowTs_call method_call = new mutateRowTs_call(tableName, row, mutations, timestamp, attributes, resultHandler, this, ___protocolFactory, ___transport); this.___currentMethod = method_call; @@ -2764,7 +2769,7 @@ public class Hbase { private List mutations; private long timestamp; private Map attributes; - public mutateRowTs_call(ByteBuffer tableName, ByteBuffer row, List mutations, long timestamp, Map attributes, 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 { + public mutateRowTs_call(ByteBuffer tableName, ByteBuffer row, List mutations, long timestamp, Map attributes, 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.tableName = tableName; this.row = row; @@ -2795,7 +2800,7 @@ public class Hbase { } } - public void mutateRows(ByteBuffer tableName, List rowBatches, Map attributes, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + public void mutateRows(ByteBuffer tableName, List rowBatches, Map attributes, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { checkReady(); mutateRows_call method_call = new mutateRows_call(tableName, rowBatches, attributes, resultHandler, this, ___protocolFactory, ___transport); this.___currentMethod = method_call; @@ -2806,7 +2811,7 @@ public class Hbase { private ByteBuffer tableName; private List rowBatches; private Map attributes; - public mutateRows_call(ByteBuffer tableName, List rowBatches, Map attributes, 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 { + public mutateRows_call(ByteBuffer tableName, List rowBatches, Map attributes, 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.tableName = tableName; this.rowBatches = rowBatches; @@ -2833,7 +2838,7 @@ public class Hbase { } } - public void mutateRowsTs(ByteBuffer tableName, List rowBatches, long timestamp, Map attributes, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + public void mutateRowsTs(ByteBuffer tableName, List rowBatches, long timestamp, Map attributes, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { checkReady(); mutateRowsTs_call method_call = new mutateRowsTs_call(tableName, rowBatches, timestamp, attributes, resultHandler, this, ___protocolFactory, ___transport); this.___currentMethod = method_call; @@ -2845,7 +2850,7 @@ public class Hbase { private List rowBatches; private long timestamp; private Map attributes; - public mutateRowsTs_call(ByteBuffer tableName, List rowBatches, long timestamp, Map attributes, 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 { + public mutateRowsTs_call(ByteBuffer tableName, List rowBatches, long timestamp, Map attributes, 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.tableName = tableName; this.rowBatches = rowBatches; @@ -2874,7 +2879,7 @@ public class Hbase { } } - public void atomicIncrement(ByteBuffer tableName, ByteBuffer row, ByteBuffer column, long value, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + public void atomicIncrement(ByteBuffer tableName, ByteBuffer row, ByteBuffer column, long value, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { checkReady(); atomicIncrement_call method_call = new atomicIncrement_call(tableName, row, column, value, resultHandler, this, ___protocolFactory, ___transport); this.___currentMethod = method_call; @@ -2886,7 +2891,7 @@ public class Hbase { private ByteBuffer row; private ByteBuffer column; private long value; - public atomicIncrement_call(ByteBuffer tableName, ByteBuffer row, ByteBuffer column, long value, 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 { + public atomicIncrement_call(ByteBuffer tableName, ByteBuffer row, ByteBuffer column, long value, 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.tableName = tableName; this.row = row; @@ -2915,7 +2920,7 @@ public class Hbase { } } - public void deleteAll(ByteBuffer tableName, ByteBuffer row, ByteBuffer column, Map attributes, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + public void deleteAll(ByteBuffer tableName, ByteBuffer row, ByteBuffer column, Map attributes, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { checkReady(); deleteAll_call method_call = new deleteAll_call(tableName, row, column, attributes, resultHandler, this, ___protocolFactory, ___transport); this.___currentMethod = method_call; @@ -2927,7 +2932,7 @@ public class Hbase { private ByteBuffer row; private ByteBuffer column; private Map attributes; - public deleteAll_call(ByteBuffer tableName, ByteBuffer row, ByteBuffer column, Map attributes, 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 { + public deleteAll_call(ByteBuffer tableName, ByteBuffer row, ByteBuffer column, Map attributes, 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.tableName = tableName; this.row = row; @@ -2956,7 +2961,7 @@ public class Hbase { } } - public void deleteAllTs(ByteBuffer tableName, ByteBuffer row, ByteBuffer column, long timestamp, Map attributes, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + public void deleteAllTs(ByteBuffer tableName, ByteBuffer row, ByteBuffer column, long timestamp, Map attributes, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { checkReady(); deleteAllTs_call method_call = new deleteAllTs_call(tableName, row, column, timestamp, attributes, resultHandler, this, ___protocolFactory, ___transport); this.___currentMethod = method_call; @@ -2969,7 +2974,7 @@ public class Hbase { private ByteBuffer column; private long timestamp; private Map attributes; - public deleteAllTs_call(ByteBuffer tableName, ByteBuffer row, ByteBuffer column, long timestamp, Map attributes, 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 { + public deleteAllTs_call(ByteBuffer tableName, ByteBuffer row, ByteBuffer column, long timestamp, Map attributes, 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.tableName = tableName; this.row = row; @@ -3000,7 +3005,7 @@ public class Hbase { } } - public void deleteAllRow(ByteBuffer tableName, ByteBuffer row, Map attributes, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + public void deleteAllRow(ByteBuffer tableName, ByteBuffer row, Map attributes, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { checkReady(); deleteAllRow_call method_call = new deleteAllRow_call(tableName, row, attributes, resultHandler, this, ___protocolFactory, ___transport); this.___currentMethod = method_call; @@ -3011,7 +3016,7 @@ public class Hbase { private ByteBuffer tableName; private ByteBuffer row; private Map attributes; - public deleteAllRow_call(ByteBuffer tableName, ByteBuffer row, Map attributes, 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 { + public deleteAllRow_call(ByteBuffer tableName, ByteBuffer row, Map attributes, 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.tableName = tableName; this.row = row; @@ -3038,7 +3043,7 @@ public class Hbase { } } - public void increment(TIncrement increment, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + public void increment(TIncrement increment, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { checkReady(); increment_call method_call = new increment_call(increment, resultHandler, this, ___protocolFactory, ___transport); this.___currentMethod = method_call; @@ -3047,7 +3052,7 @@ public class Hbase { public static class increment_call extends org.apache.thrift.async.TAsyncMethodCall { private TIncrement increment; - public increment_call(TIncrement increment, 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 { + public increment_call(TIncrement increment, 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.increment = increment; } @@ -3070,7 +3075,7 @@ public class Hbase { } } - public void incrementRows(List increments, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + public void incrementRows(List increments, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { checkReady(); incrementRows_call method_call = new incrementRows_call(increments, resultHandler, this, ___protocolFactory, ___transport); this.___currentMethod = method_call; @@ -3079,7 +3084,7 @@ public class Hbase { public static class incrementRows_call extends org.apache.thrift.async.TAsyncMethodCall { private List increments; - public incrementRows_call(List increments, 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 { + public incrementRows_call(List increments, 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.increments = increments; } @@ -3102,7 +3107,7 @@ public class Hbase { } } - public void deleteAllRowTs(ByteBuffer tableName, ByteBuffer row, long timestamp, Map attributes, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + public void deleteAllRowTs(ByteBuffer tableName, ByteBuffer row, long timestamp, Map attributes, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { checkReady(); deleteAllRowTs_call method_call = new deleteAllRowTs_call(tableName, row, timestamp, attributes, resultHandler, this, ___protocolFactory, ___transport); this.___currentMethod = method_call; @@ -3114,7 +3119,7 @@ public class Hbase { private ByteBuffer row; private long timestamp; private Map attributes; - public deleteAllRowTs_call(ByteBuffer tableName, ByteBuffer row, long timestamp, Map attributes, 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 { + public deleteAllRowTs_call(ByteBuffer tableName, ByteBuffer row, long timestamp, Map attributes, 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.tableName = tableName; this.row = row; @@ -3143,7 +3148,7 @@ public class Hbase { } } - public void scannerOpenWithScan(ByteBuffer tableName, TScan scan, Map attributes, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + public void scannerOpenWithScan(ByteBuffer tableName, TScan scan, Map attributes, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { checkReady(); scannerOpenWithScan_call method_call = new scannerOpenWithScan_call(tableName, scan, attributes, resultHandler, this, ___protocolFactory, ___transport); this.___currentMethod = method_call; @@ -3154,7 +3159,7 @@ public class Hbase { private ByteBuffer tableName; private TScan scan; private Map attributes; - public scannerOpenWithScan_call(ByteBuffer tableName, TScan scan, Map attributes, 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 { + public scannerOpenWithScan_call(ByteBuffer tableName, TScan scan, Map attributes, 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.tableName = tableName; this.scan = scan; @@ -3181,7 +3186,7 @@ public class Hbase { } } - public void scannerOpen(ByteBuffer tableName, ByteBuffer startRow, List columns, Map attributes, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + public void scannerOpen(ByteBuffer tableName, ByteBuffer startRow, List columns, Map attributes, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { checkReady(); scannerOpen_call method_call = new scannerOpen_call(tableName, startRow, columns, attributes, resultHandler, this, ___protocolFactory, ___transport); this.___currentMethod = method_call; @@ -3193,7 +3198,7 @@ public class Hbase { private ByteBuffer startRow; private List columns; private Map attributes; - public scannerOpen_call(ByteBuffer tableName, ByteBuffer startRow, List columns, Map attributes, 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 { + public scannerOpen_call(ByteBuffer tableName, ByteBuffer startRow, List columns, Map attributes, 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.tableName = tableName; this.startRow = startRow; @@ -3222,7 +3227,7 @@ public class Hbase { } } - public void scannerOpenWithStop(ByteBuffer tableName, ByteBuffer startRow, ByteBuffer stopRow, List columns, Map attributes, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + public void scannerOpenWithStop(ByteBuffer tableName, ByteBuffer startRow, ByteBuffer stopRow, List columns, Map attributes, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { checkReady(); scannerOpenWithStop_call method_call = new scannerOpenWithStop_call(tableName, startRow, stopRow, columns, attributes, resultHandler, this, ___protocolFactory, ___transport); this.___currentMethod = method_call; @@ -3235,7 +3240,7 @@ public class Hbase { private ByteBuffer stopRow; private List columns; private Map attributes; - public scannerOpenWithStop_call(ByteBuffer tableName, ByteBuffer startRow, ByteBuffer stopRow, List columns, Map attributes, 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 { + public scannerOpenWithStop_call(ByteBuffer tableName, ByteBuffer startRow, ByteBuffer stopRow, List columns, Map attributes, 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.tableName = tableName; this.startRow = startRow; @@ -3266,7 +3271,7 @@ public class Hbase { } } - public void scannerOpenWithPrefix(ByteBuffer tableName, ByteBuffer startAndPrefix, List columns, Map attributes, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + public void scannerOpenWithPrefix(ByteBuffer tableName, ByteBuffer startAndPrefix, List columns, Map attributes, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { checkReady(); scannerOpenWithPrefix_call method_call = new scannerOpenWithPrefix_call(tableName, startAndPrefix, columns, attributes, resultHandler, this, ___protocolFactory, ___transport); this.___currentMethod = method_call; @@ -3278,7 +3283,7 @@ public class Hbase { private ByteBuffer startAndPrefix; private List columns; private Map attributes; - public scannerOpenWithPrefix_call(ByteBuffer tableName, ByteBuffer startAndPrefix, List columns, Map attributes, 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 { + public scannerOpenWithPrefix_call(ByteBuffer tableName, ByteBuffer startAndPrefix, List columns, Map attributes, 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.tableName = tableName; this.startAndPrefix = startAndPrefix; @@ -3307,7 +3312,7 @@ public class Hbase { } } - public void scannerOpenTs(ByteBuffer tableName, ByteBuffer startRow, List columns, long timestamp, Map attributes, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + public void scannerOpenTs(ByteBuffer tableName, ByteBuffer startRow, List columns, long timestamp, Map attributes, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { checkReady(); scannerOpenTs_call method_call = new scannerOpenTs_call(tableName, startRow, columns, timestamp, attributes, resultHandler, this, ___protocolFactory, ___transport); this.___currentMethod = method_call; @@ -3320,7 +3325,7 @@ public class Hbase { private List columns; private long timestamp; private Map attributes; - public scannerOpenTs_call(ByteBuffer tableName, ByteBuffer startRow, List columns, long timestamp, Map attributes, 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 { + public scannerOpenTs_call(ByteBuffer tableName, ByteBuffer startRow, List columns, long timestamp, Map attributes, 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.tableName = tableName; this.startRow = startRow; @@ -3351,7 +3356,7 @@ public class Hbase { } } - public void scannerOpenWithStopTs(ByteBuffer tableName, ByteBuffer startRow, ByteBuffer stopRow, List columns, long timestamp, Map attributes, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + public void scannerOpenWithStopTs(ByteBuffer tableName, ByteBuffer startRow, ByteBuffer stopRow, List columns, long timestamp, Map attributes, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { checkReady(); scannerOpenWithStopTs_call method_call = new scannerOpenWithStopTs_call(tableName, startRow, stopRow, columns, timestamp, attributes, resultHandler, this, ___protocolFactory, ___transport); this.___currentMethod = method_call; @@ -3365,7 +3370,7 @@ public class Hbase { private List columns; private long timestamp; private Map attributes; - public scannerOpenWithStopTs_call(ByteBuffer tableName, ByteBuffer startRow, ByteBuffer stopRow, List columns, long timestamp, Map attributes, 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 { + public scannerOpenWithStopTs_call(ByteBuffer tableName, ByteBuffer startRow, ByteBuffer stopRow, List columns, long timestamp, Map attributes, 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.tableName = tableName; this.startRow = startRow; @@ -3398,7 +3403,7 @@ public class Hbase { } } - public void scannerGet(int id, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + public void scannerGet(int id, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { checkReady(); scannerGet_call method_call = new scannerGet_call(id, resultHandler, this, ___protocolFactory, ___transport); this.___currentMethod = method_call; @@ -3407,7 +3412,7 @@ public class Hbase { public static class scannerGet_call extends org.apache.thrift.async.TAsyncMethodCall { private int id; - public scannerGet_call(int id, 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 { + public scannerGet_call(int id, 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.id = id; } @@ -3430,7 +3435,7 @@ public class Hbase { } } - public void scannerGetList(int id, int nbRows, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + public void scannerGetList(int id, int nbRows, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { checkReady(); scannerGetList_call method_call = new scannerGetList_call(id, nbRows, resultHandler, this, ___protocolFactory, ___transport); this.___currentMethod = method_call; @@ -3440,7 +3445,7 @@ public class Hbase { public static class scannerGetList_call extends org.apache.thrift.async.TAsyncMethodCall { private int id; private int nbRows; - public scannerGetList_call(int id, int nbRows, 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 { + public scannerGetList_call(int id, int nbRows, 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.id = id; this.nbRows = nbRows; @@ -3465,7 +3470,7 @@ public class Hbase { } } - public void scannerClose(int id, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + public void scannerClose(int id, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { checkReady(); scannerClose_call method_call = new scannerClose_call(id, resultHandler, this, ___protocolFactory, ___transport); this.___currentMethod = method_call; @@ -3474,7 +3479,7 @@ public class Hbase { public static class scannerClose_call extends org.apache.thrift.async.TAsyncMethodCall { private int id; - public scannerClose_call(int id, 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 { + public scannerClose_call(int id, 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.id = id; } @@ -3497,7 +3502,7 @@ public class Hbase { } } - public void getRowOrBefore(ByteBuffer tableName, ByteBuffer row, ByteBuffer family, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + public void getRowOrBefore(ByteBuffer tableName, ByteBuffer row, ByteBuffer family, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { checkReady(); getRowOrBefore_call method_call = new getRowOrBefore_call(tableName, row, family, resultHandler, this, ___protocolFactory, ___transport); this.___currentMethod = method_call; @@ -3508,7 +3513,7 @@ public class Hbase { private ByteBuffer tableName; private ByteBuffer row; private ByteBuffer family; - public getRowOrBefore_call(ByteBuffer tableName, ByteBuffer row, ByteBuffer family, 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 { + public getRowOrBefore_call(ByteBuffer tableName, ByteBuffer row, ByteBuffer family, 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.tableName = tableName; this.row = row; @@ -3535,7 +3540,7 @@ public class Hbase { } } - public void getRegionInfo(ByteBuffer row, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + public void getRegionInfo(ByteBuffer row, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { checkReady(); getRegionInfo_call method_call = new getRegionInfo_call(row, resultHandler, this, ___protocolFactory, ___transport); this.___currentMethod = method_call; @@ -3544,7 +3549,7 @@ public class Hbase { public static class getRegionInfo_call extends org.apache.thrift.async.TAsyncMethodCall { private ByteBuffer row; - public getRegionInfo_call(ByteBuffer row, 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 { + public getRegionInfo_call(ByteBuffer row, 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.row = row; } @@ -3626,16 +3631,20 @@ public class Hbase { return processMap; } - private static class enableTable extends org.apache.thrift.ProcessFunction { + public static class enableTable extends org.apache.thrift.ProcessFunction { public enableTable() { super("enableTable"); } - protected enableTable_args getEmptyArgsInstance() { + public enableTable_args getEmptyArgsInstance() { return new enableTable_args(); } - protected enableTable_result getResult(I iface, enableTable_args args) throws org.apache.thrift.TException { + protected boolean isOneway() { + return false; + } + + public enableTable_result getResult(I iface, enableTable_args args) throws org.apache.thrift.TException { enableTable_result result = new enableTable_result(); try { iface.enableTable(args.tableName); @@ -3646,16 +3655,20 @@ public class Hbase { } } - private static class disableTable extends org.apache.thrift.ProcessFunction { + public static class disableTable extends org.apache.thrift.ProcessFunction { public disableTable() { super("disableTable"); } - protected disableTable_args getEmptyArgsInstance() { + public disableTable_args getEmptyArgsInstance() { return new disableTable_args(); } - protected disableTable_result getResult(I iface, disableTable_args args) throws org.apache.thrift.TException { + protected boolean isOneway() { + return false; + } + + public disableTable_result getResult(I iface, disableTable_args args) throws org.apache.thrift.TException { disableTable_result result = new disableTable_result(); try { iface.disableTable(args.tableName); @@ -3666,16 +3679,20 @@ public class Hbase { } } - private static class isTableEnabled extends org.apache.thrift.ProcessFunction { + public static class isTableEnabled extends org.apache.thrift.ProcessFunction { public isTableEnabled() { super("isTableEnabled"); } - protected isTableEnabled_args getEmptyArgsInstance() { + public isTableEnabled_args getEmptyArgsInstance() { return new isTableEnabled_args(); } - protected isTableEnabled_result getResult(I iface, isTableEnabled_args args) throws org.apache.thrift.TException { + protected boolean isOneway() { + return false; + } + + public isTableEnabled_result getResult(I iface, isTableEnabled_args args) throws org.apache.thrift.TException { isTableEnabled_result result = new isTableEnabled_result(); try { result.success = iface.isTableEnabled(args.tableName); @@ -3687,16 +3704,20 @@ public class Hbase { } } - private static class compact extends org.apache.thrift.ProcessFunction { + public static class compact extends org.apache.thrift.ProcessFunction { public compact() { super("compact"); } - protected compact_args getEmptyArgsInstance() { + public compact_args getEmptyArgsInstance() { return new compact_args(); } - protected compact_result getResult(I iface, compact_args args) throws org.apache.thrift.TException { + 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(); try { iface.compact(args.tableNameOrRegionName); @@ -3707,16 +3728,20 @@ public class Hbase { } } - private static class majorCompact extends org.apache.thrift.ProcessFunction { + public static class majorCompact extends org.apache.thrift.ProcessFunction { public majorCompact() { super("majorCompact"); } - protected majorCompact_args getEmptyArgsInstance() { + public majorCompact_args getEmptyArgsInstance() { return new majorCompact_args(); } - protected majorCompact_result getResult(I iface, majorCompact_args args) throws org.apache.thrift.TException { + protected boolean isOneway() { + return false; + } + + public majorCompact_result getResult(I iface, majorCompact_args args) throws org.apache.thrift.TException { majorCompact_result result = new majorCompact_result(); try { iface.majorCompact(args.tableNameOrRegionName); @@ -3727,16 +3752,20 @@ public class Hbase { } } - private static class getTableNames extends org.apache.thrift.ProcessFunction { + public static class getTableNames extends org.apache.thrift.ProcessFunction { public getTableNames() { super("getTableNames"); } - protected getTableNames_args getEmptyArgsInstance() { + public getTableNames_args getEmptyArgsInstance() { return new getTableNames_args(); } - protected getTableNames_result getResult(I iface, getTableNames_args args) throws org.apache.thrift.TException { + protected boolean isOneway() { + return false; + } + + public getTableNames_result getResult(I iface, getTableNames_args args) throws org.apache.thrift.TException { getTableNames_result result = new getTableNames_result(); try { result.success = iface.getTableNames(); @@ -3747,16 +3776,20 @@ public class Hbase { } } - private static class getColumnDescriptors extends org.apache.thrift.ProcessFunction { + public static class getColumnDescriptors extends org.apache.thrift.ProcessFunction { public getColumnDescriptors() { super("getColumnDescriptors"); } - protected getColumnDescriptors_args getEmptyArgsInstance() { + public getColumnDescriptors_args getEmptyArgsInstance() { return new getColumnDescriptors_args(); } - protected getColumnDescriptors_result getResult(I iface, getColumnDescriptors_args args) throws org.apache.thrift.TException { + protected boolean isOneway() { + return false; + } + + public getColumnDescriptors_result getResult(I iface, getColumnDescriptors_args args) throws org.apache.thrift.TException { getColumnDescriptors_result result = new getColumnDescriptors_result(); try { result.success = iface.getColumnDescriptors(args.tableName); @@ -3767,16 +3800,20 @@ public class Hbase { } } - private static class getTableRegions extends org.apache.thrift.ProcessFunction { + public static class getTableRegions extends org.apache.thrift.ProcessFunction { public getTableRegions() { super("getTableRegions"); } - protected getTableRegions_args getEmptyArgsInstance() { + public getTableRegions_args getEmptyArgsInstance() { return new getTableRegions_args(); } - protected getTableRegions_result getResult(I iface, getTableRegions_args args) throws org.apache.thrift.TException { + protected boolean isOneway() { + return false; + } + + public getTableRegions_result getResult(I iface, getTableRegions_args args) throws org.apache.thrift.TException { getTableRegions_result result = new getTableRegions_result(); try { result.success = iface.getTableRegions(args.tableName); @@ -3787,16 +3824,20 @@ public class Hbase { } } - private static class createTable extends org.apache.thrift.ProcessFunction { + public static class createTable extends org.apache.thrift.ProcessFunction { public createTable() { super("createTable"); } - protected createTable_args getEmptyArgsInstance() { + public createTable_args getEmptyArgsInstance() { return new createTable_args(); } - protected createTable_result getResult(I iface, createTable_args args) throws org.apache.thrift.TException { + protected boolean isOneway() { + return false; + } + + public createTable_result getResult(I iface, createTable_args args) throws org.apache.thrift.TException { createTable_result result = new createTable_result(); try { iface.createTable(args.tableName, args.columnFamilies); @@ -3811,16 +3852,20 @@ public class Hbase { } } - private static class deleteTable extends org.apache.thrift.ProcessFunction { + public static class deleteTable extends org.apache.thrift.ProcessFunction { public deleteTable() { super("deleteTable"); } - protected deleteTable_args getEmptyArgsInstance() { + public deleteTable_args getEmptyArgsInstance() { return new deleteTable_args(); } - protected deleteTable_result getResult(I iface, deleteTable_args args) throws org.apache.thrift.TException { + protected boolean isOneway() { + return false; + } + + public deleteTable_result getResult(I iface, deleteTable_args args) throws org.apache.thrift.TException { deleteTable_result result = new deleteTable_result(); try { iface.deleteTable(args.tableName); @@ -3831,16 +3876,20 @@ public class Hbase { } } - private static class get extends org.apache.thrift.ProcessFunction { + public static class get extends org.apache.thrift.ProcessFunction { public get() { super("get"); } - protected get_args getEmptyArgsInstance() { + public get_args getEmptyArgsInstance() { return new get_args(); } - protected get_result getResult(I iface, get_args args) throws org.apache.thrift.TException { + protected boolean isOneway() { + return false; + } + + public get_result getResult(I iface, get_args args) throws org.apache.thrift.TException { get_result result = new get_result(); try { result.success = iface.get(args.tableName, args.row, args.column, args.attributes); @@ -3851,16 +3900,20 @@ public class Hbase { } } - private static class getVer extends org.apache.thrift.ProcessFunction { + public static class getVer extends org.apache.thrift.ProcessFunction { public getVer() { super("getVer"); } - protected getVer_args getEmptyArgsInstance() { + public getVer_args getEmptyArgsInstance() { return new getVer_args(); } - protected getVer_result getResult(I iface, getVer_args args) throws org.apache.thrift.TException { + protected boolean isOneway() { + return false; + } + + public getVer_result getResult(I iface, getVer_args args) throws org.apache.thrift.TException { getVer_result result = new getVer_result(); try { result.success = iface.getVer(args.tableName, args.row, args.column, args.numVersions, args.attributes); @@ -3871,16 +3924,20 @@ public class Hbase { } } - private static class getVerTs extends org.apache.thrift.ProcessFunction { + public static class getVerTs extends org.apache.thrift.ProcessFunction { public getVerTs() { super("getVerTs"); } - protected getVerTs_args getEmptyArgsInstance() { + public getVerTs_args getEmptyArgsInstance() { return new getVerTs_args(); } - protected getVerTs_result getResult(I iface, getVerTs_args args) throws org.apache.thrift.TException { + protected boolean isOneway() { + return false; + } + + public getVerTs_result getResult(I iface, getVerTs_args args) throws org.apache.thrift.TException { getVerTs_result result = new getVerTs_result(); try { result.success = iface.getVerTs(args.tableName, args.row, args.column, args.timestamp, args.numVersions, args.attributes); @@ -3891,16 +3948,20 @@ public class Hbase { } } - private static class getRow extends org.apache.thrift.ProcessFunction { + public static class getRow extends org.apache.thrift.ProcessFunction { public getRow() { super("getRow"); } - protected getRow_args getEmptyArgsInstance() { + public getRow_args getEmptyArgsInstance() { return new getRow_args(); } - protected getRow_result getResult(I iface, getRow_args args) throws org.apache.thrift.TException { + protected boolean isOneway() { + return false; + } + + public getRow_result getResult(I iface, getRow_args args) throws org.apache.thrift.TException { getRow_result result = new getRow_result(); try { result.success = iface.getRow(args.tableName, args.row, args.attributes); @@ -3911,16 +3972,20 @@ public class Hbase { } } - private static class getRowWithColumns extends org.apache.thrift.ProcessFunction { + public static class getRowWithColumns extends org.apache.thrift.ProcessFunction { public getRowWithColumns() { super("getRowWithColumns"); } - protected getRowWithColumns_args getEmptyArgsInstance() { + public getRowWithColumns_args getEmptyArgsInstance() { return new getRowWithColumns_args(); } - protected getRowWithColumns_result getResult(I iface, getRowWithColumns_args args) throws org.apache.thrift.TException { + protected boolean isOneway() { + return false; + } + + public getRowWithColumns_result getResult(I iface, getRowWithColumns_args args) throws org.apache.thrift.TException { getRowWithColumns_result result = new getRowWithColumns_result(); try { result.success = iface.getRowWithColumns(args.tableName, args.row, args.columns, args.attributes); @@ -3931,16 +3996,20 @@ public class Hbase { } } - private static class getRowTs extends org.apache.thrift.ProcessFunction { + public static class getRowTs extends org.apache.thrift.ProcessFunction { public getRowTs() { super("getRowTs"); } - protected getRowTs_args getEmptyArgsInstance() { + public getRowTs_args getEmptyArgsInstance() { return new getRowTs_args(); } - protected getRowTs_result getResult(I iface, getRowTs_args args) throws org.apache.thrift.TException { + protected boolean isOneway() { + return false; + } + + public getRowTs_result getResult(I iface, getRowTs_args args) throws org.apache.thrift.TException { getRowTs_result result = new getRowTs_result(); try { result.success = iface.getRowTs(args.tableName, args.row, args.timestamp, args.attributes); @@ -3951,16 +4020,20 @@ public class Hbase { } } - private static class getRowWithColumnsTs extends org.apache.thrift.ProcessFunction { + public static class getRowWithColumnsTs extends org.apache.thrift.ProcessFunction { public getRowWithColumnsTs() { super("getRowWithColumnsTs"); } - protected getRowWithColumnsTs_args getEmptyArgsInstance() { + public getRowWithColumnsTs_args getEmptyArgsInstance() { return new getRowWithColumnsTs_args(); } - protected getRowWithColumnsTs_result getResult(I iface, getRowWithColumnsTs_args args) throws org.apache.thrift.TException { + protected boolean isOneway() { + return false; + } + + public getRowWithColumnsTs_result getResult(I iface, getRowWithColumnsTs_args args) throws org.apache.thrift.TException { getRowWithColumnsTs_result result = new getRowWithColumnsTs_result(); try { result.success = iface.getRowWithColumnsTs(args.tableName, args.row, args.columns, args.timestamp, args.attributes); @@ -3971,16 +4044,20 @@ public class Hbase { } } - private static class getRows extends org.apache.thrift.ProcessFunction { + public static class getRows extends org.apache.thrift.ProcessFunction { public getRows() { super("getRows"); } - protected getRows_args getEmptyArgsInstance() { + public getRows_args getEmptyArgsInstance() { return new getRows_args(); } - protected getRows_result getResult(I iface, getRows_args args) throws org.apache.thrift.TException { + protected boolean isOneway() { + return false; + } + + public getRows_result getResult(I iface, getRows_args args) throws org.apache.thrift.TException { getRows_result result = new getRows_result(); try { result.success = iface.getRows(args.tableName, args.rows, args.attributes); @@ -3991,16 +4068,20 @@ public class Hbase { } } - private static class getRowsWithColumns extends org.apache.thrift.ProcessFunction { + public static class getRowsWithColumns extends org.apache.thrift.ProcessFunction { public getRowsWithColumns() { super("getRowsWithColumns"); } - protected getRowsWithColumns_args getEmptyArgsInstance() { + public getRowsWithColumns_args getEmptyArgsInstance() { return new getRowsWithColumns_args(); } - protected getRowsWithColumns_result getResult(I iface, getRowsWithColumns_args args) throws org.apache.thrift.TException { + protected boolean isOneway() { + return false; + } + + public getRowsWithColumns_result getResult(I iface, getRowsWithColumns_args args) throws org.apache.thrift.TException { getRowsWithColumns_result result = new getRowsWithColumns_result(); try { result.success = iface.getRowsWithColumns(args.tableName, args.rows, args.columns, args.attributes); @@ -4011,16 +4092,20 @@ public class Hbase { } } - private static class getRowsTs extends org.apache.thrift.ProcessFunction { + public static class getRowsTs extends org.apache.thrift.ProcessFunction { public getRowsTs() { super("getRowsTs"); } - protected getRowsTs_args getEmptyArgsInstance() { + public getRowsTs_args getEmptyArgsInstance() { return new getRowsTs_args(); } - protected getRowsTs_result getResult(I iface, getRowsTs_args args) throws org.apache.thrift.TException { + protected boolean isOneway() { + return false; + } + + public getRowsTs_result getResult(I iface, getRowsTs_args args) throws org.apache.thrift.TException { getRowsTs_result result = new getRowsTs_result(); try { result.success = iface.getRowsTs(args.tableName, args.rows, args.timestamp, args.attributes); @@ -4031,16 +4116,20 @@ public class Hbase { } } - private static class getRowsWithColumnsTs extends org.apache.thrift.ProcessFunction { + public static class getRowsWithColumnsTs extends org.apache.thrift.ProcessFunction { public getRowsWithColumnsTs() { super("getRowsWithColumnsTs"); } - protected getRowsWithColumnsTs_args getEmptyArgsInstance() { + public getRowsWithColumnsTs_args getEmptyArgsInstance() { return new getRowsWithColumnsTs_args(); } - protected getRowsWithColumnsTs_result getResult(I iface, getRowsWithColumnsTs_args args) throws org.apache.thrift.TException { + protected boolean isOneway() { + return false; + } + + public getRowsWithColumnsTs_result getResult(I iface, getRowsWithColumnsTs_args args) throws org.apache.thrift.TException { getRowsWithColumnsTs_result result = new getRowsWithColumnsTs_result(); try { result.success = iface.getRowsWithColumnsTs(args.tableName, args.rows, args.columns, args.timestamp, args.attributes); @@ -4051,16 +4140,20 @@ public class Hbase { } } - private static class mutateRow extends org.apache.thrift.ProcessFunction { + public static class mutateRow extends org.apache.thrift.ProcessFunction { public mutateRow() { super("mutateRow"); } - protected mutateRow_args getEmptyArgsInstance() { + public mutateRow_args getEmptyArgsInstance() { return new mutateRow_args(); } - protected mutateRow_result getResult(I iface, mutateRow_args args) throws org.apache.thrift.TException { + protected boolean isOneway() { + return false; + } + + public mutateRow_result getResult(I iface, mutateRow_args args) throws org.apache.thrift.TException { mutateRow_result result = new mutateRow_result(); try { iface.mutateRow(args.tableName, args.row, args.mutations, args.attributes); @@ -4073,16 +4166,20 @@ public class Hbase { } } - private static class mutateRowTs extends org.apache.thrift.ProcessFunction { + public static class mutateRowTs extends org.apache.thrift.ProcessFunction { public mutateRowTs() { super("mutateRowTs"); } - protected mutateRowTs_args getEmptyArgsInstance() { + public mutateRowTs_args getEmptyArgsInstance() { return new mutateRowTs_args(); } - protected mutateRowTs_result getResult(I iface, mutateRowTs_args args) throws org.apache.thrift.TException { + protected boolean isOneway() { + return false; + } + + public mutateRowTs_result getResult(I iface, mutateRowTs_args args) throws org.apache.thrift.TException { mutateRowTs_result result = new mutateRowTs_result(); try { iface.mutateRowTs(args.tableName, args.row, args.mutations, args.timestamp, args.attributes); @@ -4095,16 +4192,20 @@ public class Hbase { } } - private static class mutateRows extends org.apache.thrift.ProcessFunction { + public static class mutateRows extends org.apache.thrift.ProcessFunction { public mutateRows() { super("mutateRows"); } - protected mutateRows_args getEmptyArgsInstance() { + public mutateRows_args getEmptyArgsInstance() { return new mutateRows_args(); } - protected mutateRows_result getResult(I iface, mutateRows_args args) throws org.apache.thrift.TException { + protected boolean isOneway() { + return false; + } + + public mutateRows_result getResult(I iface, mutateRows_args args) throws org.apache.thrift.TException { mutateRows_result result = new mutateRows_result(); try { iface.mutateRows(args.tableName, args.rowBatches, args.attributes); @@ -4117,16 +4218,20 @@ public class Hbase { } } - private static class mutateRowsTs extends org.apache.thrift.ProcessFunction { + public static class mutateRowsTs extends org.apache.thrift.ProcessFunction { public mutateRowsTs() { super("mutateRowsTs"); } - protected mutateRowsTs_args getEmptyArgsInstance() { + public mutateRowsTs_args getEmptyArgsInstance() { return new mutateRowsTs_args(); } - protected mutateRowsTs_result getResult(I iface, mutateRowsTs_args args) throws org.apache.thrift.TException { + protected boolean isOneway() { + return false; + } + + public mutateRowsTs_result getResult(I iface, mutateRowsTs_args args) throws org.apache.thrift.TException { mutateRowsTs_result result = new mutateRowsTs_result(); try { iface.mutateRowsTs(args.tableName, args.rowBatches, args.timestamp, args.attributes); @@ -4139,16 +4244,20 @@ public class Hbase { } } - private static class atomicIncrement extends org.apache.thrift.ProcessFunction { + public static class atomicIncrement extends org.apache.thrift.ProcessFunction { public atomicIncrement() { super("atomicIncrement"); } - protected atomicIncrement_args getEmptyArgsInstance() { + public atomicIncrement_args getEmptyArgsInstance() { return new atomicIncrement_args(); } - protected atomicIncrement_result getResult(I iface, atomicIncrement_args args) throws org.apache.thrift.TException { + protected boolean isOneway() { + return false; + } + + public atomicIncrement_result getResult(I iface, atomicIncrement_args args) throws org.apache.thrift.TException { atomicIncrement_result result = new atomicIncrement_result(); try { result.success = iface.atomicIncrement(args.tableName, args.row, args.column, args.value); @@ -4162,16 +4271,20 @@ public class Hbase { } } - private static class deleteAll extends org.apache.thrift.ProcessFunction { + public static class deleteAll extends org.apache.thrift.ProcessFunction { public deleteAll() { super("deleteAll"); } - protected deleteAll_args getEmptyArgsInstance() { + public deleteAll_args getEmptyArgsInstance() { return new deleteAll_args(); } - protected deleteAll_result getResult(I iface, deleteAll_args args) throws org.apache.thrift.TException { + protected boolean isOneway() { + return false; + } + + public deleteAll_result getResult(I iface, deleteAll_args args) throws org.apache.thrift.TException { deleteAll_result result = new deleteAll_result(); try { iface.deleteAll(args.tableName, args.row, args.column, args.attributes); @@ -4182,16 +4295,20 @@ public class Hbase { } } - private static class deleteAllTs extends org.apache.thrift.ProcessFunction { + public static class deleteAllTs extends org.apache.thrift.ProcessFunction { public deleteAllTs() { super("deleteAllTs"); } - protected deleteAllTs_args getEmptyArgsInstance() { + public deleteAllTs_args getEmptyArgsInstance() { return new deleteAllTs_args(); } - protected deleteAllTs_result getResult(I iface, deleteAllTs_args args) throws org.apache.thrift.TException { + protected boolean isOneway() { + return false; + } + + public deleteAllTs_result getResult(I iface, deleteAllTs_args args) throws org.apache.thrift.TException { deleteAllTs_result result = new deleteAllTs_result(); try { iface.deleteAllTs(args.tableName, args.row, args.column, args.timestamp, args.attributes); @@ -4202,321 +4319,2932 @@ public class Hbase { } } - private static class deleteAllRow extends org.apache.thrift.ProcessFunction { + public static class deleteAllRow extends org.apache.thrift.ProcessFunction { public deleteAllRow() { super("deleteAllRow"); } - protected deleteAllRow_args getEmptyArgsInstance() { + public deleteAllRow_args getEmptyArgsInstance() { return new deleteAllRow_args(); } - protected deleteAllRow_result getResult(I iface, deleteAllRow_args args) throws org.apache.thrift.TException { - deleteAllRow_result result = new deleteAllRow_result(); - try { - iface.deleteAllRow(args.tableName, args.row, args.attributes); - } catch (IOError io) { - result.io = io; - } - return result; + protected boolean isOneway() { + return false; + } + + public deleteAllRow_result getResult(I iface, deleteAllRow_args args) throws org.apache.thrift.TException { + deleteAllRow_result result = new deleteAllRow_result(); + try { + iface.deleteAllRow(args.tableName, args.row, args.attributes); + } catch (IOError io) { + result.io = io; + } + return result; + } + } + + public static class increment extends org.apache.thrift.ProcessFunction { + public increment() { + super("increment"); + } + + public increment_args getEmptyArgsInstance() { + return new increment_args(); + } + + protected boolean isOneway() { + return false; + } + + public increment_result getResult(I iface, increment_args args) throws org.apache.thrift.TException { + increment_result result = new increment_result(); + try { + iface.increment(args.increment); + } catch (IOError io) { + result.io = io; + } + return result; + } + } + + public static class incrementRows extends org.apache.thrift.ProcessFunction { + public incrementRows() { + super("incrementRows"); + } + + public incrementRows_args getEmptyArgsInstance() { + return new incrementRows_args(); + } + + protected boolean isOneway() { + return false; + } + + public incrementRows_result getResult(I iface, incrementRows_args args) throws org.apache.thrift.TException { + incrementRows_result result = new incrementRows_result(); + try { + iface.incrementRows(args.increments); + } catch (IOError io) { + result.io = io; + } + return result; + } + } + + public static class deleteAllRowTs extends org.apache.thrift.ProcessFunction { + public deleteAllRowTs() { + super("deleteAllRowTs"); + } + + public deleteAllRowTs_args getEmptyArgsInstance() { + return new deleteAllRowTs_args(); + } + + protected boolean isOneway() { + return false; + } + + public deleteAllRowTs_result getResult(I iface, deleteAllRowTs_args args) throws org.apache.thrift.TException { + deleteAllRowTs_result result = new deleteAllRowTs_result(); + try { + iface.deleteAllRowTs(args.tableName, args.row, args.timestamp, args.attributes); + } catch (IOError io) { + result.io = io; + } + return result; + } + } + + public static class scannerOpenWithScan extends org.apache.thrift.ProcessFunction { + public scannerOpenWithScan() { + super("scannerOpenWithScan"); + } + + public scannerOpenWithScan_args getEmptyArgsInstance() { + return new scannerOpenWithScan_args(); + } + + protected boolean isOneway() { + return false; + } + + public scannerOpenWithScan_result getResult(I iface, scannerOpenWithScan_args args) throws org.apache.thrift.TException { + scannerOpenWithScan_result result = new scannerOpenWithScan_result(); + try { + result.success = iface.scannerOpenWithScan(args.tableName, args.scan, args.attributes); + result.setSuccessIsSet(true); + } catch (IOError io) { + result.io = io; + } + return result; + } + } + + public static class scannerOpen extends org.apache.thrift.ProcessFunction { + public scannerOpen() { + super("scannerOpen"); + } + + public scannerOpen_args getEmptyArgsInstance() { + return new scannerOpen_args(); + } + + protected boolean isOneway() { + return false; + } + + public scannerOpen_result getResult(I iface, scannerOpen_args args) throws org.apache.thrift.TException { + scannerOpen_result result = new scannerOpen_result(); + try { + result.success = iface.scannerOpen(args.tableName, args.startRow, args.columns, args.attributes); + result.setSuccessIsSet(true); + } catch (IOError io) { + result.io = io; + } + return result; + } + } + + public static class scannerOpenWithStop extends org.apache.thrift.ProcessFunction { + public scannerOpenWithStop() { + super("scannerOpenWithStop"); + } + + public scannerOpenWithStop_args getEmptyArgsInstance() { + return new scannerOpenWithStop_args(); + } + + protected boolean isOneway() { + return false; + } + + public scannerOpenWithStop_result getResult(I iface, scannerOpenWithStop_args args) throws org.apache.thrift.TException { + scannerOpenWithStop_result result = new scannerOpenWithStop_result(); + try { + result.success = iface.scannerOpenWithStop(args.tableName, args.startRow, args.stopRow, args.columns, args.attributes); + result.setSuccessIsSet(true); + } catch (IOError io) { + result.io = io; + } + return result; + } + } + + public static class scannerOpenWithPrefix extends org.apache.thrift.ProcessFunction { + public scannerOpenWithPrefix() { + super("scannerOpenWithPrefix"); + } + + public scannerOpenWithPrefix_args getEmptyArgsInstance() { + return new scannerOpenWithPrefix_args(); + } + + protected boolean isOneway() { + return false; + } + + public scannerOpenWithPrefix_result getResult(I iface, scannerOpenWithPrefix_args args) throws org.apache.thrift.TException { + scannerOpenWithPrefix_result result = new scannerOpenWithPrefix_result(); + try { + result.success = iface.scannerOpenWithPrefix(args.tableName, args.startAndPrefix, args.columns, args.attributes); + result.setSuccessIsSet(true); + } catch (IOError io) { + result.io = io; + } + return result; + } + } + + public static class scannerOpenTs extends org.apache.thrift.ProcessFunction { + public scannerOpenTs() { + super("scannerOpenTs"); + } + + public scannerOpenTs_args getEmptyArgsInstance() { + return new scannerOpenTs_args(); + } + + protected boolean isOneway() { + return false; + } + + public scannerOpenTs_result getResult(I iface, scannerOpenTs_args args) throws org.apache.thrift.TException { + scannerOpenTs_result result = new scannerOpenTs_result(); + try { + result.success = iface.scannerOpenTs(args.tableName, args.startRow, args.columns, args.timestamp, args.attributes); + result.setSuccessIsSet(true); + } catch (IOError io) { + result.io = io; + } + return result; + } + } + + public static class scannerOpenWithStopTs extends org.apache.thrift.ProcessFunction { + public scannerOpenWithStopTs() { + super("scannerOpenWithStopTs"); + } + + public scannerOpenWithStopTs_args getEmptyArgsInstance() { + return new scannerOpenWithStopTs_args(); + } + + protected boolean isOneway() { + return false; + } + + public scannerOpenWithStopTs_result getResult(I iface, scannerOpenWithStopTs_args args) throws org.apache.thrift.TException { + scannerOpenWithStopTs_result result = new scannerOpenWithStopTs_result(); + try { + result.success = iface.scannerOpenWithStopTs(args.tableName, args.startRow, args.stopRow, args.columns, args.timestamp, args.attributes); + result.setSuccessIsSet(true); + } catch (IOError io) { + result.io = io; + } + return result; + } + } + + public static class scannerGet extends org.apache.thrift.ProcessFunction { + public scannerGet() { + super("scannerGet"); + } + + public scannerGet_args getEmptyArgsInstance() { + return new scannerGet_args(); + } + + protected boolean isOneway() { + return false; + } + + public scannerGet_result getResult(I iface, scannerGet_args args) throws org.apache.thrift.TException { + scannerGet_result result = new scannerGet_result(); + try { + result.success = iface.scannerGet(args.id); + } catch (IOError io) { + result.io = io; + } catch (IllegalArgument ia) { + result.ia = ia; + } + return result; + } + } + + public static class scannerGetList extends org.apache.thrift.ProcessFunction { + public scannerGetList() { + super("scannerGetList"); + } + + public scannerGetList_args getEmptyArgsInstance() { + return new scannerGetList_args(); + } + + protected boolean isOneway() { + return false; + } + + public scannerGetList_result getResult(I iface, scannerGetList_args args) throws org.apache.thrift.TException { + scannerGetList_result result = new scannerGetList_result(); + try { + result.success = iface.scannerGetList(args.id, args.nbRows); + } catch (IOError io) { + result.io = io; + } catch (IllegalArgument ia) { + result.ia = ia; + } + return result; + } + } + + public static class scannerClose extends org.apache.thrift.ProcessFunction { + public scannerClose() { + super("scannerClose"); + } + + public scannerClose_args getEmptyArgsInstance() { + return new scannerClose_args(); + } + + protected boolean isOneway() { + return false; + } + + public scannerClose_result getResult(I iface, scannerClose_args args) throws org.apache.thrift.TException { + scannerClose_result result = new scannerClose_result(); + try { + iface.scannerClose(args.id); + } catch (IOError io) { + result.io = io; + } catch (IllegalArgument ia) { + result.ia = ia; + } + return result; + } + } + + public static class getRowOrBefore extends org.apache.thrift.ProcessFunction { + public getRowOrBefore() { + super("getRowOrBefore"); + } + + public getRowOrBefore_args getEmptyArgsInstance() { + return new getRowOrBefore_args(); + } + + protected boolean isOneway() { + return false; + } + + public getRowOrBefore_result getResult(I iface, getRowOrBefore_args args) throws org.apache.thrift.TException { + getRowOrBefore_result result = new getRowOrBefore_result(); + try { + result.success = iface.getRowOrBefore(args.tableName, args.row, args.family); + } catch (IOError io) { + result.io = io; + } + return result; + } + } + + public static class getRegionInfo extends org.apache.thrift.ProcessFunction { + public getRegionInfo() { + super("getRegionInfo"); + } + + public getRegionInfo_args getEmptyArgsInstance() { + return new getRegionInfo_args(); + } + + protected boolean isOneway() { + return false; + } + + public getRegionInfo_result getResult(I iface, getRegionInfo_args args) throws org.apache.thrift.TException { + getRegionInfo_result result = new getRegionInfo_result(); + try { + result.success = iface.getRegionInfo(args.row); + } catch (IOError io) { + result.io = io; + } + return result; + } + } + + } + + public static class AsyncProcessor extends org.apache.thrift.TBaseAsyncProcessor { + private static final Logger LOGGER = LoggerFactory.getLogger(AsyncProcessor.class.getName()); + public AsyncProcessor(I iface) { + super(iface, getProcessMap(new HashMap>())); + } + + protected AsyncProcessor(I iface, Map> processMap) { + super(iface, getProcessMap(processMap)); + } + + private static Map> getProcessMap(Map> processMap) { + processMap.put("enableTable", new enableTable()); + processMap.put("disableTable", new disableTable()); + processMap.put("isTableEnabled", new isTableEnabled()); + processMap.put("compact", new compact()); + processMap.put("majorCompact", new majorCompact()); + processMap.put("getTableNames", new getTableNames()); + processMap.put("getColumnDescriptors", new getColumnDescriptors()); + processMap.put("getTableRegions", new getTableRegions()); + processMap.put("createTable", new createTable()); + processMap.put("deleteTable", new deleteTable()); + processMap.put("get", new get()); + processMap.put("getVer", new getVer()); + processMap.put("getVerTs", new getVerTs()); + processMap.put("getRow", new getRow()); + processMap.put("getRowWithColumns", new getRowWithColumns()); + processMap.put("getRowTs", new getRowTs()); + processMap.put("getRowWithColumnsTs", new getRowWithColumnsTs()); + processMap.put("getRows", new getRows()); + processMap.put("getRowsWithColumns", new getRowsWithColumns()); + processMap.put("getRowsTs", new getRowsTs()); + processMap.put("getRowsWithColumnsTs", new getRowsWithColumnsTs()); + processMap.put("mutateRow", new mutateRow()); + processMap.put("mutateRowTs", new mutateRowTs()); + processMap.put("mutateRows", new mutateRows()); + processMap.put("mutateRowsTs", new mutateRowsTs()); + processMap.put("atomicIncrement", new atomicIncrement()); + processMap.put("deleteAll", new deleteAll()); + processMap.put("deleteAllTs", new deleteAllTs()); + processMap.put("deleteAllRow", new deleteAllRow()); + processMap.put("increment", new increment()); + processMap.put("incrementRows", new incrementRows()); + processMap.put("deleteAllRowTs", new deleteAllRowTs()); + processMap.put("scannerOpenWithScan", new scannerOpenWithScan()); + processMap.put("scannerOpen", new scannerOpen()); + processMap.put("scannerOpenWithStop", new scannerOpenWithStop()); + processMap.put("scannerOpenWithPrefix", new scannerOpenWithPrefix()); + processMap.put("scannerOpenTs", new scannerOpenTs()); + processMap.put("scannerOpenWithStopTs", new scannerOpenWithStopTs()); + processMap.put("scannerGet", new scannerGet()); + processMap.put("scannerGetList", new scannerGetList()); + processMap.put("scannerClose", new scannerClose()); + processMap.put("getRowOrBefore", new getRowOrBefore()); + processMap.put("getRegionInfo", new getRegionInfo()); + return processMap; + } + + public static class enableTable extends org.apache.thrift.AsyncProcessFunction { + public enableTable() { + super("enableTable"); + } + + public enableTable_args getEmptyArgsInstance() { + return new enableTable_args(); + } + + public AsyncMethodCallback getResultHandler(final AsyncFrameBuffer fb, final int seqid) { + final org.apache.thrift.AsyncProcessFunction fcall = this; + return new AsyncMethodCallback() { + public void onComplete(Void o) { + enableTable_result result = new enableTable_result(); + try { + fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); + return; + } catch (Exception e) { + LOGGER.error("Exception writing to internal frame buffer", e); + } + fb.close(); + } + public void onError(Exception e) { + byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; + org.apache.thrift.TBase msg; + enableTable_result result = new enableTable_result(); + if (e instanceof IOError) { + result.io = (IOError) e; + result.setIoIsSet(true); + msg = result; + } + else + { + msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION; + msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage()); + } + try { + fcall.sendResponse(fb,msg,msgType,seqid); + return; + } catch (Exception ex) { + LOGGER.error("Exception writing to internal frame buffer", ex); + } + fb.close(); + } + }; + } + + protected boolean isOneway() { + return false; + } + + public void start(I iface, enableTable_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws TException { + iface.enableTable(args.tableName,resultHandler); + } + } + + public static class disableTable extends org.apache.thrift.AsyncProcessFunction { + public disableTable() { + super("disableTable"); + } + + public disableTable_args getEmptyArgsInstance() { + return new disableTable_args(); + } + + public AsyncMethodCallback getResultHandler(final AsyncFrameBuffer fb, final int seqid) { + final org.apache.thrift.AsyncProcessFunction fcall = this; + return new AsyncMethodCallback() { + public void onComplete(Void o) { + disableTable_result result = new disableTable_result(); + try { + fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); + return; + } catch (Exception e) { + LOGGER.error("Exception writing to internal frame buffer", e); + } + fb.close(); + } + public void onError(Exception e) { + byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; + org.apache.thrift.TBase msg; + disableTable_result result = new disableTable_result(); + if (e instanceof IOError) { + result.io = (IOError) e; + result.setIoIsSet(true); + msg = result; + } + else + { + msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION; + msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage()); + } + try { + fcall.sendResponse(fb,msg,msgType,seqid); + return; + } catch (Exception ex) { + LOGGER.error("Exception writing to internal frame buffer", ex); + } + fb.close(); + } + }; + } + + protected boolean isOneway() { + return false; + } + + public void start(I iface, disableTable_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws TException { + iface.disableTable(args.tableName,resultHandler); + } + } + + public static class isTableEnabled extends org.apache.thrift.AsyncProcessFunction { + public isTableEnabled() { + super("isTableEnabled"); + } + + public isTableEnabled_args getEmptyArgsInstance() { + return new isTableEnabled_args(); + } + + public AsyncMethodCallback getResultHandler(final AsyncFrameBuffer fb, final int seqid) { + final org.apache.thrift.AsyncProcessFunction fcall = this; + return new AsyncMethodCallback() { + public void onComplete(Boolean o) { + isTableEnabled_result result = new isTableEnabled_result(); + result.success = o; + result.setSuccessIsSet(true); + try { + fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); + return; + } catch (Exception e) { + LOGGER.error("Exception writing to internal frame buffer", e); + } + fb.close(); + } + public void onError(Exception e) { + byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; + org.apache.thrift.TBase msg; + isTableEnabled_result result = new isTableEnabled_result(); + if (e instanceof IOError) { + result.io = (IOError) e; + result.setIoIsSet(true); + msg = result; + } + else + { + msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION; + msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage()); + } + try { + fcall.sendResponse(fb,msg,msgType,seqid); + return; + } catch (Exception ex) { + LOGGER.error("Exception writing to internal frame buffer", ex); + } + fb.close(); + } + }; + } + + protected boolean isOneway() { + return false; + } + + public void start(I iface, isTableEnabled_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws TException { + iface.isTableEnabled(args.tableName,resultHandler); + } + } + + public static class compact extends org.apache.thrift.AsyncProcessFunction { + public compact() { + super("compact"); + } + + public compact_args getEmptyArgsInstance() { + return new compact_args(); + } + + public AsyncMethodCallback getResultHandler(final AsyncFrameBuffer fb, final int seqid) { + final org.apache.thrift.AsyncProcessFunction fcall = this; + return new AsyncMethodCallback() { + public void onComplete(Void o) { + compact_result result = new compact_result(); + try { + fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); + return; + } catch (Exception e) { + LOGGER.error("Exception writing to internal frame buffer", e); + } + fb.close(); + } + public void onError(Exception e) { + byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; + org.apache.thrift.TBase msg; + compact_result result = new compact_result(); + if (e instanceof IOError) { + result.io = (IOError) e; + result.setIoIsSet(true); + msg = result; + } + else + { + msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION; + msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage()); + } + try { + fcall.sendResponse(fb,msg,msgType,seqid); + return; + } catch (Exception ex) { + LOGGER.error("Exception writing to internal frame buffer", ex); + } + fb.close(); + } + }; + } + + protected boolean isOneway() { + return false; + } + + public void start(I iface, compact_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws TException { + iface.compact(args.tableNameOrRegionName,resultHandler); + } + } + + public static class majorCompact extends org.apache.thrift.AsyncProcessFunction { + public majorCompact() { + super("majorCompact"); + } + + public majorCompact_args getEmptyArgsInstance() { + return new majorCompact_args(); + } + + public AsyncMethodCallback getResultHandler(final AsyncFrameBuffer fb, final int seqid) { + final org.apache.thrift.AsyncProcessFunction fcall = this; + return new AsyncMethodCallback() { + public void onComplete(Void o) { + majorCompact_result result = new majorCompact_result(); + try { + fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); + return; + } catch (Exception e) { + LOGGER.error("Exception writing to internal frame buffer", e); + } + fb.close(); + } + public void onError(Exception e) { + byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; + org.apache.thrift.TBase msg; + majorCompact_result result = new majorCompact_result(); + if (e instanceof IOError) { + result.io = (IOError) e; + result.setIoIsSet(true); + msg = result; + } + else + { + msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION; + msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage()); + } + try { + fcall.sendResponse(fb,msg,msgType,seqid); + return; + } catch (Exception ex) { + LOGGER.error("Exception writing to internal frame buffer", ex); + } + fb.close(); + } + }; + } + + protected boolean isOneway() { + return false; + } + + public void start(I iface, majorCompact_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws TException { + iface.majorCompact(args.tableNameOrRegionName,resultHandler); + } + } + + public static class getTableNames extends org.apache.thrift.AsyncProcessFunction> { + public getTableNames() { + super("getTableNames"); + } + + public getTableNames_args getEmptyArgsInstance() { + return new getTableNames_args(); + } + + public AsyncMethodCallback> getResultHandler(final AsyncFrameBuffer fb, final int seqid) { + final org.apache.thrift.AsyncProcessFunction fcall = this; + return new AsyncMethodCallback>() { + public void onComplete(List o) { + getTableNames_result result = new getTableNames_result(); + result.success = o; + try { + fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); + return; + } catch (Exception e) { + LOGGER.error("Exception writing to internal frame buffer", e); + } + fb.close(); + } + public void onError(Exception e) { + byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; + org.apache.thrift.TBase msg; + getTableNames_result result = new getTableNames_result(); + if (e instanceof IOError) { + result.io = (IOError) e; + result.setIoIsSet(true); + msg = result; + } + else + { + msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION; + msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage()); + } + try { + fcall.sendResponse(fb,msg,msgType,seqid); + return; + } catch (Exception ex) { + LOGGER.error("Exception writing to internal frame buffer", ex); + } + fb.close(); + } + }; + } + + protected boolean isOneway() { + return false; + } + + public void start(I iface, getTableNames_args args, org.apache.thrift.async.AsyncMethodCallback> resultHandler) throws TException { + iface.getTableNames(resultHandler); + } + } + + public static class getColumnDescriptors extends org.apache.thrift.AsyncProcessFunction> { + public getColumnDescriptors() { + super("getColumnDescriptors"); + } + + public getColumnDescriptors_args getEmptyArgsInstance() { + return new getColumnDescriptors_args(); + } + + public AsyncMethodCallback> getResultHandler(final AsyncFrameBuffer fb, final int seqid) { + final org.apache.thrift.AsyncProcessFunction fcall = this; + return new AsyncMethodCallback>() { + public void onComplete(Map o) { + getColumnDescriptors_result result = new getColumnDescriptors_result(); + result.success = o; + try { + fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); + return; + } catch (Exception e) { + LOGGER.error("Exception writing to internal frame buffer", e); + } + fb.close(); + } + public void onError(Exception e) { + byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; + org.apache.thrift.TBase msg; + getColumnDescriptors_result result = new getColumnDescriptors_result(); + if (e instanceof IOError) { + result.io = (IOError) e; + result.setIoIsSet(true); + msg = result; + } + else + { + msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION; + msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage()); + } + try { + fcall.sendResponse(fb,msg,msgType,seqid); + return; + } catch (Exception ex) { + LOGGER.error("Exception writing to internal frame buffer", ex); + } + fb.close(); + } + }; + } + + protected boolean isOneway() { + return false; + } + + public void start(I iface, getColumnDescriptors_args args, org.apache.thrift.async.AsyncMethodCallback> resultHandler) throws TException { + iface.getColumnDescriptors(args.tableName,resultHandler); + } + } + + public static class getTableRegions extends org.apache.thrift.AsyncProcessFunction> { + public getTableRegions() { + super("getTableRegions"); + } + + public getTableRegions_args getEmptyArgsInstance() { + return new getTableRegions_args(); + } + + public AsyncMethodCallback> getResultHandler(final AsyncFrameBuffer fb, final int seqid) { + final org.apache.thrift.AsyncProcessFunction fcall = this; + return new AsyncMethodCallback>() { + public void onComplete(List o) { + getTableRegions_result result = new getTableRegions_result(); + result.success = o; + try { + fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); + return; + } catch (Exception e) { + LOGGER.error("Exception writing to internal frame buffer", e); + } + fb.close(); + } + public void onError(Exception e) { + byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; + org.apache.thrift.TBase msg; + getTableRegions_result result = new getTableRegions_result(); + if (e instanceof IOError) { + result.io = (IOError) e; + result.setIoIsSet(true); + msg = result; + } + else + { + msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION; + msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage()); + } + try { + fcall.sendResponse(fb,msg,msgType,seqid); + return; + } catch (Exception ex) { + LOGGER.error("Exception writing to internal frame buffer", ex); + } + fb.close(); + } + }; + } + + protected boolean isOneway() { + return false; + } + + public void start(I iface, getTableRegions_args args, org.apache.thrift.async.AsyncMethodCallback> resultHandler) throws TException { + iface.getTableRegions(args.tableName,resultHandler); + } + } + + public static class createTable extends org.apache.thrift.AsyncProcessFunction { + public createTable() { + super("createTable"); + } + + public createTable_args getEmptyArgsInstance() { + return new createTable_args(); + } + + public AsyncMethodCallback getResultHandler(final AsyncFrameBuffer fb, final int seqid) { + final org.apache.thrift.AsyncProcessFunction fcall = this; + return new AsyncMethodCallback() { + public void onComplete(Void o) { + createTable_result result = new createTable_result(); + try { + fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); + return; + } catch (Exception e) { + LOGGER.error("Exception writing to internal frame buffer", e); + } + fb.close(); + } + public void onError(Exception e) { + byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; + org.apache.thrift.TBase msg; + createTable_result result = new createTable_result(); + if (e instanceof IOError) { + result.io = (IOError) e; + result.setIoIsSet(true); + msg = result; + } + else if (e instanceof IllegalArgument) { + result.ia = (IllegalArgument) e; + result.setIaIsSet(true); + msg = result; + } + else if (e instanceof AlreadyExists) { + result.exist = (AlreadyExists) e; + result.setExistIsSet(true); + msg = result; + } + else + { + msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION; + msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage()); + } + try { + fcall.sendResponse(fb,msg,msgType,seqid); + return; + } catch (Exception ex) { + LOGGER.error("Exception writing to internal frame buffer", ex); + } + fb.close(); + } + }; + } + + protected boolean isOneway() { + return false; + } + + public void start(I iface, createTable_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws TException { + iface.createTable(args.tableName, args.columnFamilies,resultHandler); + } + } + + public static class deleteTable extends org.apache.thrift.AsyncProcessFunction { + public deleteTable() { + super("deleteTable"); + } + + public deleteTable_args getEmptyArgsInstance() { + return new deleteTable_args(); + } + + public AsyncMethodCallback getResultHandler(final AsyncFrameBuffer fb, final int seqid) { + final org.apache.thrift.AsyncProcessFunction fcall = this; + return new AsyncMethodCallback() { + public void onComplete(Void o) { + deleteTable_result result = new deleteTable_result(); + try { + fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); + return; + } catch (Exception e) { + LOGGER.error("Exception writing to internal frame buffer", e); + } + fb.close(); + } + public void onError(Exception e) { + byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; + org.apache.thrift.TBase msg; + deleteTable_result result = new deleteTable_result(); + if (e instanceof IOError) { + result.io = (IOError) e; + result.setIoIsSet(true); + msg = result; + } + else + { + msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION; + msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage()); + } + try { + fcall.sendResponse(fb,msg,msgType,seqid); + return; + } catch (Exception ex) { + LOGGER.error("Exception writing to internal frame buffer", ex); + } + fb.close(); + } + }; + } + + protected boolean isOneway() { + return false; + } + + public void start(I iface, deleteTable_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws TException { + iface.deleteTable(args.tableName,resultHandler); + } + } + + public static class get extends org.apache.thrift.AsyncProcessFunction> { + public get() { + super("get"); + } + + public get_args getEmptyArgsInstance() { + return new get_args(); + } + + public AsyncMethodCallback> getResultHandler(final AsyncFrameBuffer fb, final int seqid) { + final org.apache.thrift.AsyncProcessFunction fcall = this; + return new AsyncMethodCallback>() { + public void onComplete(List o) { + get_result result = new get_result(); + result.success = o; + try { + fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); + return; + } catch (Exception e) { + LOGGER.error("Exception writing to internal frame buffer", e); + } + fb.close(); + } + public void onError(Exception e) { + byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; + org.apache.thrift.TBase msg; + get_result result = new get_result(); + if (e instanceof IOError) { + result.io = (IOError) e; + result.setIoIsSet(true); + msg = result; + } + else + { + msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION; + msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage()); + } + try { + fcall.sendResponse(fb,msg,msgType,seqid); + return; + } catch (Exception ex) { + LOGGER.error("Exception writing to internal frame buffer", ex); + } + fb.close(); + } + }; + } + + protected boolean isOneway() { + return false; + } + + public void start(I iface, get_args args, org.apache.thrift.async.AsyncMethodCallback> resultHandler) throws TException { + iface.get(args.tableName, args.row, args.column, args.attributes,resultHandler); + } + } + + public static class getVer extends org.apache.thrift.AsyncProcessFunction> { + public getVer() { + super("getVer"); + } + + public getVer_args getEmptyArgsInstance() { + return new getVer_args(); + } + + public AsyncMethodCallback> getResultHandler(final AsyncFrameBuffer fb, final int seqid) { + final org.apache.thrift.AsyncProcessFunction fcall = this; + return new AsyncMethodCallback>() { + public void onComplete(List o) { + getVer_result result = new getVer_result(); + result.success = o; + try { + fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); + return; + } catch (Exception e) { + LOGGER.error("Exception writing to internal frame buffer", e); + } + fb.close(); + } + public void onError(Exception e) { + byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; + org.apache.thrift.TBase msg; + getVer_result result = new getVer_result(); + if (e instanceof IOError) { + result.io = (IOError) e; + result.setIoIsSet(true); + msg = result; + } + else + { + msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION; + msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage()); + } + try { + fcall.sendResponse(fb,msg,msgType,seqid); + return; + } catch (Exception ex) { + LOGGER.error("Exception writing to internal frame buffer", ex); + } + fb.close(); + } + }; + } + + protected boolean isOneway() { + return false; + } + + public void start(I iface, getVer_args args, org.apache.thrift.async.AsyncMethodCallback> resultHandler) throws TException { + iface.getVer(args.tableName, args.row, args.column, args.numVersions, args.attributes,resultHandler); + } + } + + public static class getVerTs extends org.apache.thrift.AsyncProcessFunction> { + public getVerTs() { + super("getVerTs"); + } + + public getVerTs_args getEmptyArgsInstance() { + return new getVerTs_args(); + } + + public AsyncMethodCallback> getResultHandler(final AsyncFrameBuffer fb, final int seqid) { + final org.apache.thrift.AsyncProcessFunction fcall = this; + return new AsyncMethodCallback>() { + public void onComplete(List o) { + getVerTs_result result = new getVerTs_result(); + result.success = o; + try { + fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); + return; + } catch (Exception e) { + LOGGER.error("Exception writing to internal frame buffer", e); + } + fb.close(); + } + public void onError(Exception e) { + byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; + org.apache.thrift.TBase msg; + getVerTs_result result = new getVerTs_result(); + if (e instanceof IOError) { + result.io = (IOError) e; + result.setIoIsSet(true); + msg = result; + } + else + { + msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION; + msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage()); + } + try { + fcall.sendResponse(fb,msg,msgType,seqid); + return; + } catch (Exception ex) { + LOGGER.error("Exception writing to internal frame buffer", ex); + } + fb.close(); + } + }; + } + + protected boolean isOneway() { + return false; + } + + public void start(I iface, getVerTs_args args, org.apache.thrift.async.AsyncMethodCallback> resultHandler) throws TException { + iface.getVerTs(args.tableName, args.row, args.column, args.timestamp, args.numVersions, args.attributes,resultHandler); + } + } + + public static class getRow extends org.apache.thrift.AsyncProcessFunction> { + public getRow() { + super("getRow"); + } + + public getRow_args getEmptyArgsInstance() { + return new getRow_args(); + } + + public AsyncMethodCallback> getResultHandler(final AsyncFrameBuffer fb, final int seqid) { + final org.apache.thrift.AsyncProcessFunction fcall = this; + return new AsyncMethodCallback>() { + public void onComplete(List o) { + getRow_result result = new getRow_result(); + result.success = o; + try { + fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); + return; + } catch (Exception e) { + LOGGER.error("Exception writing to internal frame buffer", e); + } + fb.close(); + } + public void onError(Exception e) { + byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; + org.apache.thrift.TBase msg; + getRow_result result = new getRow_result(); + if (e instanceof IOError) { + result.io = (IOError) e; + result.setIoIsSet(true); + msg = result; + } + else + { + msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION; + msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage()); + } + try { + fcall.sendResponse(fb,msg,msgType,seqid); + return; + } catch (Exception ex) { + LOGGER.error("Exception writing to internal frame buffer", ex); + } + fb.close(); + } + }; + } + + protected boolean isOneway() { + return false; + } + + public void start(I iface, getRow_args args, org.apache.thrift.async.AsyncMethodCallback> resultHandler) throws TException { + iface.getRow(args.tableName, args.row, args.attributes,resultHandler); + } + } + + public static class getRowWithColumns extends org.apache.thrift.AsyncProcessFunction> { + public getRowWithColumns() { + super("getRowWithColumns"); + } + + public getRowWithColumns_args getEmptyArgsInstance() { + return new getRowWithColumns_args(); + } + + public AsyncMethodCallback> getResultHandler(final AsyncFrameBuffer fb, final int seqid) { + final org.apache.thrift.AsyncProcessFunction fcall = this; + return new AsyncMethodCallback>() { + public void onComplete(List o) { + getRowWithColumns_result result = new getRowWithColumns_result(); + result.success = o; + try { + fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); + return; + } catch (Exception e) { + LOGGER.error("Exception writing to internal frame buffer", e); + } + fb.close(); + } + public void onError(Exception e) { + byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; + org.apache.thrift.TBase msg; + getRowWithColumns_result result = new getRowWithColumns_result(); + if (e instanceof IOError) { + result.io = (IOError) e; + result.setIoIsSet(true); + msg = result; + } + else + { + msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION; + msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage()); + } + try { + fcall.sendResponse(fb,msg,msgType,seqid); + return; + } catch (Exception ex) { + LOGGER.error("Exception writing to internal frame buffer", ex); + } + fb.close(); + } + }; + } + + protected boolean isOneway() { + return false; + } + + public void start(I iface, getRowWithColumns_args args, org.apache.thrift.async.AsyncMethodCallback> resultHandler) throws TException { + iface.getRowWithColumns(args.tableName, args.row, args.columns, args.attributes,resultHandler); + } + } + + public static class getRowTs extends org.apache.thrift.AsyncProcessFunction> { + public getRowTs() { + super("getRowTs"); + } + + public getRowTs_args getEmptyArgsInstance() { + return new getRowTs_args(); + } + + public AsyncMethodCallback> getResultHandler(final AsyncFrameBuffer fb, final int seqid) { + final org.apache.thrift.AsyncProcessFunction fcall = this; + return new AsyncMethodCallback>() { + public void onComplete(List o) { + getRowTs_result result = new getRowTs_result(); + result.success = o; + try { + fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); + return; + } catch (Exception e) { + LOGGER.error("Exception writing to internal frame buffer", e); + } + fb.close(); + } + public void onError(Exception e) { + byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; + org.apache.thrift.TBase msg; + getRowTs_result result = new getRowTs_result(); + if (e instanceof IOError) { + result.io = (IOError) e; + result.setIoIsSet(true); + msg = result; + } + else + { + msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION; + msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage()); + } + try { + fcall.sendResponse(fb,msg,msgType,seqid); + return; + } catch (Exception ex) { + LOGGER.error("Exception writing to internal frame buffer", ex); + } + fb.close(); + } + }; + } + + protected boolean isOneway() { + return false; + } + + public void start(I iface, getRowTs_args args, org.apache.thrift.async.AsyncMethodCallback> resultHandler) throws TException { + iface.getRowTs(args.tableName, args.row, args.timestamp, args.attributes,resultHandler); + } + } + + public static class getRowWithColumnsTs extends org.apache.thrift.AsyncProcessFunction> { + public getRowWithColumnsTs() { + super("getRowWithColumnsTs"); + } + + public getRowWithColumnsTs_args getEmptyArgsInstance() { + return new getRowWithColumnsTs_args(); + } + + public AsyncMethodCallback> getResultHandler(final AsyncFrameBuffer fb, final int seqid) { + final org.apache.thrift.AsyncProcessFunction fcall = this; + return new AsyncMethodCallback>() { + public void onComplete(List o) { + getRowWithColumnsTs_result result = new getRowWithColumnsTs_result(); + result.success = o; + try { + fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); + return; + } catch (Exception e) { + LOGGER.error("Exception writing to internal frame buffer", e); + } + fb.close(); + } + public void onError(Exception e) { + byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; + org.apache.thrift.TBase msg; + getRowWithColumnsTs_result result = new getRowWithColumnsTs_result(); + if (e instanceof IOError) { + result.io = (IOError) e; + result.setIoIsSet(true); + msg = result; + } + else + { + msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION; + msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage()); + } + try { + fcall.sendResponse(fb,msg,msgType,seqid); + return; + } catch (Exception ex) { + LOGGER.error("Exception writing to internal frame buffer", ex); + } + fb.close(); + } + }; + } + + protected boolean isOneway() { + return false; + } + + public void start(I iface, getRowWithColumnsTs_args args, org.apache.thrift.async.AsyncMethodCallback> resultHandler) throws TException { + iface.getRowWithColumnsTs(args.tableName, args.row, args.columns, args.timestamp, args.attributes,resultHandler); + } + } + + public static class getRows extends org.apache.thrift.AsyncProcessFunction> { + public getRows() { + super("getRows"); + } + + public getRows_args getEmptyArgsInstance() { + return new getRows_args(); + } + + public AsyncMethodCallback> getResultHandler(final AsyncFrameBuffer fb, final int seqid) { + final org.apache.thrift.AsyncProcessFunction fcall = this; + return new AsyncMethodCallback>() { + public void onComplete(List o) { + getRows_result result = new getRows_result(); + result.success = o; + try { + fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); + return; + } catch (Exception e) { + LOGGER.error("Exception writing to internal frame buffer", e); + } + fb.close(); + } + public void onError(Exception e) { + byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; + org.apache.thrift.TBase msg; + getRows_result result = new getRows_result(); + if (e instanceof IOError) { + result.io = (IOError) e; + result.setIoIsSet(true); + msg = result; + } + else + { + msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION; + msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage()); + } + try { + fcall.sendResponse(fb,msg,msgType,seqid); + return; + } catch (Exception ex) { + LOGGER.error("Exception writing to internal frame buffer", ex); + } + fb.close(); + } + }; + } + + protected boolean isOneway() { + return false; + } + + public void start(I iface, getRows_args args, org.apache.thrift.async.AsyncMethodCallback> resultHandler) throws TException { + iface.getRows(args.tableName, args.rows, args.attributes,resultHandler); + } + } + + public static class getRowsWithColumns extends org.apache.thrift.AsyncProcessFunction> { + public getRowsWithColumns() { + super("getRowsWithColumns"); + } + + public getRowsWithColumns_args getEmptyArgsInstance() { + return new getRowsWithColumns_args(); + } + + public AsyncMethodCallback> getResultHandler(final AsyncFrameBuffer fb, final int seqid) { + final org.apache.thrift.AsyncProcessFunction fcall = this; + return new AsyncMethodCallback>() { + public void onComplete(List o) { + getRowsWithColumns_result result = new getRowsWithColumns_result(); + result.success = o; + try { + fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); + return; + } catch (Exception e) { + LOGGER.error("Exception writing to internal frame buffer", e); + } + fb.close(); + } + public void onError(Exception e) { + byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; + org.apache.thrift.TBase msg; + getRowsWithColumns_result result = new getRowsWithColumns_result(); + if (e instanceof IOError) { + result.io = (IOError) e; + result.setIoIsSet(true); + msg = result; + } + else + { + msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION; + msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage()); + } + try { + fcall.sendResponse(fb,msg,msgType,seqid); + return; + } catch (Exception ex) { + LOGGER.error("Exception writing to internal frame buffer", ex); + } + fb.close(); + } + }; + } + + protected boolean isOneway() { + return false; + } + + public void start(I iface, getRowsWithColumns_args args, org.apache.thrift.async.AsyncMethodCallback> resultHandler) throws TException { + iface.getRowsWithColumns(args.tableName, args.rows, args.columns, args.attributes,resultHandler); + } + } + + public static class getRowsTs extends org.apache.thrift.AsyncProcessFunction> { + public getRowsTs() { + super("getRowsTs"); + } + + public getRowsTs_args getEmptyArgsInstance() { + return new getRowsTs_args(); + } + + public AsyncMethodCallback> getResultHandler(final AsyncFrameBuffer fb, final int seqid) { + final org.apache.thrift.AsyncProcessFunction fcall = this; + return new AsyncMethodCallback>() { + public void onComplete(List o) { + getRowsTs_result result = new getRowsTs_result(); + result.success = o; + try { + fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); + return; + } catch (Exception e) { + LOGGER.error("Exception writing to internal frame buffer", e); + } + fb.close(); + } + public void onError(Exception e) { + byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; + org.apache.thrift.TBase msg; + getRowsTs_result result = new getRowsTs_result(); + if (e instanceof IOError) { + result.io = (IOError) e; + result.setIoIsSet(true); + msg = result; + } + else + { + msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION; + msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage()); + } + try { + fcall.sendResponse(fb,msg,msgType,seqid); + return; + } catch (Exception ex) { + LOGGER.error("Exception writing to internal frame buffer", ex); + } + fb.close(); + } + }; + } + + protected boolean isOneway() { + return false; + } + + public void start(I iface, getRowsTs_args args, org.apache.thrift.async.AsyncMethodCallback> resultHandler) throws TException { + iface.getRowsTs(args.tableName, args.rows, args.timestamp, args.attributes,resultHandler); + } + } + + public static class getRowsWithColumnsTs extends org.apache.thrift.AsyncProcessFunction> { + public getRowsWithColumnsTs() { + super("getRowsWithColumnsTs"); + } + + public getRowsWithColumnsTs_args getEmptyArgsInstance() { + return new getRowsWithColumnsTs_args(); + } + + public AsyncMethodCallback> getResultHandler(final AsyncFrameBuffer fb, final int seqid) { + final org.apache.thrift.AsyncProcessFunction fcall = this; + return new AsyncMethodCallback>() { + public void onComplete(List o) { + getRowsWithColumnsTs_result result = new getRowsWithColumnsTs_result(); + result.success = o; + try { + fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); + return; + } catch (Exception e) { + LOGGER.error("Exception writing to internal frame buffer", e); + } + fb.close(); + } + public void onError(Exception e) { + byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; + org.apache.thrift.TBase msg; + getRowsWithColumnsTs_result result = new getRowsWithColumnsTs_result(); + if (e instanceof IOError) { + result.io = (IOError) e; + result.setIoIsSet(true); + msg = result; + } + else + { + msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION; + msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage()); + } + try { + fcall.sendResponse(fb,msg,msgType,seqid); + return; + } catch (Exception ex) { + LOGGER.error("Exception writing to internal frame buffer", ex); + } + fb.close(); + } + }; + } + + protected boolean isOneway() { + return false; + } + + public void start(I iface, getRowsWithColumnsTs_args args, org.apache.thrift.async.AsyncMethodCallback> resultHandler) throws TException { + iface.getRowsWithColumnsTs(args.tableName, args.rows, args.columns, args.timestamp, args.attributes,resultHandler); + } + } + + public static class mutateRow extends org.apache.thrift.AsyncProcessFunction { + public mutateRow() { + super("mutateRow"); + } + + public mutateRow_args getEmptyArgsInstance() { + return new mutateRow_args(); + } + + public AsyncMethodCallback getResultHandler(final AsyncFrameBuffer fb, final int seqid) { + final org.apache.thrift.AsyncProcessFunction fcall = this; + return new AsyncMethodCallback() { + public void onComplete(Void o) { + mutateRow_result result = new mutateRow_result(); + try { + fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); + return; + } catch (Exception e) { + LOGGER.error("Exception writing to internal frame buffer", e); + } + fb.close(); + } + public void onError(Exception e) { + byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; + org.apache.thrift.TBase msg; + mutateRow_result result = new mutateRow_result(); + if (e instanceof IOError) { + result.io = (IOError) e; + result.setIoIsSet(true); + msg = result; + } + else if (e instanceof IllegalArgument) { + result.ia = (IllegalArgument) e; + result.setIaIsSet(true); + msg = result; + } + else + { + msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION; + msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage()); + } + try { + fcall.sendResponse(fb,msg,msgType,seqid); + return; + } catch (Exception ex) { + LOGGER.error("Exception writing to internal frame buffer", ex); + } + fb.close(); + } + }; + } + + protected boolean isOneway() { + return false; + } + + public void start(I iface, mutateRow_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws TException { + iface.mutateRow(args.tableName, args.row, args.mutations, args.attributes,resultHandler); + } + } + + public static class mutateRowTs extends org.apache.thrift.AsyncProcessFunction { + public mutateRowTs() { + super("mutateRowTs"); + } + + public mutateRowTs_args getEmptyArgsInstance() { + return new mutateRowTs_args(); + } + + public AsyncMethodCallback getResultHandler(final AsyncFrameBuffer fb, final int seqid) { + final org.apache.thrift.AsyncProcessFunction fcall = this; + return new AsyncMethodCallback() { + public void onComplete(Void o) { + mutateRowTs_result result = new mutateRowTs_result(); + try { + fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); + return; + } catch (Exception e) { + LOGGER.error("Exception writing to internal frame buffer", e); + } + fb.close(); + } + public void onError(Exception e) { + byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; + org.apache.thrift.TBase msg; + mutateRowTs_result result = new mutateRowTs_result(); + if (e instanceof IOError) { + result.io = (IOError) e; + result.setIoIsSet(true); + msg = result; + } + else if (e instanceof IllegalArgument) { + result.ia = (IllegalArgument) e; + result.setIaIsSet(true); + msg = result; + } + else + { + msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION; + msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage()); + } + try { + fcall.sendResponse(fb,msg,msgType,seqid); + return; + } catch (Exception ex) { + LOGGER.error("Exception writing to internal frame buffer", ex); + } + fb.close(); + } + }; + } + + protected boolean isOneway() { + return false; + } + + public void start(I iface, mutateRowTs_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws TException { + iface.mutateRowTs(args.tableName, args.row, args.mutations, args.timestamp, args.attributes,resultHandler); + } + } + + public static class mutateRows extends org.apache.thrift.AsyncProcessFunction { + public mutateRows() { + super("mutateRows"); + } + + public mutateRows_args getEmptyArgsInstance() { + return new mutateRows_args(); + } + + public AsyncMethodCallback getResultHandler(final AsyncFrameBuffer fb, final int seqid) { + final org.apache.thrift.AsyncProcessFunction fcall = this; + return new AsyncMethodCallback() { + public void onComplete(Void o) { + mutateRows_result result = new mutateRows_result(); + try { + fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); + return; + } catch (Exception e) { + LOGGER.error("Exception writing to internal frame buffer", e); + } + fb.close(); + } + public void onError(Exception e) { + byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; + org.apache.thrift.TBase msg; + mutateRows_result result = new mutateRows_result(); + if (e instanceof IOError) { + result.io = (IOError) e; + result.setIoIsSet(true); + msg = result; + } + else if (e instanceof IllegalArgument) { + result.ia = (IllegalArgument) e; + result.setIaIsSet(true); + msg = result; + } + else + { + msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION; + msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage()); + } + try { + fcall.sendResponse(fb,msg,msgType,seqid); + return; + } catch (Exception ex) { + LOGGER.error("Exception writing to internal frame buffer", ex); + } + fb.close(); + } + }; + } + + protected boolean isOneway() { + return false; + } + + public void start(I iface, mutateRows_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws TException { + iface.mutateRows(args.tableName, args.rowBatches, args.attributes,resultHandler); + } + } + + public static class mutateRowsTs extends org.apache.thrift.AsyncProcessFunction { + public mutateRowsTs() { + super("mutateRowsTs"); + } + + public mutateRowsTs_args getEmptyArgsInstance() { + return new mutateRowsTs_args(); + } + + public AsyncMethodCallback getResultHandler(final AsyncFrameBuffer fb, final int seqid) { + final org.apache.thrift.AsyncProcessFunction fcall = this; + return new AsyncMethodCallback() { + public void onComplete(Void o) { + mutateRowsTs_result result = new mutateRowsTs_result(); + try { + fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); + return; + } catch (Exception e) { + LOGGER.error("Exception writing to internal frame buffer", e); + } + fb.close(); + } + public void onError(Exception e) { + byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; + org.apache.thrift.TBase msg; + mutateRowsTs_result result = new mutateRowsTs_result(); + if (e instanceof IOError) { + result.io = (IOError) e; + result.setIoIsSet(true); + msg = result; + } + else if (e instanceof IllegalArgument) { + result.ia = (IllegalArgument) e; + result.setIaIsSet(true); + msg = result; + } + else + { + msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION; + msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage()); + } + try { + fcall.sendResponse(fb,msg,msgType,seqid); + return; + } catch (Exception ex) { + LOGGER.error("Exception writing to internal frame buffer", ex); + } + fb.close(); + } + }; + } + + protected boolean isOneway() { + return false; + } + + public void start(I iface, mutateRowsTs_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws TException { + iface.mutateRowsTs(args.tableName, args.rowBatches, args.timestamp, args.attributes,resultHandler); + } + } + + public static class atomicIncrement extends org.apache.thrift.AsyncProcessFunction { + public atomicIncrement() { + super("atomicIncrement"); + } + + public atomicIncrement_args getEmptyArgsInstance() { + return new atomicIncrement_args(); + } + + public AsyncMethodCallback getResultHandler(final AsyncFrameBuffer fb, final int seqid) { + final org.apache.thrift.AsyncProcessFunction fcall = this; + return new AsyncMethodCallback() { + public void onComplete(Long o) { + atomicIncrement_result result = new atomicIncrement_result(); + result.success = o; + result.setSuccessIsSet(true); + try { + fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); + return; + } catch (Exception e) { + LOGGER.error("Exception writing to internal frame buffer", e); + } + fb.close(); + } + public void onError(Exception e) { + byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; + org.apache.thrift.TBase msg; + atomicIncrement_result result = new atomicIncrement_result(); + if (e instanceof IOError) { + result.io = (IOError) e; + result.setIoIsSet(true); + msg = result; + } + else if (e instanceof IllegalArgument) { + result.ia = (IllegalArgument) e; + result.setIaIsSet(true); + msg = result; + } + else + { + msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION; + msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage()); + } + try { + fcall.sendResponse(fb,msg,msgType,seqid); + return; + } catch (Exception ex) { + LOGGER.error("Exception writing to internal frame buffer", ex); + } + fb.close(); + } + }; + } + + protected boolean isOneway() { + return false; + } + + public void start(I iface, atomicIncrement_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws TException { + iface.atomicIncrement(args.tableName, args.row, args.column, args.value,resultHandler); + } + } + + public static class deleteAll extends org.apache.thrift.AsyncProcessFunction { + public deleteAll() { + super("deleteAll"); + } + + public deleteAll_args getEmptyArgsInstance() { + return new deleteAll_args(); + } + + public AsyncMethodCallback getResultHandler(final AsyncFrameBuffer fb, final int seqid) { + final org.apache.thrift.AsyncProcessFunction fcall = this; + return new AsyncMethodCallback() { + public void onComplete(Void o) { + deleteAll_result result = new deleteAll_result(); + try { + fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); + return; + } catch (Exception e) { + LOGGER.error("Exception writing to internal frame buffer", e); + } + fb.close(); + } + public void onError(Exception e) { + byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; + org.apache.thrift.TBase msg; + deleteAll_result result = new deleteAll_result(); + if (e instanceof IOError) { + result.io = (IOError) e; + result.setIoIsSet(true); + msg = result; + } + else + { + msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION; + msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage()); + } + try { + fcall.sendResponse(fb,msg,msgType,seqid); + return; + } catch (Exception ex) { + LOGGER.error("Exception writing to internal frame buffer", ex); + } + fb.close(); + } + }; + } + + protected boolean isOneway() { + return false; + } + + public void start(I iface, deleteAll_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws TException { + iface.deleteAll(args.tableName, args.row, args.column, args.attributes,resultHandler); + } + } + + public static class deleteAllTs extends org.apache.thrift.AsyncProcessFunction { + public deleteAllTs() { + super("deleteAllTs"); + } + + public deleteAllTs_args getEmptyArgsInstance() { + return new deleteAllTs_args(); + } + + public AsyncMethodCallback getResultHandler(final AsyncFrameBuffer fb, final int seqid) { + final org.apache.thrift.AsyncProcessFunction fcall = this; + return new AsyncMethodCallback() { + public void onComplete(Void o) { + deleteAllTs_result result = new deleteAllTs_result(); + try { + fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); + return; + } catch (Exception e) { + LOGGER.error("Exception writing to internal frame buffer", e); + } + fb.close(); + } + public void onError(Exception e) { + byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; + org.apache.thrift.TBase msg; + deleteAllTs_result result = new deleteAllTs_result(); + if (e instanceof IOError) { + result.io = (IOError) e; + result.setIoIsSet(true); + msg = result; + } + else + { + msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION; + msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage()); + } + try { + fcall.sendResponse(fb,msg,msgType,seqid); + return; + } catch (Exception ex) { + LOGGER.error("Exception writing to internal frame buffer", ex); + } + fb.close(); + } + }; + } + + protected boolean isOneway() { + return false; + } + + public void start(I iface, deleteAllTs_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws TException { + iface.deleteAllTs(args.tableName, args.row, args.column, args.timestamp, args.attributes,resultHandler); + } + } + + public static class deleteAllRow extends org.apache.thrift.AsyncProcessFunction { + public deleteAllRow() { + super("deleteAllRow"); + } + + public deleteAllRow_args getEmptyArgsInstance() { + return new deleteAllRow_args(); + } + + public AsyncMethodCallback getResultHandler(final AsyncFrameBuffer fb, final int seqid) { + final org.apache.thrift.AsyncProcessFunction fcall = this; + return new AsyncMethodCallback() { + public void onComplete(Void o) { + deleteAllRow_result result = new deleteAllRow_result(); + try { + fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); + return; + } catch (Exception e) { + LOGGER.error("Exception writing to internal frame buffer", e); + } + fb.close(); + } + public void onError(Exception e) { + byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; + org.apache.thrift.TBase msg; + deleteAllRow_result result = new deleteAllRow_result(); + if (e instanceof IOError) { + result.io = (IOError) e; + result.setIoIsSet(true); + msg = result; + } + else + { + msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION; + msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage()); + } + try { + fcall.sendResponse(fb,msg,msgType,seqid); + return; + } catch (Exception ex) { + LOGGER.error("Exception writing to internal frame buffer", ex); + } + fb.close(); + } + }; + } + + protected boolean isOneway() { + return false; + } + + public void start(I iface, deleteAllRow_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws TException { + iface.deleteAllRow(args.tableName, args.row, args.attributes,resultHandler); } } - private static class increment extends org.apache.thrift.ProcessFunction { + public static class increment extends org.apache.thrift.AsyncProcessFunction { public increment() { super("increment"); } - protected increment_args getEmptyArgsInstance() { + public increment_args getEmptyArgsInstance() { return new increment_args(); } - protected increment_result getResult(I iface, increment_args args) throws org.apache.thrift.TException { - increment_result result = new increment_result(); - try { - iface.increment(args.increment); - } catch (IOError io) { - result.io = io; - } - return result; + public AsyncMethodCallback getResultHandler(final AsyncFrameBuffer fb, final int seqid) { + final org.apache.thrift.AsyncProcessFunction fcall = this; + return new AsyncMethodCallback() { + public void onComplete(Void o) { + increment_result result = new increment_result(); + try { + fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); + return; + } catch (Exception e) { + LOGGER.error("Exception writing to internal frame buffer", e); + } + fb.close(); + } + public void onError(Exception e) { + byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; + org.apache.thrift.TBase msg; + increment_result result = new increment_result(); + if (e instanceof IOError) { + result.io = (IOError) e; + result.setIoIsSet(true); + msg = result; + } + else + { + msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION; + msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage()); + } + try { + fcall.sendResponse(fb,msg,msgType,seqid); + return; + } catch (Exception ex) { + LOGGER.error("Exception writing to internal frame buffer", ex); + } + fb.close(); + } + }; + } + + protected boolean isOneway() { + return false; + } + + public void start(I iface, increment_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws TException { + iface.increment(args.increment,resultHandler); } } - private static class incrementRows extends org.apache.thrift.ProcessFunction { + public static class incrementRows extends org.apache.thrift.AsyncProcessFunction { public incrementRows() { super("incrementRows"); } - protected incrementRows_args getEmptyArgsInstance() { + public incrementRows_args getEmptyArgsInstance() { return new incrementRows_args(); } - protected incrementRows_result getResult(I iface, incrementRows_args args) throws org.apache.thrift.TException { - incrementRows_result result = new incrementRows_result(); - try { - iface.incrementRows(args.increments); - } catch (IOError io) { - result.io = io; - } - return result; + public AsyncMethodCallback getResultHandler(final AsyncFrameBuffer fb, final int seqid) { + final org.apache.thrift.AsyncProcessFunction fcall = this; + return new AsyncMethodCallback() { + public void onComplete(Void o) { + incrementRows_result result = new incrementRows_result(); + try { + fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); + return; + } catch (Exception e) { + LOGGER.error("Exception writing to internal frame buffer", e); + } + fb.close(); + } + public void onError(Exception e) { + byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; + org.apache.thrift.TBase msg; + incrementRows_result result = new incrementRows_result(); + if (e instanceof IOError) { + result.io = (IOError) e; + result.setIoIsSet(true); + msg = result; + } + else + { + msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION; + msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage()); + } + try { + fcall.sendResponse(fb,msg,msgType,seqid); + return; + } catch (Exception ex) { + LOGGER.error("Exception writing to internal frame buffer", ex); + } + fb.close(); + } + }; + } + + protected boolean isOneway() { + return false; + } + + public void start(I iface, incrementRows_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws TException { + iface.incrementRows(args.increments,resultHandler); } } - private static class deleteAllRowTs extends org.apache.thrift.ProcessFunction { + public static class deleteAllRowTs extends org.apache.thrift.AsyncProcessFunction { public deleteAllRowTs() { super("deleteAllRowTs"); } - protected deleteAllRowTs_args getEmptyArgsInstance() { + public deleteAllRowTs_args getEmptyArgsInstance() { return new deleteAllRowTs_args(); } - protected deleteAllRowTs_result getResult(I iface, deleteAllRowTs_args args) throws org.apache.thrift.TException { - deleteAllRowTs_result result = new deleteAllRowTs_result(); - try { - iface.deleteAllRowTs(args.tableName, args.row, args.timestamp, args.attributes); - } catch (IOError io) { - result.io = io; - } - return result; + public AsyncMethodCallback getResultHandler(final AsyncFrameBuffer fb, final int seqid) { + final org.apache.thrift.AsyncProcessFunction fcall = this; + return new AsyncMethodCallback() { + public void onComplete(Void o) { + deleteAllRowTs_result result = new deleteAllRowTs_result(); + try { + fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); + return; + } catch (Exception e) { + LOGGER.error("Exception writing to internal frame buffer", e); + } + fb.close(); + } + public void onError(Exception e) { + byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; + org.apache.thrift.TBase msg; + deleteAllRowTs_result result = new deleteAllRowTs_result(); + if (e instanceof IOError) { + result.io = (IOError) e; + result.setIoIsSet(true); + msg = result; + } + else + { + msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION; + msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage()); + } + try { + fcall.sendResponse(fb,msg,msgType,seqid); + return; + } catch (Exception ex) { + LOGGER.error("Exception writing to internal frame buffer", ex); + } + fb.close(); + } + }; + } + + protected boolean isOneway() { + return false; + } + + public void start(I iface, deleteAllRowTs_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws TException { + iface.deleteAllRowTs(args.tableName, args.row, args.timestamp, args.attributes,resultHandler); } } - private static class scannerOpenWithScan extends org.apache.thrift.ProcessFunction { + public static class scannerOpenWithScan extends org.apache.thrift.AsyncProcessFunction { public scannerOpenWithScan() { super("scannerOpenWithScan"); } - protected scannerOpenWithScan_args getEmptyArgsInstance() { + public scannerOpenWithScan_args getEmptyArgsInstance() { return new scannerOpenWithScan_args(); } - protected scannerOpenWithScan_result getResult(I iface, scannerOpenWithScan_args args) throws org.apache.thrift.TException { - scannerOpenWithScan_result result = new scannerOpenWithScan_result(); - try { - result.success = iface.scannerOpenWithScan(args.tableName, args.scan, args.attributes); - result.setSuccessIsSet(true); - } catch (IOError io) { - result.io = io; - } - return result; + public AsyncMethodCallback getResultHandler(final AsyncFrameBuffer fb, final int seqid) { + final org.apache.thrift.AsyncProcessFunction fcall = this; + return new AsyncMethodCallback() { + public void onComplete(Integer o) { + scannerOpenWithScan_result result = new scannerOpenWithScan_result(); + result.success = o; + result.setSuccessIsSet(true); + try { + fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); + return; + } catch (Exception e) { + LOGGER.error("Exception writing to internal frame buffer", e); + } + fb.close(); + } + public void onError(Exception e) { + byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; + org.apache.thrift.TBase msg; + scannerOpenWithScan_result result = new scannerOpenWithScan_result(); + if (e instanceof IOError) { + result.io = (IOError) e; + result.setIoIsSet(true); + msg = result; + } + else + { + msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION; + msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage()); + } + try { + fcall.sendResponse(fb,msg,msgType,seqid); + return; + } catch (Exception ex) { + LOGGER.error("Exception writing to internal frame buffer", ex); + } + fb.close(); + } + }; + } + + protected boolean isOneway() { + return false; + } + + public void start(I iface, scannerOpenWithScan_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws TException { + iface.scannerOpenWithScan(args.tableName, args.scan, args.attributes,resultHandler); } } - private static class scannerOpen extends org.apache.thrift.ProcessFunction { + public static class scannerOpen extends org.apache.thrift.AsyncProcessFunction { public scannerOpen() { super("scannerOpen"); } - protected scannerOpen_args getEmptyArgsInstance() { + public scannerOpen_args getEmptyArgsInstance() { return new scannerOpen_args(); } - protected scannerOpen_result getResult(I iface, scannerOpen_args args) throws org.apache.thrift.TException { - scannerOpen_result result = new scannerOpen_result(); - try { - result.success = iface.scannerOpen(args.tableName, args.startRow, args.columns, args.attributes); - result.setSuccessIsSet(true); - } catch (IOError io) { - result.io = io; - } - return result; + public AsyncMethodCallback getResultHandler(final AsyncFrameBuffer fb, final int seqid) { + final org.apache.thrift.AsyncProcessFunction fcall = this; + return new AsyncMethodCallback() { + public void onComplete(Integer o) { + scannerOpen_result result = new scannerOpen_result(); + result.success = o; + result.setSuccessIsSet(true); + try { + fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); + return; + } catch (Exception e) { + LOGGER.error("Exception writing to internal frame buffer", e); + } + fb.close(); + } + public void onError(Exception e) { + byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; + org.apache.thrift.TBase msg; + scannerOpen_result result = new scannerOpen_result(); + if (e instanceof IOError) { + result.io = (IOError) e; + result.setIoIsSet(true); + msg = result; + } + else + { + msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION; + msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage()); + } + try { + fcall.sendResponse(fb,msg,msgType,seqid); + return; + } catch (Exception ex) { + LOGGER.error("Exception writing to internal frame buffer", ex); + } + fb.close(); + } + }; + } + + protected boolean isOneway() { + return false; + } + + public void start(I iface, scannerOpen_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws TException { + iface.scannerOpen(args.tableName, args.startRow, args.columns, args.attributes,resultHandler); } } - private static class scannerOpenWithStop extends org.apache.thrift.ProcessFunction { + public static class scannerOpenWithStop extends org.apache.thrift.AsyncProcessFunction { public scannerOpenWithStop() { super("scannerOpenWithStop"); } - protected scannerOpenWithStop_args getEmptyArgsInstance() { + public scannerOpenWithStop_args getEmptyArgsInstance() { return new scannerOpenWithStop_args(); } - protected scannerOpenWithStop_result getResult(I iface, scannerOpenWithStop_args args) throws org.apache.thrift.TException { - scannerOpenWithStop_result result = new scannerOpenWithStop_result(); - try { - result.success = iface.scannerOpenWithStop(args.tableName, args.startRow, args.stopRow, args.columns, args.attributes); - result.setSuccessIsSet(true); - } catch (IOError io) { - result.io = io; - } - return result; + public AsyncMethodCallback getResultHandler(final AsyncFrameBuffer fb, final int seqid) { + final org.apache.thrift.AsyncProcessFunction fcall = this; + return new AsyncMethodCallback() { + public void onComplete(Integer o) { + scannerOpenWithStop_result result = new scannerOpenWithStop_result(); + result.success = o; + result.setSuccessIsSet(true); + try { + fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); + return; + } catch (Exception e) { + LOGGER.error("Exception writing to internal frame buffer", e); + } + fb.close(); + } + public void onError(Exception e) { + byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; + org.apache.thrift.TBase msg; + scannerOpenWithStop_result result = new scannerOpenWithStop_result(); + if (e instanceof IOError) { + result.io = (IOError) e; + result.setIoIsSet(true); + msg = result; + } + else + { + msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION; + msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage()); + } + try { + fcall.sendResponse(fb,msg,msgType,seqid); + return; + } catch (Exception ex) { + LOGGER.error("Exception writing to internal frame buffer", ex); + } + fb.close(); + } + }; + } + + protected boolean isOneway() { + return false; + } + + public void start(I iface, scannerOpenWithStop_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws TException { + iface.scannerOpenWithStop(args.tableName, args.startRow, args.stopRow, args.columns, args.attributes,resultHandler); } } - private static class scannerOpenWithPrefix extends org.apache.thrift.ProcessFunction { + public static class scannerOpenWithPrefix extends org.apache.thrift.AsyncProcessFunction { public scannerOpenWithPrefix() { super("scannerOpenWithPrefix"); } - protected scannerOpenWithPrefix_args getEmptyArgsInstance() { + public scannerOpenWithPrefix_args getEmptyArgsInstance() { return new scannerOpenWithPrefix_args(); } - protected scannerOpenWithPrefix_result getResult(I iface, scannerOpenWithPrefix_args args) throws org.apache.thrift.TException { - scannerOpenWithPrefix_result result = new scannerOpenWithPrefix_result(); - try { - result.success = iface.scannerOpenWithPrefix(args.tableName, args.startAndPrefix, args.columns, args.attributes); - result.setSuccessIsSet(true); - } catch (IOError io) { - result.io = io; - } - return result; + public AsyncMethodCallback getResultHandler(final AsyncFrameBuffer fb, final int seqid) { + final org.apache.thrift.AsyncProcessFunction fcall = this; + return new AsyncMethodCallback() { + public void onComplete(Integer o) { + scannerOpenWithPrefix_result result = new scannerOpenWithPrefix_result(); + result.success = o; + result.setSuccessIsSet(true); + try { + fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); + return; + } catch (Exception e) { + LOGGER.error("Exception writing to internal frame buffer", e); + } + fb.close(); + } + public void onError(Exception e) { + byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; + org.apache.thrift.TBase msg; + scannerOpenWithPrefix_result result = new scannerOpenWithPrefix_result(); + if (e instanceof IOError) { + result.io = (IOError) e; + result.setIoIsSet(true); + msg = result; + } + else + { + msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION; + msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage()); + } + try { + fcall.sendResponse(fb,msg,msgType,seqid); + return; + } catch (Exception ex) { + LOGGER.error("Exception writing to internal frame buffer", ex); + } + fb.close(); + } + }; + } + + protected boolean isOneway() { + return false; + } + + public void start(I iface, scannerOpenWithPrefix_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws TException { + iface.scannerOpenWithPrefix(args.tableName, args.startAndPrefix, args.columns, args.attributes,resultHandler); } } - private static class scannerOpenTs extends org.apache.thrift.ProcessFunction { + public static class scannerOpenTs extends org.apache.thrift.AsyncProcessFunction { public scannerOpenTs() { super("scannerOpenTs"); } - protected scannerOpenTs_args getEmptyArgsInstance() { + public scannerOpenTs_args getEmptyArgsInstance() { return new scannerOpenTs_args(); } - protected scannerOpenTs_result getResult(I iface, scannerOpenTs_args args) throws org.apache.thrift.TException { - scannerOpenTs_result result = new scannerOpenTs_result(); - try { - result.success = iface.scannerOpenTs(args.tableName, args.startRow, args.columns, args.timestamp, args.attributes); - result.setSuccessIsSet(true); - } catch (IOError io) { - result.io = io; - } - return result; + public AsyncMethodCallback getResultHandler(final AsyncFrameBuffer fb, final int seqid) { + final org.apache.thrift.AsyncProcessFunction fcall = this; + return new AsyncMethodCallback() { + public void onComplete(Integer o) { + scannerOpenTs_result result = new scannerOpenTs_result(); + result.success = o; + result.setSuccessIsSet(true); + try { + fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); + return; + } catch (Exception e) { + LOGGER.error("Exception writing to internal frame buffer", e); + } + fb.close(); + } + public void onError(Exception e) { + byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; + org.apache.thrift.TBase msg; + scannerOpenTs_result result = new scannerOpenTs_result(); + if (e instanceof IOError) { + result.io = (IOError) e; + result.setIoIsSet(true); + msg = result; + } + else + { + msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION; + msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage()); + } + try { + fcall.sendResponse(fb,msg,msgType,seqid); + return; + } catch (Exception ex) { + LOGGER.error("Exception writing to internal frame buffer", ex); + } + fb.close(); + } + }; + } + + protected boolean isOneway() { + return false; + } + + public void start(I iface, scannerOpenTs_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws TException { + iface.scannerOpenTs(args.tableName, args.startRow, args.columns, args.timestamp, args.attributes,resultHandler); } } - private static class scannerOpenWithStopTs extends org.apache.thrift.ProcessFunction { + public static class scannerOpenWithStopTs extends org.apache.thrift.AsyncProcessFunction { public scannerOpenWithStopTs() { super("scannerOpenWithStopTs"); } - protected scannerOpenWithStopTs_args getEmptyArgsInstance() { + public scannerOpenWithStopTs_args getEmptyArgsInstance() { return new scannerOpenWithStopTs_args(); } - protected scannerOpenWithStopTs_result getResult(I iface, scannerOpenWithStopTs_args args) throws org.apache.thrift.TException { - scannerOpenWithStopTs_result result = new scannerOpenWithStopTs_result(); - try { - result.success = iface.scannerOpenWithStopTs(args.tableName, args.startRow, args.stopRow, args.columns, args.timestamp, args.attributes); - result.setSuccessIsSet(true); - } catch (IOError io) { - result.io = io; - } - return result; + public AsyncMethodCallback getResultHandler(final AsyncFrameBuffer fb, final int seqid) { + final org.apache.thrift.AsyncProcessFunction fcall = this; + return new AsyncMethodCallback() { + public void onComplete(Integer o) { + scannerOpenWithStopTs_result result = new scannerOpenWithStopTs_result(); + result.success = o; + result.setSuccessIsSet(true); + try { + fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); + return; + } catch (Exception e) { + LOGGER.error("Exception writing to internal frame buffer", e); + } + fb.close(); + } + public void onError(Exception e) { + byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; + org.apache.thrift.TBase msg; + scannerOpenWithStopTs_result result = new scannerOpenWithStopTs_result(); + if (e instanceof IOError) { + result.io = (IOError) e; + result.setIoIsSet(true); + msg = result; + } + else + { + msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION; + msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage()); + } + try { + fcall.sendResponse(fb,msg,msgType,seqid); + return; + } catch (Exception ex) { + LOGGER.error("Exception writing to internal frame buffer", ex); + } + fb.close(); + } + }; + } + + protected boolean isOneway() { + return false; + } + + public void start(I iface, scannerOpenWithStopTs_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws TException { + iface.scannerOpenWithStopTs(args.tableName, args.startRow, args.stopRow, args.columns, args.timestamp, args.attributes,resultHandler); } } - private static class scannerGet extends org.apache.thrift.ProcessFunction { + public static class scannerGet extends org.apache.thrift.AsyncProcessFunction> { public scannerGet() { super("scannerGet"); } - protected scannerGet_args getEmptyArgsInstance() { + public scannerGet_args getEmptyArgsInstance() { return new scannerGet_args(); } - protected scannerGet_result getResult(I iface, scannerGet_args args) throws org.apache.thrift.TException { - scannerGet_result result = new scannerGet_result(); - try { - result.success = iface.scannerGet(args.id); - } catch (IOError io) { - result.io = io; - } catch (IllegalArgument ia) { - result.ia = ia; - } - return result; + public AsyncMethodCallback> getResultHandler(final AsyncFrameBuffer fb, final int seqid) { + final org.apache.thrift.AsyncProcessFunction fcall = this; + return new AsyncMethodCallback>() { + public void onComplete(List o) { + scannerGet_result result = new scannerGet_result(); + result.success = o; + try { + fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); + return; + } catch (Exception e) { + LOGGER.error("Exception writing to internal frame buffer", e); + } + fb.close(); + } + public void onError(Exception e) { + byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; + org.apache.thrift.TBase msg; + scannerGet_result result = new scannerGet_result(); + if (e instanceof IOError) { + result.io = (IOError) e; + result.setIoIsSet(true); + msg = result; + } + else if (e instanceof IllegalArgument) { + result.ia = (IllegalArgument) e; + result.setIaIsSet(true); + msg = result; + } + else + { + msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION; + msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage()); + } + try { + fcall.sendResponse(fb,msg,msgType,seqid); + return; + } catch (Exception ex) { + LOGGER.error("Exception writing to internal frame buffer", ex); + } + fb.close(); + } + }; + } + + protected boolean isOneway() { + return false; + } + + public void start(I iface, scannerGet_args args, org.apache.thrift.async.AsyncMethodCallback> resultHandler) throws TException { + iface.scannerGet(args.id,resultHandler); } } - private static class scannerGetList extends org.apache.thrift.ProcessFunction { + public static class scannerGetList extends org.apache.thrift.AsyncProcessFunction> { public scannerGetList() { super("scannerGetList"); } - protected scannerGetList_args getEmptyArgsInstance() { + public scannerGetList_args getEmptyArgsInstance() { return new scannerGetList_args(); } - protected scannerGetList_result getResult(I iface, scannerGetList_args args) throws org.apache.thrift.TException { - scannerGetList_result result = new scannerGetList_result(); - try { - result.success = iface.scannerGetList(args.id, args.nbRows); - } catch (IOError io) { - result.io = io; - } catch (IllegalArgument ia) { - result.ia = ia; - } - return result; + public AsyncMethodCallback> getResultHandler(final AsyncFrameBuffer fb, final int seqid) { + final org.apache.thrift.AsyncProcessFunction fcall = this; + return new AsyncMethodCallback>() { + public void onComplete(List o) { + scannerGetList_result result = new scannerGetList_result(); + result.success = o; + try { + fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); + return; + } catch (Exception e) { + LOGGER.error("Exception writing to internal frame buffer", e); + } + fb.close(); + } + public void onError(Exception e) { + byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; + org.apache.thrift.TBase msg; + scannerGetList_result result = new scannerGetList_result(); + if (e instanceof IOError) { + result.io = (IOError) e; + result.setIoIsSet(true); + msg = result; + } + else if (e instanceof IllegalArgument) { + result.ia = (IllegalArgument) e; + result.setIaIsSet(true); + msg = result; + } + else + { + msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION; + msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage()); + } + try { + fcall.sendResponse(fb,msg,msgType,seqid); + return; + } catch (Exception ex) { + LOGGER.error("Exception writing to internal frame buffer", ex); + } + fb.close(); + } + }; + } + + protected boolean isOneway() { + return false; + } + + public void start(I iface, scannerGetList_args args, org.apache.thrift.async.AsyncMethodCallback> resultHandler) throws TException { + iface.scannerGetList(args.id, args.nbRows,resultHandler); } } - private static class scannerClose extends org.apache.thrift.ProcessFunction { + public static class scannerClose extends org.apache.thrift.AsyncProcessFunction { public scannerClose() { super("scannerClose"); } - protected scannerClose_args getEmptyArgsInstance() { + public scannerClose_args getEmptyArgsInstance() { return new scannerClose_args(); } - protected scannerClose_result getResult(I iface, scannerClose_args args) throws org.apache.thrift.TException { - scannerClose_result result = new scannerClose_result(); - try { - iface.scannerClose(args.id); - } catch (IOError io) { - result.io = io; - } catch (IllegalArgument ia) { - result.ia = ia; - } - return result; + public AsyncMethodCallback getResultHandler(final AsyncFrameBuffer fb, final int seqid) { + final org.apache.thrift.AsyncProcessFunction fcall = this; + return new AsyncMethodCallback() { + public void onComplete(Void o) { + scannerClose_result result = new scannerClose_result(); + try { + fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); + return; + } catch (Exception e) { + LOGGER.error("Exception writing to internal frame buffer", e); + } + fb.close(); + } + public void onError(Exception e) { + byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; + org.apache.thrift.TBase msg; + scannerClose_result result = new scannerClose_result(); + if (e instanceof IOError) { + result.io = (IOError) e; + result.setIoIsSet(true); + msg = result; + } + else if (e instanceof IllegalArgument) { + result.ia = (IllegalArgument) e; + result.setIaIsSet(true); + msg = result; + } + else + { + msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION; + msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage()); + } + try { + fcall.sendResponse(fb,msg,msgType,seqid); + return; + } catch (Exception ex) { + LOGGER.error("Exception writing to internal frame buffer", ex); + } + fb.close(); + } + }; + } + + protected boolean isOneway() { + return false; + } + + public void start(I iface, scannerClose_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws TException { + iface.scannerClose(args.id,resultHandler); } } - private static class getRowOrBefore extends org.apache.thrift.ProcessFunction { + public static class getRowOrBefore extends org.apache.thrift.AsyncProcessFunction> { public getRowOrBefore() { super("getRowOrBefore"); } - protected getRowOrBefore_args getEmptyArgsInstance() { + public getRowOrBefore_args getEmptyArgsInstance() { return new getRowOrBefore_args(); } - protected getRowOrBefore_result getResult(I iface, getRowOrBefore_args args) throws org.apache.thrift.TException { - getRowOrBefore_result result = new getRowOrBefore_result(); - try { - result.success = iface.getRowOrBefore(args.tableName, args.row, args.family); - } catch (IOError io) { - result.io = io; - } - return result; + public AsyncMethodCallback> getResultHandler(final AsyncFrameBuffer fb, final int seqid) { + final org.apache.thrift.AsyncProcessFunction fcall = this; + return new AsyncMethodCallback>() { + public void onComplete(List o) { + getRowOrBefore_result result = new getRowOrBefore_result(); + result.success = o; + try { + fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); + return; + } catch (Exception e) { + LOGGER.error("Exception writing to internal frame buffer", e); + } + fb.close(); + } + public void onError(Exception e) { + byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; + org.apache.thrift.TBase msg; + getRowOrBefore_result result = new getRowOrBefore_result(); + if (e instanceof IOError) { + result.io = (IOError) e; + result.setIoIsSet(true); + msg = result; + } + else + { + msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION; + msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage()); + } + try { + fcall.sendResponse(fb,msg,msgType,seqid); + return; + } catch (Exception ex) { + LOGGER.error("Exception writing to internal frame buffer", ex); + } + fb.close(); + } + }; + } + + protected boolean isOneway() { + return false; + } + + public void start(I iface, getRowOrBefore_args args, org.apache.thrift.async.AsyncMethodCallback> resultHandler) throws TException { + iface.getRowOrBefore(args.tableName, args.row, args.family,resultHandler); } } - private static class getRegionInfo extends org.apache.thrift.ProcessFunction { + public static class getRegionInfo extends org.apache.thrift.AsyncProcessFunction { public getRegionInfo() { super("getRegionInfo"); } - protected getRegionInfo_args getEmptyArgsInstance() { + public getRegionInfo_args getEmptyArgsInstance() { return new getRegionInfo_args(); } - protected getRegionInfo_result getResult(I iface, getRegionInfo_args args) throws org.apache.thrift.TException { - getRegionInfo_result result = new getRegionInfo_result(); - try { - result.success = iface.getRegionInfo(args.row); - } catch (IOError io) { - result.io = io; - } - return result; + public AsyncMethodCallback getResultHandler(final AsyncFrameBuffer fb, final int seqid) { + final org.apache.thrift.AsyncProcessFunction fcall = this; + return new AsyncMethodCallback() { + public void onComplete(TRegionInfo o) { + getRegionInfo_result result = new getRegionInfo_result(); + result.success = o; + try { + fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); + return; + } catch (Exception e) { + LOGGER.error("Exception writing to internal frame buffer", e); + } + fb.close(); + } + public void onError(Exception e) { + byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; + org.apache.thrift.TBase msg; + getRegionInfo_result result = new getRegionInfo_result(); + if (e instanceof IOError) { + result.io = (IOError) e; + result.setIoIsSet(true); + msg = result; + } + else + { + msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION; + msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage()); + } + try { + fcall.sendResponse(fb,msg,msgType,seqid); + return; + } catch (Exception ex) { + LOGGER.error("Exception writing to internal frame buffer", ex); + } + fb.close(); + } + }; + } + + protected boolean isOneway() { + return false; + } + + public void start(I iface, getRegionInfo_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws TException { + iface.getRegionInfo(args.row,resultHandler); } } } - public static class enableTable_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + public static class enableTable_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("enableTable_args"); private static final org.apache.thrift.protocol.TField TABLE_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("tableName", org.apache.thrift.protocol.TType.STRING, (short)1); @@ -4736,20 +7464,20 @@ public class Hbase { return 0; } + @Override public int compareTo(enableTable_args other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; - enableTable_args typedOther = (enableTable_args)other; - lastComparison = Boolean.valueOf(isSetTableName()).compareTo(typedOther.isSetTableName()); + lastComparison = Boolean.valueOf(isSetTableName()).compareTo(other.isSetTableName()); if (lastComparison != 0) { return lastComparison; } if (isSetTableName()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tableName, typedOther.tableName); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tableName, other.tableName); if (lastComparison != 0) { return lastComparison; } @@ -4787,6 +7515,7 @@ public class Hbase { 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 { @@ -4891,7 +7620,7 @@ public class Hbase { } - public static class enableTable_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + public static class enableTable_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("enableTable_result"); private static final org.apache.thrift.protocol.TField IO_FIELD_DESC = new org.apache.thrift.protocol.TField("io", org.apache.thrift.protocol.TType.STRUCT, (short)1); @@ -5089,20 +7818,20 @@ public class Hbase { return 0; } + @Override public int compareTo(enableTable_result other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; - enableTable_result typedOther = (enableTable_result)other; - lastComparison = Boolean.valueOf(isSetIo()).compareTo(typedOther.isSetIo()); + lastComparison = Boolean.valueOf(isSetIo()).compareTo(other.isSetIo()); if (lastComparison != 0) { return lastComparison; } if (isSetIo()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.io, typedOther.io); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.io, other.io); if (lastComparison != 0) { return lastComparison; } @@ -5140,6 +7869,7 @@ public class Hbase { 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 { @@ -5246,7 +7976,7 @@ public class Hbase { } - public static class disableTable_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + public static class disableTable_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("disableTable_args"); private static final org.apache.thrift.protocol.TField TABLE_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("tableName", org.apache.thrift.protocol.TType.STRING, (short)1); @@ -5466,20 +8196,20 @@ public class Hbase { return 0; } + @Override public int compareTo(disableTable_args other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; - disableTable_args typedOther = (disableTable_args)other; - lastComparison = Boolean.valueOf(isSetTableName()).compareTo(typedOther.isSetTableName()); + lastComparison = Boolean.valueOf(isSetTableName()).compareTo(other.isSetTableName()); if (lastComparison != 0) { return lastComparison; } if (isSetTableName()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tableName, typedOther.tableName); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tableName, other.tableName); if (lastComparison != 0) { return lastComparison; } @@ -5517,6 +8247,7 @@ public class Hbase { 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 { @@ -5621,7 +8352,7 @@ public class Hbase { } - public static class disableTable_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + public static class disableTable_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("disableTable_result"); private static final org.apache.thrift.protocol.TField IO_FIELD_DESC = new org.apache.thrift.protocol.TField("io", org.apache.thrift.protocol.TType.STRUCT, (short)1); @@ -5819,20 +8550,20 @@ public class Hbase { return 0; } + @Override public int compareTo(disableTable_result other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; - disableTable_result typedOther = (disableTable_result)other; - lastComparison = Boolean.valueOf(isSetIo()).compareTo(typedOther.isSetIo()); + lastComparison = Boolean.valueOf(isSetIo()).compareTo(other.isSetIo()); if (lastComparison != 0) { return lastComparison; } if (isSetIo()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.io, typedOther.io); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.io, other.io); if (lastComparison != 0) { return lastComparison; } @@ -5870,6 +8601,7 @@ public class Hbase { 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 { @@ -5976,7 +8708,7 @@ public class Hbase { } - public static class isTableEnabled_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + public static class isTableEnabled_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("isTableEnabled_args"); private static final org.apache.thrift.protocol.TField TABLE_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("tableName", org.apache.thrift.protocol.TType.STRING, (short)1); @@ -6196,20 +8928,20 @@ public class Hbase { return 0; } + @Override public int compareTo(isTableEnabled_args other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; - isTableEnabled_args typedOther = (isTableEnabled_args)other; - lastComparison = Boolean.valueOf(isSetTableName()).compareTo(typedOther.isSetTableName()); + lastComparison = Boolean.valueOf(isSetTableName()).compareTo(other.isSetTableName()); if (lastComparison != 0) { return lastComparison; } if (isSetTableName()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tableName, typedOther.tableName); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tableName, other.tableName); if (lastComparison != 0) { return lastComparison; } @@ -6247,6 +8979,7 @@ public class Hbase { 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 { @@ -6351,7 +9084,7 @@ public class Hbase { } - public static class isTableEnabled_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + public static class isTableEnabled_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("isTableEnabled_result"); private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.BOOL, (short)0); @@ -6429,7 +9162,7 @@ public class Hbase { // isset id assignments private static final int __SUCCESS_ISSET_ID = 0; - private BitSet __isset_bit_vector = new BitSet(1); + 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); @@ -6458,8 +9191,7 @@ public class Hbase { * Performs a deep copy on other. */ public isTableEnabled_result(isTableEnabled_result other) { - __isset_bit_vector.clear(); - __isset_bit_vector.or(other.__isset_bit_vector); + __isset_bitfield = other.__isset_bitfield; this.success = other.success; if (other.isSetIo()) { this.io = new IOError(other.io); @@ -6488,16 +9220,16 @@ public class Hbase { } public void unsetSuccess() { - __isset_bit_vector.clear(__SUCCESS_ISSET_ID); + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __SUCCESS_ISSET_ID); } /** Returns true if field success is set (has been assigned a value) and false otherwise */ public boolean isSetSuccess() { - return __isset_bit_vector.get(__SUCCESS_ISSET_ID); + return EncodingUtils.testBit(__isset_bitfield, __SUCCESS_ISSET_ID); } public void setSuccessIsSet(boolean value) { - __isset_bit_vector.set(__SUCCESS_ISSET_ID, value); + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __SUCCESS_ISSET_ID, value); } public IOError getIo() { @@ -6611,30 +9343,30 @@ public class Hbase { return 0; } + @Override public int compareTo(isTableEnabled_result other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; - isTableEnabled_result typedOther = (isTableEnabled_result)other; - lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(typedOther.isSetSuccess()); + lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(other.isSetSuccess()); if (lastComparison != 0) { return lastComparison; } if (isSetSuccess()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, typedOther.success); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetIo()).compareTo(typedOther.isSetIo()); + lastComparison = Boolean.valueOf(isSetIo()).compareTo(other.isSetIo()); if (lastComparison != 0) { return lastComparison; } if (isSetIo()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.io, typedOther.io); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.io, other.io); if (lastComparison != 0) { return lastComparison; } @@ -6676,6 +9408,7 @@ public class Hbase { 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 { @@ -6689,7 +9422,7 @@ public class Hbase { 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_bit_vector = new BitSet(1); + __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); @@ -6746,9 +9479,11 @@ public class Hbase { struct.validate(); oprot.writeStructBegin(STRUCT_DESC); - oprot.writeFieldBegin(SUCCESS_FIELD_DESC); - oprot.writeBool(struct.success); - oprot.writeFieldEnd(); + if (struct.isSetSuccess()) { + oprot.writeFieldBegin(SUCCESS_FIELD_DESC); + oprot.writeBool(struct.success); + oprot.writeFieldEnd(); + } if (struct.io != null) { oprot.writeFieldBegin(IO_FIELD_DESC); struct.io.write(oprot); @@ -6805,7 +9540,7 @@ public class Hbase { } - public static class compact_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + public static class compact_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { 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 TABLE_NAME_OR_REGION_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("tableNameOrRegionName", org.apache.thrift.protocol.TType.STRING, (short)1); @@ -7013,20 +9748,20 @@ public class Hbase { return 0; } + @Override 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(isSetTableNameOrRegionName()).compareTo(typedOther.isSetTableNameOrRegionName()); + lastComparison = Boolean.valueOf(isSetTableNameOrRegionName()).compareTo(other.isSetTableNameOrRegionName()); if (lastComparison != 0) { return lastComparison; } if (isSetTableNameOrRegionName()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tableNameOrRegionName, typedOther.tableNameOrRegionName); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tableNameOrRegionName, other.tableNameOrRegionName); if (lastComparison != 0) { return lastComparison; } @@ -7064,6 +9799,7 @@ public class Hbase { 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 { @@ -7168,7 +9904,7 @@ public class Hbase { } - public static class compact_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + public static class compact_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("compact_result"); private static final org.apache.thrift.protocol.TField IO_FIELD_DESC = new org.apache.thrift.protocol.TField("io", org.apache.thrift.protocol.TType.STRUCT, (short)1); @@ -7366,20 +10102,20 @@ public class Hbase { return 0; } + @Override 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; - lastComparison = Boolean.valueOf(isSetIo()).compareTo(typedOther.isSetIo()); + lastComparison = Boolean.valueOf(isSetIo()).compareTo(other.isSetIo()); if (lastComparison != 0) { return lastComparison; } if (isSetIo()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.io, typedOther.io); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.io, other.io); if (lastComparison != 0) { return lastComparison; } @@ -7417,6 +10153,7 @@ public class Hbase { 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 { @@ -7523,7 +10260,7 @@ public class Hbase { } - public static class majorCompact_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + public static class majorCompact_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("majorCompact_args"); private static final org.apache.thrift.protocol.TField TABLE_NAME_OR_REGION_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("tableNameOrRegionName", org.apache.thrift.protocol.TType.STRING, (short)1); @@ -7731,20 +10468,20 @@ public class Hbase { return 0; } + @Override public int compareTo(majorCompact_args other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; - majorCompact_args typedOther = (majorCompact_args)other; - lastComparison = Boolean.valueOf(isSetTableNameOrRegionName()).compareTo(typedOther.isSetTableNameOrRegionName()); + lastComparison = Boolean.valueOf(isSetTableNameOrRegionName()).compareTo(other.isSetTableNameOrRegionName()); if (lastComparison != 0) { return lastComparison; } if (isSetTableNameOrRegionName()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tableNameOrRegionName, typedOther.tableNameOrRegionName); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tableNameOrRegionName, other.tableNameOrRegionName); if (lastComparison != 0) { return lastComparison; } @@ -7782,6 +10519,7 @@ public class Hbase { 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 { @@ -7886,7 +10624,7 @@ public class Hbase { } - public static class majorCompact_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + public static class majorCompact_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("majorCompact_result"); private static final org.apache.thrift.protocol.TField IO_FIELD_DESC = new org.apache.thrift.protocol.TField("io", org.apache.thrift.protocol.TType.STRUCT, (short)1); @@ -8084,20 +10822,20 @@ public class Hbase { return 0; } + @Override public int compareTo(majorCompact_result other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; - majorCompact_result typedOther = (majorCompact_result)other; - lastComparison = Boolean.valueOf(isSetIo()).compareTo(typedOther.isSetIo()); + lastComparison = Boolean.valueOf(isSetIo()).compareTo(other.isSetIo()); if (lastComparison != 0) { return lastComparison; } if (isSetIo()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.io, typedOther.io); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.io, other.io); if (lastComparison != 0) { return lastComparison; } @@ -8135,6 +10873,7 @@ public class Hbase { 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 { @@ -8241,7 +10980,7 @@ public class Hbase { } - public static class getTableNames_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + public static class getTableNames_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getTableNames_args"); @@ -8374,13 +11113,13 @@ public class Hbase { return 0; } + @Override public int compareTo(getTableNames_args other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; - getTableNames_args typedOther = (getTableNames_args)other; return 0; } @@ -8408,6 +11147,7 @@ public class Hbase { 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 { @@ -8486,7 +11226,7 @@ public class Hbase { } - public static class getTableNames_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + public static class getTableNames_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getTableNames_result"); private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.LIST, (short)0); @@ -8592,7 +11332,7 @@ public class Hbase { */ public getTableNames_result(getTableNames_result other) { if (other.isSetSuccess()) { - List __this__success = new ArrayList(); + List __this__success = new ArrayList(other.success.size()); for (ByteBuffer other_element : other.success) { __this__success.add(other_element); } @@ -8763,30 +11503,30 @@ public class Hbase { return 0; } + @Override public int compareTo(getTableNames_result other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; - getTableNames_result typedOther = (getTableNames_result)other; - lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(typedOther.isSetSuccess()); + lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(other.isSetSuccess()); if (lastComparison != 0) { return lastComparison; } if (isSetSuccess()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, typedOther.success); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetIo()).compareTo(typedOther.isSetIo()); + lastComparison = Boolean.valueOf(isSetIo()).compareTo(other.isSetIo()); if (lastComparison != 0) { return lastComparison; } if (isSetIo()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.io, typedOther.io); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.io, other.io); if (lastComparison != 0) { return lastComparison; } @@ -8832,6 +11572,7 @@ public class Hbase { 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 { @@ -8871,13 +11612,13 @@ public class Hbase { case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list26 = iprot.readListBegin(); - struct.success = new ArrayList(_list26.size); - for (int _i27 = 0; _i27 < _list26.size; ++_i27) + org.apache.thrift.protocol.TList _list34 = iprot.readListBegin(); + struct.success = new ArrayList(_list34.size); + for (int _i35 = 0; _i35 < _list34.size; ++_i35) { - ByteBuffer _elem28; // optional - _elem28 = iprot.readBinary(); - struct.success.add(_elem28); + ByteBuffer _elem36; + _elem36 = iprot.readBinary(); + struct.success.add(_elem36); } iprot.readListEnd(); } @@ -8914,9 +11655,9 @@ public class Hbase { oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size())); - for (ByteBuffer _iter29 : struct.success) + for (ByteBuffer _iter37 : struct.success) { - oprot.writeBinary(_iter29); + oprot.writeBinary(_iter37); } oprot.writeListEnd(); } @@ -8955,9 +11696,9 @@ public class Hbase { if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (ByteBuffer _iter30 : struct.success) + for (ByteBuffer _iter38 : struct.success) { - oprot.writeBinary(_iter30); + oprot.writeBinary(_iter38); } } } @@ -8972,13 +11713,13 @@ public class Hbase { BitSet incoming = iprot.readBitSet(2); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list31 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); - struct.success = new ArrayList(_list31.size); - for (int _i32 = 0; _i32 < _list31.size; ++_i32) + org.apache.thrift.protocol.TList _list39 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); + struct.success = new ArrayList(_list39.size); + for (int _i40 = 0; _i40 < _list39.size; ++_i40) { - ByteBuffer _elem33; // optional - _elem33 = iprot.readBinary(); - struct.success.add(_elem33); + ByteBuffer _elem41; + _elem41 = iprot.readBinary(); + struct.success.add(_elem41); } } struct.setSuccessIsSet(true); @@ -8993,7 +11734,7 @@ public class Hbase { } - public static class getColumnDescriptors_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + public static class getColumnDescriptors_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getColumnDescriptors_args"); private static final org.apache.thrift.protocol.TField TABLE_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("tableName", org.apache.thrift.protocol.TType.STRING, (short)1); @@ -9213,20 +11954,20 @@ public class Hbase { return 0; } + @Override public int compareTo(getColumnDescriptors_args other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; - getColumnDescriptors_args typedOther = (getColumnDescriptors_args)other; - lastComparison = Boolean.valueOf(isSetTableName()).compareTo(typedOther.isSetTableName()); + lastComparison = Boolean.valueOf(isSetTableName()).compareTo(other.isSetTableName()); if (lastComparison != 0) { return lastComparison; } if (isSetTableName()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tableName, typedOther.tableName); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tableName, other.tableName); if (lastComparison != 0) { return lastComparison; } @@ -9264,6 +12005,7 @@ public class Hbase { 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 { @@ -9368,7 +12110,7 @@ public class Hbase { } - public static class getColumnDescriptors_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + public static class getColumnDescriptors_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getColumnDescriptors_result"); private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.MAP, (short)0); @@ -9475,7 +12217,7 @@ public class Hbase { */ public getColumnDescriptors_result(getColumnDescriptors_result other) { if (other.isSetSuccess()) { - Map __this__success = new HashMap(); + Map __this__success = new HashMap(other.success.size()); for (Map.Entry other_element : other.success.entrySet()) { ByteBuffer other_element_key = other_element.getKey(); @@ -9650,30 +12392,30 @@ public class Hbase { return 0; } + @Override public int compareTo(getColumnDescriptors_result other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; - getColumnDescriptors_result typedOther = (getColumnDescriptors_result)other; - lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(typedOther.isSetSuccess()); + lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(other.isSetSuccess()); if (lastComparison != 0) { return lastComparison; } if (isSetSuccess()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, typedOther.success); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetIo()).compareTo(typedOther.isSetIo()); + lastComparison = Boolean.valueOf(isSetIo()).compareTo(other.isSetIo()); if (lastComparison != 0) { return lastComparison; } if (isSetIo()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.io, typedOther.io); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.io, other.io); if (lastComparison != 0) { return lastComparison; } @@ -9719,6 +12461,7 @@ public class Hbase { 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 { @@ -9758,16 +12501,16 @@ public class Hbase { case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { { - org.apache.thrift.protocol.TMap _map34 = iprot.readMapBegin(); - struct.success = new HashMap(2*_map34.size); - for (int _i35 = 0; _i35 < _map34.size; ++_i35) + org.apache.thrift.protocol.TMap _map42 = iprot.readMapBegin(); + struct.success = new HashMap(2*_map42.size); + for (int _i43 = 0; _i43 < _map42.size; ++_i43) { - ByteBuffer _key36; // required - ColumnDescriptor _val37; // optional - _key36 = iprot.readBinary(); - _val37 = new ColumnDescriptor(); - _val37.read(iprot); - struct.success.put(_key36, _val37); + ByteBuffer _key44; + ColumnDescriptor _val45; + _key44 = iprot.readBinary(); + _val45 = new ColumnDescriptor(); + _val45.read(iprot); + struct.success.put(_key44, _val45); } iprot.readMapEnd(); } @@ -9804,10 +12547,10 @@ public class Hbase { 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 _iter38 : struct.success.entrySet()) + for (Map.Entry _iter46 : struct.success.entrySet()) { - oprot.writeBinary(_iter38.getKey()); - _iter38.getValue().write(oprot); + oprot.writeBinary(_iter46.getKey()); + _iter46.getValue().write(oprot); } oprot.writeMapEnd(); } @@ -9846,10 +12589,10 @@ public class Hbase { if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (Map.Entry _iter39 : struct.success.entrySet()) + for (Map.Entry _iter47 : struct.success.entrySet()) { - oprot.writeBinary(_iter39.getKey()); - _iter39.getValue().write(oprot); + oprot.writeBinary(_iter47.getKey()); + _iter47.getValue().write(oprot); } } } @@ -9864,16 +12607,16 @@ public class Hbase { BitSet incoming = iprot.readBitSet(2); if (incoming.get(0)) { { - org.apache.thrift.protocol.TMap _map40 = 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*_map40.size); - for (int _i41 = 0; _i41 < _map40.size; ++_i41) + org.apache.thrift.protocol.TMap _map48 = 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*_map48.size); + for (int _i49 = 0; _i49 < _map48.size; ++_i49) { - ByteBuffer _key42; // required - ColumnDescriptor _val43; // optional - _key42 = iprot.readBinary(); - _val43 = new ColumnDescriptor(); - _val43.read(iprot); - struct.success.put(_key42, _val43); + ByteBuffer _key50; + ColumnDescriptor _val51; + _key50 = iprot.readBinary(); + _val51 = new ColumnDescriptor(); + _val51.read(iprot); + struct.success.put(_key50, _val51); } } struct.setSuccessIsSet(true); @@ -9888,7 +12631,7 @@ public class Hbase { } - public static class getTableRegions_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + public static class getTableRegions_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getTableRegions_args"); private static final org.apache.thrift.protocol.TField TABLE_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("tableName", org.apache.thrift.protocol.TType.STRING, (short)1); @@ -10108,20 +12851,20 @@ public class Hbase { return 0; } + @Override public int compareTo(getTableRegions_args other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; - getTableRegions_args typedOther = (getTableRegions_args)other; - lastComparison = Boolean.valueOf(isSetTableName()).compareTo(typedOther.isSetTableName()); + lastComparison = Boolean.valueOf(isSetTableName()).compareTo(other.isSetTableName()); if (lastComparison != 0) { return lastComparison; } if (isSetTableName()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tableName, typedOther.tableName); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tableName, other.tableName); if (lastComparison != 0) { return lastComparison; } @@ -10159,6 +12902,7 @@ public class Hbase { 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 { @@ -10263,7 +13007,7 @@ public class Hbase { } - public static class getTableRegions_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + public static class getTableRegions_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getTableRegions_result"); private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.LIST, (short)0); @@ -10369,7 +13113,7 @@ public class Hbase { */ public getTableRegions_result(getTableRegions_result other) { if (other.isSetSuccess()) { - List __this__success = new ArrayList(); + List __this__success = new ArrayList(other.success.size()); for (TRegionInfo other_element : other.success) { __this__success.add(new TRegionInfo(other_element)); } @@ -10540,30 +13284,30 @@ public class Hbase { return 0; } + @Override public int compareTo(getTableRegions_result other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; - getTableRegions_result typedOther = (getTableRegions_result)other; - lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(typedOther.isSetSuccess()); + lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(other.isSetSuccess()); if (lastComparison != 0) { return lastComparison; } if (isSetSuccess()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, typedOther.success); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetIo()).compareTo(typedOther.isSetIo()); + lastComparison = Boolean.valueOf(isSetIo()).compareTo(other.isSetIo()); if (lastComparison != 0) { return lastComparison; } if (isSetIo()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.io, typedOther.io); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.io, other.io); if (lastComparison != 0) { return lastComparison; } @@ -10609,6 +13353,7 @@ public class Hbase { 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 { @@ -10648,14 +13393,14 @@ public class Hbase { case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list44 = iprot.readListBegin(); - struct.success = new ArrayList(_list44.size); - for (int _i45 = 0; _i45 < _list44.size; ++_i45) + org.apache.thrift.protocol.TList _list52 = iprot.readListBegin(); + struct.success = new ArrayList(_list52.size); + for (int _i53 = 0; _i53 < _list52.size; ++_i53) { - TRegionInfo _elem46; // optional - _elem46 = new TRegionInfo(); - _elem46.read(iprot); - struct.success.add(_elem46); + TRegionInfo _elem54; + _elem54 = new TRegionInfo(); + _elem54.read(iprot); + struct.success.add(_elem54); } iprot.readListEnd(); } @@ -10692,9 +13437,9 @@ public class Hbase { oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size())); - for (TRegionInfo _iter47 : struct.success) + for (TRegionInfo _iter55 : struct.success) { - _iter47.write(oprot); + _iter55.write(oprot); } oprot.writeListEnd(); } @@ -10733,9 +13478,9 @@ public class Hbase { if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (TRegionInfo _iter48 : struct.success) + for (TRegionInfo _iter56 : struct.success) { - _iter48.write(oprot); + _iter56.write(oprot); } } } @@ -10750,14 +13495,14 @@ public class Hbase { BitSet incoming = iprot.readBitSet(2); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list49 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); - struct.success = new ArrayList(_list49.size); - for (int _i50 = 0; _i50 < _list49.size; ++_i50) + org.apache.thrift.protocol.TList _list57 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); + struct.success = new ArrayList(_list57.size); + for (int _i58 = 0; _i58 < _list57.size; ++_i58) { - TRegionInfo _elem51; // optional - _elem51 = new TRegionInfo(); - _elem51.read(iprot); - struct.success.add(_elem51); + TRegionInfo _elem59; + _elem59 = new TRegionInfo(); + _elem59.read(iprot); + struct.success.add(_elem59); } } struct.setSuccessIsSet(true); @@ -10772,7 +13517,7 @@ public class Hbase { } - public static class createTable_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + public static class createTable_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("createTable_args"); private static final org.apache.thrift.protocol.TField TABLE_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("tableName", org.apache.thrift.protocol.TType.STRING, (short)1); @@ -10893,7 +13638,7 @@ public class Hbase { this.tableName = other.tableName; } if (other.isSetColumnFamilies()) { - List __this__columnFamilies = new ArrayList(); + List __this__columnFamilies = new ArrayList(other.columnFamilies.size()); for (ColumnDescriptor other_element : other.columnFamilies) { __this__columnFamilies.add(new ColumnDescriptor(other_element)); } @@ -11083,30 +13828,30 @@ public class Hbase { return 0; } + @Override public int compareTo(createTable_args other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; - createTable_args typedOther = (createTable_args)other; - lastComparison = Boolean.valueOf(isSetTableName()).compareTo(typedOther.isSetTableName()); + lastComparison = Boolean.valueOf(isSetTableName()).compareTo(other.isSetTableName()); if (lastComparison != 0) { return lastComparison; } if (isSetTableName()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tableName, typedOther.tableName); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tableName, other.tableName); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetColumnFamilies()).compareTo(typedOther.isSetColumnFamilies()); + lastComparison = Boolean.valueOf(isSetColumnFamilies()).compareTo(other.isSetColumnFamilies()); if (lastComparison != 0) { return lastComparison; } if (isSetColumnFamilies()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.columnFamilies, typedOther.columnFamilies); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.columnFamilies, other.columnFamilies); if (lastComparison != 0) { return lastComparison; } @@ -11152,6 +13897,7 @@ public class Hbase { 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 { @@ -11199,14 +13945,14 @@ public class Hbase { case 2: // COLUMN_FAMILIES if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list52 = iprot.readListBegin(); - struct.columnFamilies = new ArrayList(_list52.size); - for (int _i53 = 0; _i53 < _list52.size; ++_i53) + org.apache.thrift.protocol.TList _list60 = iprot.readListBegin(); + struct.columnFamilies = new ArrayList(_list60.size); + for (int _i61 = 0; _i61 < _list60.size; ++_i61) { - ColumnDescriptor _elem54; // optional - _elem54 = new ColumnDescriptor(); - _elem54.read(iprot); - struct.columnFamilies.add(_elem54); + ColumnDescriptor _elem62; + _elem62 = new ColumnDescriptor(); + _elem62.read(iprot); + struct.columnFamilies.add(_elem62); } iprot.readListEnd(); } @@ -11239,9 +13985,9 @@ public class Hbase { oprot.writeFieldBegin(COLUMN_FAMILIES_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.columnFamilies.size())); - for (ColumnDescriptor _iter55 : struct.columnFamilies) + for (ColumnDescriptor _iter63 : struct.columnFamilies) { - _iter55.write(oprot); + _iter63.write(oprot); } oprot.writeListEnd(); } @@ -11278,9 +14024,9 @@ public class Hbase { if (struct.isSetColumnFamilies()) { { oprot.writeI32(struct.columnFamilies.size()); - for (ColumnDescriptor _iter56 : struct.columnFamilies) + for (ColumnDescriptor _iter64 : struct.columnFamilies) { - _iter56.write(oprot); + _iter64.write(oprot); } } } @@ -11296,14 +14042,14 @@ public class Hbase { } if (incoming.get(1)) { { - org.apache.thrift.protocol.TList _list57 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); - struct.columnFamilies = new ArrayList(_list57.size); - for (int _i58 = 0; _i58 < _list57.size; ++_i58) + org.apache.thrift.protocol.TList _list65 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); + struct.columnFamilies = new ArrayList(_list65.size); + for (int _i66 = 0; _i66 < _list65.size; ++_i66) { - ColumnDescriptor _elem59; // optional - _elem59 = new ColumnDescriptor(); - _elem59.read(iprot); - struct.columnFamilies.add(_elem59); + ColumnDescriptor _elem67; + _elem67 = new ColumnDescriptor(); + _elem67.read(iprot); + struct.columnFamilies.add(_elem67); } } struct.setColumnFamiliesIsSet(true); @@ -11313,7 +14059,7 @@ public class Hbase { } - public static class createTable_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + public static class createTable_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("createTable_result"); private static final org.apache.thrift.protocol.TField IO_FIELD_DESC = new org.apache.thrift.protocol.TField("io", org.apache.thrift.protocol.TType.STRUCT, (short)1); @@ -11629,40 +14375,40 @@ public class Hbase { return 0; } + @Override public int compareTo(createTable_result other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; - createTable_result typedOther = (createTable_result)other; - lastComparison = Boolean.valueOf(isSetIo()).compareTo(typedOther.isSetIo()); + lastComparison = Boolean.valueOf(isSetIo()).compareTo(other.isSetIo()); if (lastComparison != 0) { return lastComparison; } if (isSetIo()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.io, typedOther.io); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.io, other.io); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetIa()).compareTo(typedOther.isSetIa()); + lastComparison = Boolean.valueOf(isSetIa()).compareTo(other.isSetIa()); if (lastComparison != 0) { return lastComparison; } if (isSetIa()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.ia, typedOther.ia); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.ia, other.ia); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetExist()).compareTo(typedOther.isSetExist()); + lastComparison = Boolean.valueOf(isSetExist()).compareTo(other.isSetExist()); if (lastComparison != 0) { return lastComparison; } if (isSetExist()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.exist, typedOther.exist); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.exist, other.exist); if (lastComparison != 0) { return lastComparison; } @@ -11716,6 +14462,7 @@ public class Hbase { 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 { @@ -11872,7 +14619,7 @@ public class Hbase { } - public static class deleteTable_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + public static class deleteTable_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("deleteTable_args"); private static final org.apache.thrift.protocol.TField TABLE_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("tableName", org.apache.thrift.protocol.TType.STRING, (short)1); @@ -12092,20 +14839,20 @@ public class Hbase { return 0; } + @Override public int compareTo(deleteTable_args other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; - deleteTable_args typedOther = (deleteTable_args)other; - lastComparison = Boolean.valueOf(isSetTableName()).compareTo(typedOther.isSetTableName()); + lastComparison = Boolean.valueOf(isSetTableName()).compareTo(other.isSetTableName()); if (lastComparison != 0) { return lastComparison; } if (isSetTableName()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tableName, typedOther.tableName); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tableName, other.tableName); if (lastComparison != 0) { return lastComparison; } @@ -12143,6 +14890,7 @@ public class Hbase { 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 { @@ -12247,7 +14995,7 @@ public class Hbase { } - public static class deleteTable_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + public static class deleteTable_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("deleteTable_result"); private static final org.apache.thrift.protocol.TField IO_FIELD_DESC = new org.apache.thrift.protocol.TField("io", org.apache.thrift.protocol.TType.STRUCT, (short)1); @@ -12445,20 +15193,20 @@ public class Hbase { return 0; } + @Override public int compareTo(deleteTable_result other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; - deleteTable_result typedOther = (deleteTable_result)other; - lastComparison = Boolean.valueOf(isSetIo()).compareTo(typedOther.isSetIo()); + lastComparison = Boolean.valueOf(isSetIo()).compareTo(other.isSetIo()); if (lastComparison != 0) { return lastComparison; } if (isSetIo()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.io, typedOther.io); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.io, other.io); if (lastComparison != 0) { return lastComparison; } @@ -12496,6 +15244,7 @@ public class Hbase { 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 { @@ -12602,7 +15351,7 @@ public class Hbase { } - public static class get_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + public static class get_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("get_args"); private static final org.apache.thrift.protocol.TField TABLE_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("tableName", org.apache.thrift.protocol.TType.STRING, (short)1); @@ -12760,7 +15509,7 @@ public class Hbase { this.column = other.column; } if (other.isSetAttributes()) { - Map __this__attributes = new HashMap(); + Map __this__attributes = new HashMap(other.attributes.size()); for (Map.Entry other_element : other.attributes.entrySet()) { ByteBuffer other_element_key = other_element.getKey(); @@ -13080,50 +15829,50 @@ public class Hbase { return 0; } + @Override public int compareTo(get_args other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; - get_args typedOther = (get_args)other; - lastComparison = Boolean.valueOf(isSetTableName()).compareTo(typedOther.isSetTableName()); + lastComparison = Boolean.valueOf(isSetTableName()).compareTo(other.isSetTableName()); if (lastComparison != 0) { return lastComparison; } if (isSetTableName()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tableName, typedOther.tableName); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tableName, other.tableName); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetRow()).compareTo(typedOther.isSetRow()); + lastComparison = Boolean.valueOf(isSetRow()).compareTo(other.isSetRow()); if (lastComparison != 0) { return lastComparison; } if (isSetRow()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.row, typedOther.row); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.row, other.row); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetColumn()).compareTo(typedOther.isSetColumn()); + lastComparison = Boolean.valueOf(isSetColumn()).compareTo(other.isSetColumn()); if (lastComparison != 0) { return lastComparison; } if (isSetColumn()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.column, typedOther.column); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.column, other.column); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetAttributes()).compareTo(typedOther.isSetAttributes()); + lastComparison = Boolean.valueOf(isSetAttributes()).compareTo(other.isSetAttributes()); if (lastComparison != 0) { return lastComparison; } if (isSetAttributes()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.attributes, typedOther.attributes); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.attributes, other.attributes); if (lastComparison != 0) { return lastComparison; } @@ -13185,6 +15934,7 @@ public class Hbase { 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 { @@ -13248,15 +15998,15 @@ public class Hbase { case 4: // ATTRIBUTES if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { { - org.apache.thrift.protocol.TMap _map60 = iprot.readMapBegin(); - struct.attributes = new HashMap(2*_map60.size); - for (int _i61 = 0; _i61 < _map60.size; ++_i61) + org.apache.thrift.protocol.TMap _map68 = iprot.readMapBegin(); + struct.attributes = new HashMap(2*_map68.size); + for (int _i69 = 0; _i69 < _map68.size; ++_i69) { - ByteBuffer _key62; // required - ByteBuffer _val63; // optional - _key62 = iprot.readBinary(); - _val63 = iprot.readBinary(); - struct.attributes.put(_key62, _val63); + ByteBuffer _key70; + ByteBuffer _val71; + _key70 = iprot.readBinary(); + _val71 = iprot.readBinary(); + struct.attributes.put(_key70, _val71); } iprot.readMapEnd(); } @@ -13299,10 +16049,10 @@ public class Hbase { oprot.writeFieldBegin(ATTRIBUTES_FIELD_DESC); { oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.attributes.size())); - for (Map.Entry _iter64 : struct.attributes.entrySet()) + for (Map.Entry _iter72 : struct.attributes.entrySet()) { - oprot.writeBinary(_iter64.getKey()); - oprot.writeBinary(_iter64.getValue()); + oprot.writeBinary(_iter72.getKey()); + oprot.writeBinary(_iter72.getValue()); } oprot.writeMapEnd(); } @@ -13351,10 +16101,10 @@ public class Hbase { if (struct.isSetAttributes()) { { oprot.writeI32(struct.attributes.size()); - for (Map.Entry _iter65 : struct.attributes.entrySet()) + for (Map.Entry _iter73 : struct.attributes.entrySet()) { - oprot.writeBinary(_iter65.getKey()); - oprot.writeBinary(_iter65.getValue()); + oprot.writeBinary(_iter73.getKey()); + oprot.writeBinary(_iter73.getValue()); } } } @@ -13378,15 +16128,15 @@ public class Hbase { } if (incoming.get(3)) { { - org.apache.thrift.protocol.TMap _map66 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32()); - struct.attributes = new HashMap(2*_map66.size); - for (int _i67 = 0; _i67 < _map66.size; ++_i67) + org.apache.thrift.protocol.TMap _map74 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32()); + struct.attributes = new HashMap(2*_map74.size); + for (int _i75 = 0; _i75 < _map74.size; ++_i75) { - ByteBuffer _key68; // required - ByteBuffer _val69; // optional - _key68 = iprot.readBinary(); - _val69 = iprot.readBinary(); - struct.attributes.put(_key68, _val69); + ByteBuffer _key76; + ByteBuffer _val77; + _key76 = iprot.readBinary(); + _val77 = iprot.readBinary(); + struct.attributes.put(_key76, _val77); } } struct.setAttributesIsSet(true); @@ -13396,7 +16146,7 @@ public class Hbase { } - public static class get_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + public static class get_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("get_result"); private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.LIST, (short)0); @@ -13502,7 +16252,7 @@ public class Hbase { */ public get_result(get_result other) { if (other.isSetSuccess()) { - List __this__success = new ArrayList(); + List __this__success = new ArrayList(other.success.size()); for (TCell other_element : other.success) { __this__success.add(new TCell(other_element)); } @@ -13673,30 +16423,30 @@ public class Hbase { return 0; } + @Override public int compareTo(get_result other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; - get_result typedOther = (get_result)other; - lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(typedOther.isSetSuccess()); + lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(other.isSetSuccess()); if (lastComparison != 0) { return lastComparison; } if (isSetSuccess()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, typedOther.success); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetIo()).compareTo(typedOther.isSetIo()); + lastComparison = Boolean.valueOf(isSetIo()).compareTo(other.isSetIo()); if (lastComparison != 0) { return lastComparison; } if (isSetIo()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.io, typedOther.io); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.io, other.io); if (lastComparison != 0) { return lastComparison; } @@ -13742,6 +16492,7 @@ public class Hbase { 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 { @@ -13781,14 +16532,14 @@ public class Hbase { case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list70 = iprot.readListBegin(); - struct.success = new ArrayList(_list70.size); - for (int _i71 = 0; _i71 < _list70.size; ++_i71) + org.apache.thrift.protocol.TList _list78 = iprot.readListBegin(); + struct.success = new ArrayList(_list78.size); + for (int _i79 = 0; _i79 < _list78.size; ++_i79) { - TCell _elem72; // optional - _elem72 = new TCell(); - _elem72.read(iprot); - struct.success.add(_elem72); + TCell _elem80; + _elem80 = new TCell(); + _elem80.read(iprot); + struct.success.add(_elem80); } iprot.readListEnd(); } @@ -13825,9 +16576,9 @@ public class Hbase { oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size())); - for (TCell _iter73 : struct.success) + for (TCell _iter81 : struct.success) { - _iter73.write(oprot); + _iter81.write(oprot); } oprot.writeListEnd(); } @@ -13866,9 +16617,9 @@ public class Hbase { if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (TCell _iter74 : struct.success) + for (TCell _iter82 : struct.success) { - _iter74.write(oprot); + _iter82.write(oprot); } } } @@ -13883,14 +16634,14 @@ public class Hbase { BitSet incoming = iprot.readBitSet(2); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list75 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); - struct.success = new ArrayList(_list75.size); - for (int _i76 = 0; _i76 < _list75.size; ++_i76) + org.apache.thrift.protocol.TList _list83 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); + struct.success = new ArrayList(_list83.size); + for (int _i84 = 0; _i84 < _list83.size; ++_i84) { - TCell _elem77; // optional - _elem77 = new TCell(); - _elem77.read(iprot); - struct.success.add(_elem77); + TCell _elem85; + _elem85 = new TCell(); + _elem85.read(iprot); + struct.success.add(_elem85); } } struct.setSuccessIsSet(true); @@ -13905,7 +16656,7 @@ public class Hbase { } - public static class getVer_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + public static class getVer_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getVer_args"); private static final org.apache.thrift.protocol.TField TABLE_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("tableName", org.apache.thrift.protocol.TType.STRING, (short)1); @@ -14028,7 +16779,7 @@ public class Hbase { // isset id assignments private static final int __NUMVERSIONS_ISSET_ID = 0; - private BitSet __isset_bit_vector = new BitSet(1); + 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); @@ -14071,8 +16822,7 @@ public class Hbase { * Performs a deep copy on other. */ public getVer_args(getVer_args other) { - __isset_bit_vector.clear(); - __isset_bit_vector.or(other.__isset_bit_vector); + __isset_bitfield = other.__isset_bitfield; if (other.isSetTableName()) { this.tableName = other.tableName; } @@ -14084,7 +16834,7 @@ public class Hbase { } this.numVersions = other.numVersions; if (other.isSetAttributes()) { - Map __this__attributes = new HashMap(); + Map __this__attributes = new HashMap(other.attributes.size()); for (Map.Entry other_element : other.attributes.entrySet()) { ByteBuffer other_element_key = other_element.getKey(); @@ -14251,16 +17001,16 @@ public class Hbase { } public void unsetNumVersions() { - __isset_bit_vector.clear(__NUMVERSIONS_ISSET_ID); + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __NUMVERSIONS_ISSET_ID); } /** Returns true if field numVersions is set (has been assigned a value) and false otherwise */ public boolean isSetNumVersions() { - return __isset_bit_vector.get(__NUMVERSIONS_ISSET_ID); + return EncodingUtils.testBit(__isset_bitfield, __NUMVERSIONS_ISSET_ID); } public void setNumVersionsIsSet(boolean value) { - __isset_bit_vector.set(__NUMVERSIONS_ISSET_ID, value); + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __NUMVERSIONS_ISSET_ID, value); } public int getAttributesSize() { @@ -14457,60 +17207,60 @@ public class Hbase { return 0; } + @Override public int compareTo(getVer_args other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; - getVer_args typedOther = (getVer_args)other; - lastComparison = Boolean.valueOf(isSetTableName()).compareTo(typedOther.isSetTableName()); + lastComparison = Boolean.valueOf(isSetTableName()).compareTo(other.isSetTableName()); if (lastComparison != 0) { return lastComparison; } if (isSetTableName()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tableName, typedOther.tableName); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tableName, other.tableName); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetRow()).compareTo(typedOther.isSetRow()); + lastComparison = Boolean.valueOf(isSetRow()).compareTo(other.isSetRow()); if (lastComparison != 0) { return lastComparison; } if (isSetRow()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.row, typedOther.row); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.row, other.row); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetColumn()).compareTo(typedOther.isSetColumn()); + lastComparison = Boolean.valueOf(isSetColumn()).compareTo(other.isSetColumn()); if (lastComparison != 0) { return lastComparison; } if (isSetColumn()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.column, typedOther.column); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.column, other.column); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetNumVersions()).compareTo(typedOther.isSetNumVersions()); + lastComparison = Boolean.valueOf(isSetNumVersions()).compareTo(other.isSetNumVersions()); if (lastComparison != 0) { return lastComparison; } if (isSetNumVersions()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.numVersions, typedOther.numVersions); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.numVersions, other.numVersions); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetAttributes()).compareTo(typedOther.isSetAttributes()); + lastComparison = Boolean.valueOf(isSetAttributes()).compareTo(other.isSetAttributes()); if (lastComparison != 0) { return lastComparison; } if (isSetAttributes()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.attributes, typedOther.attributes); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.attributes, other.attributes); if (lastComparison != 0) { return lastComparison; } @@ -14576,6 +17326,7 @@ public class Hbase { 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 { @@ -14589,7 +17340,7 @@ public class Hbase { 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_bit_vector = new BitSet(1); + __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); @@ -14649,15 +17400,15 @@ public class Hbase { case 5: // ATTRIBUTES if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { { - org.apache.thrift.protocol.TMap _map78 = iprot.readMapBegin(); - struct.attributes = new HashMap(2*_map78.size); - for (int _i79 = 0; _i79 < _map78.size; ++_i79) + org.apache.thrift.protocol.TMap _map86 = iprot.readMapBegin(); + struct.attributes = new HashMap(2*_map86.size); + for (int _i87 = 0; _i87 < _map86.size; ++_i87) { - ByteBuffer _key80; // required - ByteBuffer _val81; // optional - _key80 = iprot.readBinary(); - _val81 = iprot.readBinary(); - struct.attributes.put(_key80, _val81); + ByteBuffer _key88; + ByteBuffer _val89; + _key88 = iprot.readBinary(); + _val89 = iprot.readBinary(); + struct.attributes.put(_key88, _val89); } iprot.readMapEnd(); } @@ -14703,10 +17454,10 @@ public class Hbase { oprot.writeFieldBegin(ATTRIBUTES_FIELD_DESC); { oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.attributes.size())); - for (Map.Entry _iter82 : struct.attributes.entrySet()) + for (Map.Entry _iter90 : struct.attributes.entrySet()) { - oprot.writeBinary(_iter82.getKey()); - oprot.writeBinary(_iter82.getValue()); + oprot.writeBinary(_iter90.getKey()); + oprot.writeBinary(_iter90.getValue()); } oprot.writeMapEnd(); } @@ -14761,10 +17512,10 @@ public class Hbase { if (struct.isSetAttributes()) { { oprot.writeI32(struct.attributes.size()); - for (Map.Entry _iter83 : struct.attributes.entrySet()) + for (Map.Entry _iter91 : struct.attributes.entrySet()) { - oprot.writeBinary(_iter83.getKey()); - oprot.writeBinary(_iter83.getValue()); + oprot.writeBinary(_iter91.getKey()); + oprot.writeBinary(_iter91.getValue()); } } } @@ -14792,15 +17543,15 @@ public class Hbase { } if (incoming.get(4)) { { - org.apache.thrift.protocol.TMap _map84 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32()); - struct.attributes = new HashMap(2*_map84.size); - for (int _i85 = 0; _i85 < _map84.size; ++_i85) + org.apache.thrift.protocol.TMap _map92 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32()); + struct.attributes = new HashMap(2*_map92.size); + for (int _i93 = 0; _i93 < _map92.size; ++_i93) { - ByteBuffer _key86; // required - ByteBuffer _val87; // optional - _key86 = iprot.readBinary(); - _val87 = iprot.readBinary(); - struct.attributes.put(_key86, _val87); + ByteBuffer _key94; + ByteBuffer _val95; + _key94 = iprot.readBinary(); + _val95 = iprot.readBinary(); + struct.attributes.put(_key94, _val95); } } struct.setAttributesIsSet(true); @@ -14810,7 +17561,7 @@ public class Hbase { } - public static class getVer_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + public static class getVer_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getVer_result"); private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.LIST, (short)0); @@ -14916,7 +17667,7 @@ public class Hbase { */ public getVer_result(getVer_result other) { if (other.isSetSuccess()) { - List __this__success = new ArrayList(); + List __this__success = new ArrayList(other.success.size()); for (TCell other_element : other.success) { __this__success.add(new TCell(other_element)); } @@ -15087,30 +17838,30 @@ public class Hbase { return 0; } + @Override public int compareTo(getVer_result other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; - getVer_result typedOther = (getVer_result)other; - lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(typedOther.isSetSuccess()); + lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(other.isSetSuccess()); if (lastComparison != 0) { return lastComparison; } if (isSetSuccess()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, typedOther.success); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetIo()).compareTo(typedOther.isSetIo()); + lastComparison = Boolean.valueOf(isSetIo()).compareTo(other.isSetIo()); if (lastComparison != 0) { return lastComparison; } if (isSetIo()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.io, typedOther.io); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.io, other.io); if (lastComparison != 0) { return lastComparison; } @@ -15156,6 +17907,7 @@ public class Hbase { 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 { @@ -15195,14 +17947,14 @@ public class Hbase { case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list88 = iprot.readListBegin(); - struct.success = new ArrayList(_list88.size); - for (int _i89 = 0; _i89 < _list88.size; ++_i89) + org.apache.thrift.protocol.TList _list96 = iprot.readListBegin(); + struct.success = new ArrayList(_list96.size); + for (int _i97 = 0; _i97 < _list96.size; ++_i97) { - TCell _elem90; // optional - _elem90 = new TCell(); - _elem90.read(iprot); - struct.success.add(_elem90); + TCell _elem98; + _elem98 = new TCell(); + _elem98.read(iprot); + struct.success.add(_elem98); } iprot.readListEnd(); } @@ -15239,9 +17991,9 @@ public class Hbase { oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size())); - for (TCell _iter91 : struct.success) + for (TCell _iter99 : struct.success) { - _iter91.write(oprot); + _iter99.write(oprot); } oprot.writeListEnd(); } @@ -15280,9 +18032,9 @@ public class Hbase { if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (TCell _iter92 : struct.success) + for (TCell _iter100 : struct.success) { - _iter92.write(oprot); + _iter100.write(oprot); } } } @@ -15297,14 +18049,14 @@ public class Hbase { BitSet incoming = iprot.readBitSet(2); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list93 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); - struct.success = new ArrayList(_list93.size); - for (int _i94 = 0; _i94 < _list93.size; ++_i94) + org.apache.thrift.protocol.TList _list101 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); + struct.success = new ArrayList(_list101.size); + for (int _i102 = 0; _i102 < _list101.size; ++_i102) { - TCell _elem95; // optional - _elem95 = new TCell(); - _elem95.read(iprot); - struct.success.add(_elem95); + TCell _elem103; + _elem103 = new TCell(); + _elem103.read(iprot); + struct.success.add(_elem103); } } struct.setSuccessIsSet(true); @@ -15319,7 +18071,7 @@ public class Hbase { } - public static class getVerTs_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + public static class getVerTs_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getVerTs_args"); private static final org.apache.thrift.protocol.TField TABLE_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("tableName", org.apache.thrift.protocol.TType.STRING, (short)1); @@ -15454,7 +18206,7 @@ public class Hbase { // isset id assignments private static final int __TIMESTAMP_ISSET_ID = 0; private static final int __NUMVERSIONS_ISSET_ID = 1; - private BitSet __isset_bit_vector = new BitSet(2); + 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); @@ -15502,8 +18254,7 @@ public class Hbase { * Performs a deep copy on other. */ public getVerTs_args(getVerTs_args other) { - __isset_bit_vector.clear(); - __isset_bit_vector.or(other.__isset_bit_vector); + __isset_bitfield = other.__isset_bitfield; if (other.isSetTableName()) { this.tableName = other.tableName; } @@ -15516,7 +18267,7 @@ public class Hbase { this.timestamp = other.timestamp; this.numVersions = other.numVersions; if (other.isSetAttributes()) { - Map __this__attributes = new HashMap(); + Map __this__attributes = new HashMap(other.attributes.size()); for (Map.Entry other_element : other.attributes.entrySet()) { ByteBuffer other_element_key = other_element.getKey(); @@ -15685,16 +18436,16 @@ public class Hbase { } public void unsetTimestamp() { - __isset_bit_vector.clear(__TIMESTAMP_ISSET_ID); + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __TIMESTAMP_ISSET_ID); } /** Returns true if field timestamp is set (has been assigned a value) and false otherwise */ public boolean isSetTimestamp() { - return __isset_bit_vector.get(__TIMESTAMP_ISSET_ID); + return EncodingUtils.testBit(__isset_bitfield, __TIMESTAMP_ISSET_ID); } public void setTimestampIsSet(boolean value) { - __isset_bit_vector.set(__TIMESTAMP_ISSET_ID, value); + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __TIMESTAMP_ISSET_ID, value); } /** @@ -15714,16 +18465,16 @@ public class Hbase { } public void unsetNumVersions() { - __isset_bit_vector.clear(__NUMVERSIONS_ISSET_ID); + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __NUMVERSIONS_ISSET_ID); } /** Returns true if field numVersions is set (has been assigned a value) and false otherwise */ public boolean isSetNumVersions() { - return __isset_bit_vector.get(__NUMVERSIONS_ISSET_ID); + return EncodingUtils.testBit(__isset_bitfield, __NUMVERSIONS_ISSET_ID); } public void setNumVersionsIsSet(boolean value) { - __isset_bit_vector.set(__NUMVERSIONS_ISSET_ID, value); + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __NUMVERSIONS_ISSET_ID, value); } public int getAttributesSize() { @@ -15942,70 +18693,70 @@ public class Hbase { return 0; } + @Override public int compareTo(getVerTs_args other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; - getVerTs_args typedOther = (getVerTs_args)other; - lastComparison = Boolean.valueOf(isSetTableName()).compareTo(typedOther.isSetTableName()); + lastComparison = Boolean.valueOf(isSetTableName()).compareTo(other.isSetTableName()); if (lastComparison != 0) { return lastComparison; } if (isSetTableName()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tableName, typedOther.tableName); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tableName, other.tableName); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetRow()).compareTo(typedOther.isSetRow()); + lastComparison = Boolean.valueOf(isSetRow()).compareTo(other.isSetRow()); if (lastComparison != 0) { return lastComparison; } if (isSetRow()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.row, typedOther.row); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.row, other.row); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetColumn()).compareTo(typedOther.isSetColumn()); + lastComparison = Boolean.valueOf(isSetColumn()).compareTo(other.isSetColumn()); if (lastComparison != 0) { return lastComparison; } if (isSetColumn()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.column, typedOther.column); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.column, other.column); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetTimestamp()).compareTo(typedOther.isSetTimestamp()); + lastComparison = Boolean.valueOf(isSetTimestamp()).compareTo(other.isSetTimestamp()); if (lastComparison != 0) { return lastComparison; } if (isSetTimestamp()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.timestamp, typedOther.timestamp); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.timestamp, other.timestamp); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetNumVersions()).compareTo(typedOther.isSetNumVersions()); + lastComparison = Boolean.valueOf(isSetNumVersions()).compareTo(other.isSetNumVersions()); if (lastComparison != 0) { return lastComparison; } if (isSetNumVersions()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.numVersions, typedOther.numVersions); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.numVersions, other.numVersions); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetAttributes()).compareTo(typedOther.isSetAttributes()); + lastComparison = Boolean.valueOf(isSetAttributes()).compareTo(other.isSetAttributes()); if (lastComparison != 0) { return lastComparison; } if (isSetAttributes()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.attributes, typedOther.attributes); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.attributes, other.attributes); if (lastComparison != 0) { return lastComparison; } @@ -16075,6 +18826,7 @@ public class Hbase { 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 { @@ -16088,7 +18840,7 @@ public class Hbase { 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_bit_vector = new BitSet(1); + __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); @@ -16156,15 +18908,15 @@ public class Hbase { case 6: // ATTRIBUTES if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { { - org.apache.thrift.protocol.TMap _map96 = iprot.readMapBegin(); - struct.attributes = new HashMap(2*_map96.size); - for (int _i97 = 0; _i97 < _map96.size; ++_i97) + org.apache.thrift.protocol.TMap _map104 = iprot.readMapBegin(); + struct.attributes = new HashMap(2*_map104.size); + for (int _i105 = 0; _i105 < _map104.size; ++_i105) { - ByteBuffer _key98; // required - ByteBuffer _val99; // optional - _key98 = iprot.readBinary(); - _val99 = iprot.readBinary(); - struct.attributes.put(_key98, _val99); + ByteBuffer _key106; + ByteBuffer _val107; + _key106 = iprot.readBinary(); + _val107 = iprot.readBinary(); + struct.attributes.put(_key106, _val107); } iprot.readMapEnd(); } @@ -16213,10 +18965,10 @@ public class Hbase { oprot.writeFieldBegin(ATTRIBUTES_FIELD_DESC); { oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.attributes.size())); - for (Map.Entry _iter100 : struct.attributes.entrySet()) + for (Map.Entry _iter108 : struct.attributes.entrySet()) { - oprot.writeBinary(_iter100.getKey()); - oprot.writeBinary(_iter100.getValue()); + oprot.writeBinary(_iter108.getKey()); + oprot.writeBinary(_iter108.getValue()); } oprot.writeMapEnd(); } @@ -16277,10 +19029,10 @@ public class Hbase { if (struct.isSetAttributes()) { { oprot.writeI32(struct.attributes.size()); - for (Map.Entry _iter101 : struct.attributes.entrySet()) + for (Map.Entry _iter109 : struct.attributes.entrySet()) { - oprot.writeBinary(_iter101.getKey()); - oprot.writeBinary(_iter101.getValue()); + oprot.writeBinary(_iter109.getKey()); + oprot.writeBinary(_iter109.getValue()); } } } @@ -16312,15 +19064,15 @@ public class Hbase { } if (incoming.get(5)) { { - org.apache.thrift.protocol.TMap _map102 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32()); - struct.attributes = new HashMap(2*_map102.size); - for (int _i103 = 0; _i103 < _map102.size; ++_i103) + org.apache.thrift.protocol.TMap _map110 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32()); + struct.attributes = new HashMap(2*_map110.size); + for (int _i111 = 0; _i111 < _map110.size; ++_i111) { - ByteBuffer _key104; // required - ByteBuffer _val105; // optional - _key104 = iprot.readBinary(); - _val105 = iprot.readBinary(); - struct.attributes.put(_key104, _val105); + ByteBuffer _key112; + ByteBuffer _val113; + _key112 = iprot.readBinary(); + _val113 = iprot.readBinary(); + struct.attributes.put(_key112, _val113); } } struct.setAttributesIsSet(true); @@ -16330,7 +19082,7 @@ public class Hbase { } - public static class getVerTs_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + public static class getVerTs_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getVerTs_result"); private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.LIST, (short)0); @@ -16436,7 +19188,7 @@ public class Hbase { */ public getVerTs_result(getVerTs_result other) { if (other.isSetSuccess()) { - List __this__success = new ArrayList(); + List __this__success = new ArrayList(other.success.size()); for (TCell other_element : other.success) { __this__success.add(new TCell(other_element)); } @@ -16607,30 +19359,30 @@ public class Hbase { return 0; } + @Override public int compareTo(getVerTs_result other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; - getVerTs_result typedOther = (getVerTs_result)other; - lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(typedOther.isSetSuccess()); + lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(other.isSetSuccess()); if (lastComparison != 0) { return lastComparison; } if (isSetSuccess()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, typedOther.success); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetIo()).compareTo(typedOther.isSetIo()); + lastComparison = Boolean.valueOf(isSetIo()).compareTo(other.isSetIo()); if (lastComparison != 0) { return lastComparison; } if (isSetIo()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.io, typedOther.io); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.io, other.io); if (lastComparison != 0) { return lastComparison; } @@ -16676,6 +19428,7 @@ public class Hbase { 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 { @@ -16715,14 +19468,14 @@ public class Hbase { case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list106 = iprot.readListBegin(); - struct.success = new ArrayList(_list106.size); - for (int _i107 = 0; _i107 < _list106.size; ++_i107) + org.apache.thrift.protocol.TList _list114 = iprot.readListBegin(); + struct.success = new ArrayList(_list114.size); + for (int _i115 = 0; _i115 < _list114.size; ++_i115) { - TCell _elem108; // optional - _elem108 = new TCell(); - _elem108.read(iprot); - struct.success.add(_elem108); + TCell _elem116; + _elem116 = new TCell(); + _elem116.read(iprot); + struct.success.add(_elem116); } iprot.readListEnd(); } @@ -16759,9 +19512,9 @@ public class Hbase { oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size())); - for (TCell _iter109 : struct.success) + for (TCell _iter117 : struct.success) { - _iter109.write(oprot); + _iter117.write(oprot); } oprot.writeListEnd(); } @@ -16800,9 +19553,9 @@ public class Hbase { if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (TCell _iter110 : struct.success) + for (TCell _iter118 : struct.success) { - _iter110.write(oprot); + _iter118.write(oprot); } } } @@ -16817,14 +19570,14 @@ public class Hbase { BitSet incoming = iprot.readBitSet(2); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list111 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); - struct.success = new ArrayList(_list111.size); - for (int _i112 = 0; _i112 < _list111.size; ++_i112) + org.apache.thrift.protocol.TList _list119 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); + struct.success = new ArrayList(_list119.size); + for (int _i120 = 0; _i120 < _list119.size; ++_i120) { - TCell _elem113; // optional - _elem113 = new TCell(); - _elem113.read(iprot); - struct.success.add(_elem113); + TCell _elem121; + _elem121 = new TCell(); + _elem121.read(iprot); + struct.success.add(_elem121); } } struct.setSuccessIsSet(true); @@ -16839,7 +19592,7 @@ public class Hbase { } - public static class getRow_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + public static class getRow_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getRow_args"); private static final org.apache.thrift.protocol.TField TABLE_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("tableName", org.apache.thrift.protocol.TType.STRING, (short)1); @@ -16979,7 +19732,7 @@ public class Hbase { this.row = other.row; } if (other.isSetAttributes()) { - Map __this__attributes = new HashMap(); + Map __this__attributes = new HashMap(other.attributes.size()); for (Map.Entry other_element : other.attributes.entrySet()) { ByteBuffer other_element_key = other_element.getKey(); @@ -17236,40 +19989,40 @@ public class Hbase { return 0; } + @Override public int compareTo(getRow_args other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; - getRow_args typedOther = (getRow_args)other; - lastComparison = Boolean.valueOf(isSetTableName()).compareTo(typedOther.isSetTableName()); + lastComparison = Boolean.valueOf(isSetTableName()).compareTo(other.isSetTableName()); if (lastComparison != 0) { return lastComparison; } if (isSetTableName()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tableName, typedOther.tableName); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tableName, other.tableName); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetRow()).compareTo(typedOther.isSetRow()); + lastComparison = Boolean.valueOf(isSetRow()).compareTo(other.isSetRow()); if (lastComparison != 0) { return lastComparison; } if (isSetRow()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.row, typedOther.row); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.row, other.row); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetAttributes()).compareTo(typedOther.isSetAttributes()); + lastComparison = Boolean.valueOf(isSetAttributes()).compareTo(other.isSetAttributes()); if (lastComparison != 0) { return lastComparison; } if (isSetAttributes()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.attributes, typedOther.attributes); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.attributes, other.attributes); if (lastComparison != 0) { return lastComparison; } @@ -17323,6 +20076,7 @@ public class Hbase { 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 { @@ -17378,15 +20132,15 @@ public class Hbase { case 3: // ATTRIBUTES if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { { - org.apache.thrift.protocol.TMap _map114 = iprot.readMapBegin(); - struct.attributes = new HashMap(2*_map114.size); - for (int _i115 = 0; _i115 < _map114.size; ++_i115) + org.apache.thrift.protocol.TMap _map122 = iprot.readMapBegin(); + struct.attributes = new HashMap(2*_map122.size); + for (int _i123 = 0; _i123 < _map122.size; ++_i123) { - ByteBuffer _key116; // required - ByteBuffer _val117; // optional - _key116 = iprot.readBinary(); - _val117 = iprot.readBinary(); - struct.attributes.put(_key116, _val117); + ByteBuffer _key124; + ByteBuffer _val125; + _key124 = iprot.readBinary(); + _val125 = iprot.readBinary(); + struct.attributes.put(_key124, _val125); } iprot.readMapEnd(); } @@ -17424,10 +20178,10 @@ public class Hbase { oprot.writeFieldBegin(ATTRIBUTES_FIELD_DESC); { oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.attributes.size())); - for (Map.Entry _iter118 : struct.attributes.entrySet()) + for (Map.Entry _iter126 : struct.attributes.entrySet()) { - oprot.writeBinary(_iter118.getKey()); - oprot.writeBinary(_iter118.getValue()); + oprot.writeBinary(_iter126.getKey()); + oprot.writeBinary(_iter126.getValue()); } oprot.writeMapEnd(); } @@ -17470,10 +20224,10 @@ public class Hbase { if (struct.isSetAttributes()) { { oprot.writeI32(struct.attributes.size()); - for (Map.Entry _iter119 : struct.attributes.entrySet()) + for (Map.Entry _iter127 : struct.attributes.entrySet()) { - oprot.writeBinary(_iter119.getKey()); - oprot.writeBinary(_iter119.getValue()); + oprot.writeBinary(_iter127.getKey()); + oprot.writeBinary(_iter127.getValue()); } } } @@ -17493,15 +20247,15 @@ public class Hbase { } if (incoming.get(2)) { { - org.apache.thrift.protocol.TMap _map120 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32()); - struct.attributes = new HashMap(2*_map120.size); - for (int _i121 = 0; _i121 < _map120.size; ++_i121) + org.apache.thrift.protocol.TMap _map128 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32()); + struct.attributes = new HashMap(2*_map128.size); + for (int _i129 = 0; _i129 < _map128.size; ++_i129) { - ByteBuffer _key122; // required - ByteBuffer _val123; // optional - _key122 = iprot.readBinary(); - _val123 = iprot.readBinary(); - struct.attributes.put(_key122, _val123); + ByteBuffer _key130; + ByteBuffer _val131; + _key130 = iprot.readBinary(); + _val131 = iprot.readBinary(); + struct.attributes.put(_key130, _val131); } } struct.setAttributesIsSet(true); @@ -17511,7 +20265,7 @@ public class Hbase { } - public static class getRow_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + public static class getRow_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getRow_result"); private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.LIST, (short)0); @@ -17617,7 +20371,7 @@ public class Hbase { */ public getRow_result(getRow_result other) { if (other.isSetSuccess()) { - List __this__success = new ArrayList(); + List __this__success = new ArrayList(other.success.size()); for (TRowResult other_element : other.success) { __this__success.add(new TRowResult(other_element)); } @@ -17788,30 +20542,30 @@ public class Hbase { return 0; } + @Override public int compareTo(getRow_result other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; - getRow_result typedOther = (getRow_result)other; - lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(typedOther.isSetSuccess()); + lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(other.isSetSuccess()); if (lastComparison != 0) { return lastComparison; } if (isSetSuccess()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, typedOther.success); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetIo()).compareTo(typedOther.isSetIo()); + lastComparison = Boolean.valueOf(isSetIo()).compareTo(other.isSetIo()); if (lastComparison != 0) { return lastComparison; } if (isSetIo()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.io, typedOther.io); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.io, other.io); if (lastComparison != 0) { return lastComparison; } @@ -17857,6 +20611,7 @@ public class Hbase { 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 { @@ -17896,14 +20651,14 @@ public class Hbase { case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list124 = iprot.readListBegin(); - struct.success = new ArrayList(_list124.size); - for (int _i125 = 0; _i125 < _list124.size; ++_i125) + org.apache.thrift.protocol.TList _list132 = iprot.readListBegin(); + struct.success = new ArrayList(_list132.size); + for (int _i133 = 0; _i133 < _list132.size; ++_i133) { - TRowResult _elem126; // optional - _elem126 = new TRowResult(); - _elem126.read(iprot); - struct.success.add(_elem126); + TRowResult _elem134; + _elem134 = new TRowResult(); + _elem134.read(iprot); + struct.success.add(_elem134); } iprot.readListEnd(); } @@ -17940,9 +20695,9 @@ public class Hbase { oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size())); - for (TRowResult _iter127 : struct.success) + for (TRowResult _iter135 : struct.success) { - _iter127.write(oprot); + _iter135.write(oprot); } oprot.writeListEnd(); } @@ -17981,9 +20736,9 @@ public class Hbase { if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (TRowResult _iter128 : struct.success) + for (TRowResult _iter136 : struct.success) { - _iter128.write(oprot); + _iter136.write(oprot); } } } @@ -17998,14 +20753,14 @@ public class Hbase { BitSet incoming = iprot.readBitSet(2); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list129 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); - struct.success = new ArrayList(_list129.size); - for (int _i130 = 0; _i130 < _list129.size; ++_i130) + org.apache.thrift.protocol.TList _list137 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); + struct.success = new ArrayList(_list137.size); + for (int _i138 = 0; _i138 < _list137.size; ++_i138) { - TRowResult _elem131; // optional - _elem131 = new TRowResult(); - _elem131.read(iprot); - struct.success.add(_elem131); + TRowResult _elem139; + _elem139 = new TRowResult(); + _elem139.read(iprot); + struct.success.add(_elem139); } } struct.setSuccessIsSet(true); @@ -18020,7 +20775,7 @@ public class Hbase { } - public static class getRowWithColumns_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + public static class getRowWithColumns_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getRowWithColumns_args"); private static final org.apache.thrift.protocol.TField TABLE_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("tableName", org.apache.thrift.protocol.TType.STRING, (short)1); @@ -18176,14 +20931,14 @@ public class Hbase { this.row = other.row; } if (other.isSetColumns()) { - List __this__columns = new ArrayList(); + List __this__columns = new ArrayList(other.columns.size()); for (ByteBuffer other_element : other.columns) { __this__columns.add(other_element); } this.columns = __this__columns; } if (other.isSetAttributes()) { - Map __this__attributes = new HashMap(); + Map __this__attributes = new HashMap(other.attributes.size()); for (Map.Entry other_element : other.attributes.entrySet()) { ByteBuffer other_element_key = other_element.getKey(); @@ -18508,50 +21263,50 @@ public class Hbase { return 0; } + @Override public int compareTo(getRowWithColumns_args other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; - getRowWithColumns_args typedOther = (getRowWithColumns_args)other; - lastComparison = Boolean.valueOf(isSetTableName()).compareTo(typedOther.isSetTableName()); + lastComparison = Boolean.valueOf(isSetTableName()).compareTo(other.isSetTableName()); if (lastComparison != 0) { return lastComparison; } if (isSetTableName()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tableName, typedOther.tableName); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tableName, other.tableName); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetRow()).compareTo(typedOther.isSetRow()); + lastComparison = Boolean.valueOf(isSetRow()).compareTo(other.isSetRow()); if (lastComparison != 0) { return lastComparison; } if (isSetRow()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.row, typedOther.row); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.row, other.row); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetColumns()).compareTo(typedOther.isSetColumns()); + lastComparison = Boolean.valueOf(isSetColumns()).compareTo(other.isSetColumns()); if (lastComparison != 0) { return lastComparison; } if (isSetColumns()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.columns, typedOther.columns); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.columns, other.columns); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetAttributes()).compareTo(typedOther.isSetAttributes()); + lastComparison = Boolean.valueOf(isSetAttributes()).compareTo(other.isSetAttributes()); if (lastComparison != 0) { return lastComparison; } if (isSetAttributes()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.attributes, typedOther.attributes); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.attributes, other.attributes); if (lastComparison != 0) { return lastComparison; } @@ -18613,6 +21368,7 @@ public class Hbase { 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 { @@ -18668,13 +21424,13 @@ public class Hbase { case 3: // COLUMNS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list132 = iprot.readListBegin(); - struct.columns = new ArrayList(_list132.size); - for (int _i133 = 0; _i133 < _list132.size; ++_i133) + org.apache.thrift.protocol.TList _list140 = iprot.readListBegin(); + struct.columns = new ArrayList(_list140.size); + for (int _i141 = 0; _i141 < _list140.size; ++_i141) { - ByteBuffer _elem134; // optional - _elem134 = iprot.readBinary(); - struct.columns.add(_elem134); + ByteBuffer _elem142; + _elem142 = iprot.readBinary(); + struct.columns.add(_elem142); } iprot.readListEnd(); } @@ -18686,15 +21442,15 @@ public class Hbase { case 4: // ATTRIBUTES if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { { - org.apache.thrift.protocol.TMap _map135 = iprot.readMapBegin(); - struct.attributes = new HashMap(2*_map135.size); - for (int _i136 = 0; _i136 < _map135.size; ++_i136) + org.apache.thrift.protocol.TMap _map143 = iprot.readMapBegin(); + struct.attributes = new HashMap(2*_map143.size); + for (int _i144 = 0; _i144 < _map143.size; ++_i144) { - ByteBuffer _key137; // required - ByteBuffer _val138; // optional - _key137 = iprot.readBinary(); - _val138 = iprot.readBinary(); - struct.attributes.put(_key137, _val138); + ByteBuffer _key145; + ByteBuffer _val146; + _key145 = iprot.readBinary(); + _val146 = iprot.readBinary(); + struct.attributes.put(_key145, _val146); } iprot.readMapEnd(); } @@ -18732,9 +21488,9 @@ public class Hbase { oprot.writeFieldBegin(COLUMNS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.columns.size())); - for (ByteBuffer _iter139 : struct.columns) + for (ByteBuffer _iter147 : struct.columns) { - oprot.writeBinary(_iter139); + oprot.writeBinary(_iter147); } oprot.writeListEnd(); } @@ -18744,10 +21500,10 @@ public class Hbase { oprot.writeFieldBegin(ATTRIBUTES_FIELD_DESC); { oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.attributes.size())); - for (Map.Entry _iter140 : struct.attributes.entrySet()) + for (Map.Entry _iter148 : struct.attributes.entrySet()) { - oprot.writeBinary(_iter140.getKey()); - oprot.writeBinary(_iter140.getValue()); + oprot.writeBinary(_iter148.getKey()); + oprot.writeBinary(_iter148.getValue()); } oprot.writeMapEnd(); } @@ -18793,19 +21549,19 @@ public class Hbase { if (struct.isSetColumns()) { { oprot.writeI32(struct.columns.size()); - for (ByteBuffer _iter141 : struct.columns) + for (ByteBuffer _iter149 : struct.columns) { - oprot.writeBinary(_iter141); + oprot.writeBinary(_iter149); } } } if (struct.isSetAttributes()) { { oprot.writeI32(struct.attributes.size()); - for (Map.Entry _iter142 : struct.attributes.entrySet()) + for (Map.Entry _iter150 : struct.attributes.entrySet()) { - oprot.writeBinary(_iter142.getKey()); - oprot.writeBinary(_iter142.getValue()); + oprot.writeBinary(_iter150.getKey()); + oprot.writeBinary(_iter150.getValue()); } } } @@ -18825,28 +21581,28 @@ public class Hbase { } if (incoming.get(2)) { { - org.apache.thrift.protocol.TList _list143 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); - struct.columns = new ArrayList(_list143.size); - for (int _i144 = 0; _i144 < _list143.size; ++_i144) + org.apache.thrift.protocol.TList _list151 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); + struct.columns = new ArrayList(_list151.size); + for (int _i152 = 0; _i152 < _list151.size; ++_i152) { - ByteBuffer _elem145; // optional - _elem145 = iprot.readBinary(); - struct.columns.add(_elem145); + ByteBuffer _elem153; + _elem153 = iprot.readBinary(); + struct.columns.add(_elem153); } } struct.setColumnsIsSet(true); } if (incoming.get(3)) { { - org.apache.thrift.protocol.TMap _map146 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32()); - struct.attributes = new HashMap(2*_map146.size); - for (int _i147 = 0; _i147 < _map146.size; ++_i147) + org.apache.thrift.protocol.TMap _map154 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32()); + struct.attributes = new HashMap(2*_map154.size); + for (int _i155 = 0; _i155 < _map154.size; ++_i155) { - ByteBuffer _key148; // required - ByteBuffer _val149; // optional - _key148 = iprot.readBinary(); - _val149 = iprot.readBinary(); - struct.attributes.put(_key148, _val149); + ByteBuffer _key156; + ByteBuffer _val157; + _key156 = iprot.readBinary(); + _val157 = iprot.readBinary(); + struct.attributes.put(_key156, _val157); } } struct.setAttributesIsSet(true); @@ -18856,7 +21612,7 @@ public class Hbase { } - public static class getRowWithColumns_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + public static class getRowWithColumns_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getRowWithColumns_result"); private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.LIST, (short)0); @@ -18962,7 +21718,7 @@ public class Hbase { */ public getRowWithColumns_result(getRowWithColumns_result other) { if (other.isSetSuccess()) { - List __this__success = new ArrayList(); + List __this__success = new ArrayList(other.success.size()); for (TRowResult other_element : other.success) { __this__success.add(new TRowResult(other_element)); } @@ -19133,30 +21889,30 @@ public class Hbase { return 0; } + @Override public int compareTo(getRowWithColumns_result other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; - getRowWithColumns_result typedOther = (getRowWithColumns_result)other; - lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(typedOther.isSetSuccess()); + lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(other.isSetSuccess()); if (lastComparison != 0) { return lastComparison; } if (isSetSuccess()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, typedOther.success); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetIo()).compareTo(typedOther.isSetIo()); + lastComparison = Boolean.valueOf(isSetIo()).compareTo(other.isSetIo()); if (lastComparison != 0) { return lastComparison; } if (isSetIo()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.io, typedOther.io); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.io, other.io); if (lastComparison != 0) { return lastComparison; } @@ -19202,6 +21958,7 @@ public class Hbase { 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 { @@ -19241,14 +21998,14 @@ public class Hbase { case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list150 = iprot.readListBegin(); - struct.success = new ArrayList(_list150.size); - for (int _i151 = 0; _i151 < _list150.size; ++_i151) + org.apache.thrift.protocol.TList _list158 = iprot.readListBegin(); + struct.success = new ArrayList(_list158.size); + for (int _i159 = 0; _i159 < _list158.size; ++_i159) { - TRowResult _elem152; // optional - _elem152 = new TRowResult(); - _elem152.read(iprot); - struct.success.add(_elem152); + TRowResult _elem160; + _elem160 = new TRowResult(); + _elem160.read(iprot); + struct.success.add(_elem160); } iprot.readListEnd(); } @@ -19285,9 +22042,9 @@ public class Hbase { oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size())); - for (TRowResult _iter153 : struct.success) + for (TRowResult _iter161 : struct.success) { - _iter153.write(oprot); + _iter161.write(oprot); } oprot.writeListEnd(); } @@ -19326,9 +22083,9 @@ public class Hbase { if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (TRowResult _iter154 : struct.success) + for (TRowResult _iter162 : struct.success) { - _iter154.write(oprot); + _iter162.write(oprot); } } } @@ -19343,14 +22100,14 @@ public class Hbase { BitSet incoming = iprot.readBitSet(2); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list155 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); - struct.success = new ArrayList(_list155.size); - for (int _i156 = 0; _i156 < _list155.size; ++_i156) + org.apache.thrift.protocol.TList _list163 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); + struct.success = new ArrayList(_list163.size); + for (int _i164 = 0; _i164 < _list163.size; ++_i164) { - TRowResult _elem157; // optional - _elem157 = new TRowResult(); - _elem157.read(iprot); - struct.success.add(_elem157); + TRowResult _elem165; + _elem165 = new TRowResult(); + _elem165.read(iprot); + struct.success.add(_elem165); } } struct.setSuccessIsSet(true); @@ -19365,7 +22122,7 @@ public class Hbase { } - public static class getRowTs_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + public static class getRowTs_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getRowTs_args"); private static final org.apache.thrift.protocol.TField TABLE_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("tableName", org.apache.thrift.protocol.TType.STRING, (short)1); @@ -19477,7 +22234,7 @@ public class Hbase { // isset id assignments private static final int __TIMESTAMP_ISSET_ID = 0; - private BitSet __isset_bit_vector = new BitSet(1); + 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); @@ -19516,8 +22273,7 @@ public class Hbase { * Performs a deep copy on other. */ public getRowTs_args(getRowTs_args other) { - __isset_bit_vector.clear(); - __isset_bit_vector.or(other.__isset_bit_vector); + __isset_bitfield = other.__isset_bitfield; if (other.isSetTableName()) { this.tableName = other.tableName; } @@ -19526,7 +22282,7 @@ public class Hbase { } this.timestamp = other.timestamp; if (other.isSetAttributes()) { - Map __this__attributes = new HashMap(); + Map __this__attributes = new HashMap(other.attributes.size()); for (Map.Entry other_element : other.attributes.entrySet()) { ByteBuffer other_element_key = other_element.getKey(); @@ -19652,16 +22408,16 @@ public class Hbase { } public void unsetTimestamp() { - __isset_bit_vector.clear(__TIMESTAMP_ISSET_ID); + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __TIMESTAMP_ISSET_ID); } /** Returns true if field timestamp is set (has been assigned a value) and false otherwise */ public boolean isSetTimestamp() { - return __isset_bit_vector.get(__TIMESTAMP_ISSET_ID); + return EncodingUtils.testBit(__isset_bitfield, __TIMESTAMP_ISSET_ID); } public void setTimestampIsSet(boolean value) { - __isset_bit_vector.set(__TIMESTAMP_ISSET_ID, value); + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __TIMESTAMP_ISSET_ID, value); } public int getAttributesSize() { @@ -19836,50 +22592,50 @@ public class Hbase { return 0; } + @Override public int compareTo(getRowTs_args other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; - getRowTs_args typedOther = (getRowTs_args)other; - lastComparison = Boolean.valueOf(isSetTableName()).compareTo(typedOther.isSetTableName()); + lastComparison = Boolean.valueOf(isSetTableName()).compareTo(other.isSetTableName()); if (lastComparison != 0) { return lastComparison; } if (isSetTableName()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tableName, typedOther.tableName); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tableName, other.tableName); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetRow()).compareTo(typedOther.isSetRow()); + lastComparison = Boolean.valueOf(isSetRow()).compareTo(other.isSetRow()); if (lastComparison != 0) { return lastComparison; } if (isSetRow()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.row, typedOther.row); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.row, other.row); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetTimestamp()).compareTo(typedOther.isSetTimestamp()); + lastComparison = Boolean.valueOf(isSetTimestamp()).compareTo(other.isSetTimestamp()); if (lastComparison != 0) { return lastComparison; } if (isSetTimestamp()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.timestamp, typedOther.timestamp); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.timestamp, other.timestamp); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetAttributes()).compareTo(typedOther.isSetAttributes()); + lastComparison = Boolean.valueOf(isSetAttributes()).compareTo(other.isSetAttributes()); if (lastComparison != 0) { return lastComparison; } if (isSetAttributes()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.attributes, typedOther.attributes); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.attributes, other.attributes); if (lastComparison != 0) { return lastComparison; } @@ -19937,6 +22693,7 @@ public class Hbase { 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 { @@ -19950,7 +22707,7 @@ public class Hbase { 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_bit_vector = new BitSet(1); + __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); @@ -20002,15 +22759,15 @@ public class Hbase { case 4: // ATTRIBUTES if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { { - org.apache.thrift.protocol.TMap _map158 = iprot.readMapBegin(); - struct.attributes = new HashMap(2*_map158.size); - for (int _i159 = 0; _i159 < _map158.size; ++_i159) + org.apache.thrift.protocol.TMap _map166 = iprot.readMapBegin(); + struct.attributes = new HashMap(2*_map166.size); + for (int _i167 = 0; _i167 < _map166.size; ++_i167) { - ByteBuffer _key160; // required - ByteBuffer _val161; // optional - _key160 = iprot.readBinary(); - _val161 = iprot.readBinary(); - struct.attributes.put(_key160, _val161); + ByteBuffer _key168; + ByteBuffer _val169; + _key168 = iprot.readBinary(); + _val169 = iprot.readBinary(); + struct.attributes.put(_key168, _val169); } iprot.readMapEnd(); } @@ -20051,10 +22808,10 @@ public class Hbase { oprot.writeFieldBegin(ATTRIBUTES_FIELD_DESC); { oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.attributes.size())); - for (Map.Entry _iter162 : struct.attributes.entrySet()) + for (Map.Entry _iter170 : struct.attributes.entrySet()) { - oprot.writeBinary(_iter162.getKey()); - oprot.writeBinary(_iter162.getValue()); + oprot.writeBinary(_iter170.getKey()); + oprot.writeBinary(_iter170.getValue()); } oprot.writeMapEnd(); } @@ -20103,10 +22860,10 @@ public class Hbase { if (struct.isSetAttributes()) { { oprot.writeI32(struct.attributes.size()); - for (Map.Entry _iter163 : struct.attributes.entrySet()) + for (Map.Entry _iter171 : struct.attributes.entrySet()) { - oprot.writeBinary(_iter163.getKey()); - oprot.writeBinary(_iter163.getValue()); + oprot.writeBinary(_iter171.getKey()); + oprot.writeBinary(_iter171.getValue()); } } } @@ -20130,15 +22887,15 @@ public class Hbase { } if (incoming.get(3)) { { - org.apache.thrift.protocol.TMap _map164 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32()); - struct.attributes = new HashMap(2*_map164.size); - for (int _i165 = 0; _i165 < _map164.size; ++_i165) + org.apache.thrift.protocol.TMap _map172 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32()); + struct.attributes = new HashMap(2*_map172.size); + for (int _i173 = 0; _i173 < _map172.size; ++_i173) { - ByteBuffer _key166; // required - ByteBuffer _val167; // optional - _key166 = iprot.readBinary(); - _val167 = iprot.readBinary(); - struct.attributes.put(_key166, _val167); + ByteBuffer _key174; + ByteBuffer _val175; + _key174 = iprot.readBinary(); + _val175 = iprot.readBinary(); + struct.attributes.put(_key174, _val175); } } struct.setAttributesIsSet(true); @@ -20148,7 +22905,7 @@ public class Hbase { } - public static class getRowTs_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + public static class getRowTs_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getRowTs_result"); private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.LIST, (short)0); @@ -20254,7 +23011,7 @@ public class Hbase { */ public getRowTs_result(getRowTs_result other) { if (other.isSetSuccess()) { - List __this__success = new ArrayList(); + List __this__success = new ArrayList(other.success.size()); for (TRowResult other_element : other.success) { __this__success.add(new TRowResult(other_element)); } @@ -20425,30 +23182,30 @@ public class Hbase { return 0; } + @Override public int compareTo(getRowTs_result other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; - getRowTs_result typedOther = (getRowTs_result)other; - lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(typedOther.isSetSuccess()); + lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(other.isSetSuccess()); if (lastComparison != 0) { return lastComparison; } if (isSetSuccess()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, typedOther.success); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetIo()).compareTo(typedOther.isSetIo()); + lastComparison = Boolean.valueOf(isSetIo()).compareTo(other.isSetIo()); if (lastComparison != 0) { return lastComparison; } if (isSetIo()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.io, typedOther.io); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.io, other.io); if (lastComparison != 0) { return lastComparison; } @@ -20494,6 +23251,7 @@ public class Hbase { 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 { @@ -20533,14 +23291,14 @@ public class Hbase { case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list168 = iprot.readListBegin(); - struct.success = new ArrayList(_list168.size); - for (int _i169 = 0; _i169 < _list168.size; ++_i169) + org.apache.thrift.protocol.TList _list176 = iprot.readListBegin(); + struct.success = new ArrayList(_list176.size); + for (int _i177 = 0; _i177 < _list176.size; ++_i177) { - TRowResult _elem170; // optional - _elem170 = new TRowResult(); - _elem170.read(iprot); - struct.success.add(_elem170); + TRowResult _elem178; + _elem178 = new TRowResult(); + _elem178.read(iprot); + struct.success.add(_elem178); } iprot.readListEnd(); } @@ -20577,9 +23335,9 @@ public class Hbase { oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size())); - for (TRowResult _iter171 : struct.success) + for (TRowResult _iter179 : struct.success) { - _iter171.write(oprot); + _iter179.write(oprot); } oprot.writeListEnd(); } @@ -20618,9 +23376,9 @@ public class Hbase { if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (TRowResult _iter172 : struct.success) + for (TRowResult _iter180 : struct.success) { - _iter172.write(oprot); + _iter180.write(oprot); } } } @@ -20635,14 +23393,14 @@ public class Hbase { BitSet incoming = iprot.readBitSet(2); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list173 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); - struct.success = new ArrayList(_list173.size); - for (int _i174 = 0; _i174 < _list173.size; ++_i174) + org.apache.thrift.protocol.TList _list181 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); + struct.success = new ArrayList(_list181.size); + for (int _i182 = 0; _i182 < _list181.size; ++_i182) { - TRowResult _elem175; // optional - _elem175 = new TRowResult(); - _elem175.read(iprot); - struct.success.add(_elem175); + TRowResult _elem183; + _elem183 = new TRowResult(); + _elem183.read(iprot); + struct.success.add(_elem183); } } struct.setSuccessIsSet(true); @@ -20657,7 +23415,7 @@ public class Hbase { } - public static class getRowWithColumnsTs_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + public static class getRowWithColumnsTs_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getRowWithColumnsTs_args"); private static final org.apache.thrift.protocol.TField TABLE_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("tableName", org.apache.thrift.protocol.TType.STRING, (short)1); @@ -20774,7 +23532,7 @@ public class Hbase { // isset id assignments private static final int __TIMESTAMP_ISSET_ID = 0; - private BitSet __isset_bit_vector = new BitSet(1); + 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); @@ -20818,8 +23576,7 @@ public class Hbase { * Performs a deep copy on other. */ public getRowWithColumnsTs_args(getRowWithColumnsTs_args other) { - __isset_bit_vector.clear(); - __isset_bit_vector.or(other.__isset_bit_vector); + __isset_bitfield = other.__isset_bitfield; if (other.isSetTableName()) { this.tableName = other.tableName; } @@ -20827,7 +23584,7 @@ public class Hbase { this.row = other.row; } if (other.isSetColumns()) { - List __this__columns = new ArrayList(); + List __this__columns = new ArrayList(other.columns.size()); for (ByteBuffer other_element : other.columns) { __this__columns.add(other_element); } @@ -20835,7 +23592,7 @@ public class Hbase { } this.timestamp = other.timestamp; if (other.isSetAttributes()) { - Map __this__attributes = new HashMap(); + Map __this__attributes = new HashMap(other.attributes.size()); for (Map.Entry other_element : other.attributes.entrySet()) { ByteBuffer other_element_key = other_element.getKey(); @@ -21001,16 +23758,16 @@ public class Hbase { } public void unsetTimestamp() { - __isset_bit_vector.clear(__TIMESTAMP_ISSET_ID); + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __TIMESTAMP_ISSET_ID); } /** Returns true if field timestamp is set (has been assigned a value) and false otherwise */ public boolean isSetTimestamp() { - return __isset_bit_vector.get(__TIMESTAMP_ISSET_ID); + return EncodingUtils.testBit(__isset_bitfield, __TIMESTAMP_ISSET_ID); } public void setTimestampIsSet(boolean value) { - __isset_bit_vector.set(__TIMESTAMP_ISSET_ID, value); + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __TIMESTAMP_ISSET_ID, value); } public int getAttributesSize() { @@ -21207,60 +23964,60 @@ public class Hbase { return 0; } + @Override public int compareTo(getRowWithColumnsTs_args other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; - getRowWithColumnsTs_args typedOther = (getRowWithColumnsTs_args)other; - lastComparison = Boolean.valueOf(isSetTableName()).compareTo(typedOther.isSetTableName()); + lastComparison = Boolean.valueOf(isSetTableName()).compareTo(other.isSetTableName()); if (lastComparison != 0) { return lastComparison; } if (isSetTableName()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tableName, typedOther.tableName); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tableName, other.tableName); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetRow()).compareTo(typedOther.isSetRow()); + lastComparison = Boolean.valueOf(isSetRow()).compareTo(other.isSetRow()); if (lastComparison != 0) { return lastComparison; } if (isSetRow()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.row, typedOther.row); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.row, other.row); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetColumns()).compareTo(typedOther.isSetColumns()); + lastComparison = Boolean.valueOf(isSetColumns()).compareTo(other.isSetColumns()); if (lastComparison != 0) { return lastComparison; } if (isSetColumns()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.columns, typedOther.columns); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.columns, other.columns); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetTimestamp()).compareTo(typedOther.isSetTimestamp()); + lastComparison = Boolean.valueOf(isSetTimestamp()).compareTo(other.isSetTimestamp()); if (lastComparison != 0) { return lastComparison; } if (isSetTimestamp()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.timestamp, typedOther.timestamp); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.timestamp, other.timestamp); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetAttributes()).compareTo(typedOther.isSetAttributes()); + lastComparison = Boolean.valueOf(isSetAttributes()).compareTo(other.isSetAttributes()); if (lastComparison != 0) { return lastComparison; } if (isSetAttributes()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.attributes, typedOther.attributes); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.attributes, other.attributes); if (lastComparison != 0) { return lastComparison; } @@ -21326,6 +24083,7 @@ public class Hbase { 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 { @@ -21339,7 +24097,7 @@ public class Hbase { 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_bit_vector = new BitSet(1); + __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); @@ -21383,13 +24141,13 @@ public class Hbase { case 3: // COLUMNS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list176 = iprot.readListBegin(); - struct.columns = new ArrayList(_list176.size); - for (int _i177 = 0; _i177 < _list176.size; ++_i177) + org.apache.thrift.protocol.TList _list184 = iprot.readListBegin(); + struct.columns = new ArrayList(_list184.size); + for (int _i185 = 0; _i185 < _list184.size; ++_i185) { - ByteBuffer _elem178; // optional - _elem178 = iprot.readBinary(); - struct.columns.add(_elem178); + ByteBuffer _elem186; + _elem186 = iprot.readBinary(); + struct.columns.add(_elem186); } iprot.readListEnd(); } @@ -21409,15 +24167,15 @@ public class Hbase { case 5: // ATTRIBUTES if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { { - org.apache.thrift.protocol.TMap _map179 = iprot.readMapBegin(); - struct.attributes = new HashMap(2*_map179.size); - for (int _i180 = 0; _i180 < _map179.size; ++_i180) + org.apache.thrift.protocol.TMap _map187 = iprot.readMapBegin(); + struct.attributes = new HashMap(2*_map187.size); + for (int _i188 = 0; _i188 < _map187.size; ++_i188) { - ByteBuffer _key181; // required - ByteBuffer _val182; // optional - _key181 = iprot.readBinary(); - _val182 = iprot.readBinary(); - struct.attributes.put(_key181, _val182); + ByteBuffer _key189; + ByteBuffer _val190; + _key189 = iprot.readBinary(); + _val190 = iprot.readBinary(); + struct.attributes.put(_key189, _val190); } iprot.readMapEnd(); } @@ -21455,9 +24213,9 @@ public class Hbase { oprot.writeFieldBegin(COLUMNS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.columns.size())); - for (ByteBuffer _iter183 : struct.columns) + for (ByteBuffer _iter191 : struct.columns) { - oprot.writeBinary(_iter183); + oprot.writeBinary(_iter191); } oprot.writeListEnd(); } @@ -21470,10 +24228,10 @@ public class Hbase { oprot.writeFieldBegin(ATTRIBUTES_FIELD_DESC); { oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.attributes.size())); - for (Map.Entry _iter184 : struct.attributes.entrySet()) + for (Map.Entry _iter192 : struct.attributes.entrySet()) { - oprot.writeBinary(_iter184.getKey()); - oprot.writeBinary(_iter184.getValue()); + oprot.writeBinary(_iter192.getKey()); + oprot.writeBinary(_iter192.getValue()); } oprot.writeMapEnd(); } @@ -21522,9 +24280,9 @@ public class Hbase { if (struct.isSetColumns()) { { oprot.writeI32(struct.columns.size()); - for (ByteBuffer _iter185 : struct.columns) + for (ByteBuffer _iter193 : struct.columns) { - oprot.writeBinary(_iter185); + oprot.writeBinary(_iter193); } } } @@ -21534,10 +24292,10 @@ public class Hbase { if (struct.isSetAttributes()) { { oprot.writeI32(struct.attributes.size()); - for (Map.Entry _iter186 : struct.attributes.entrySet()) + for (Map.Entry _iter194 : struct.attributes.entrySet()) { - oprot.writeBinary(_iter186.getKey()); - oprot.writeBinary(_iter186.getValue()); + oprot.writeBinary(_iter194.getKey()); + oprot.writeBinary(_iter194.getValue()); } } } @@ -21557,13 +24315,13 @@ public class Hbase { } if (incoming.get(2)) { { - org.apache.thrift.protocol.TList _list187 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); - struct.columns = new ArrayList(_list187.size); - for (int _i188 = 0; _i188 < _list187.size; ++_i188) + org.apache.thrift.protocol.TList _list195 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); + struct.columns = new ArrayList(_list195.size); + for (int _i196 = 0; _i196 < _list195.size; ++_i196) { - ByteBuffer _elem189; // optional - _elem189 = iprot.readBinary(); - struct.columns.add(_elem189); + ByteBuffer _elem197; + _elem197 = iprot.readBinary(); + struct.columns.add(_elem197); } } struct.setColumnsIsSet(true); @@ -21574,15 +24332,15 @@ public class Hbase { } if (incoming.get(4)) { { - org.apache.thrift.protocol.TMap _map190 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32()); - struct.attributes = new HashMap(2*_map190.size); - for (int _i191 = 0; _i191 < _map190.size; ++_i191) + org.apache.thrift.protocol.TMap _map198 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32()); + struct.attributes = new HashMap(2*_map198.size); + for (int _i199 = 0; _i199 < _map198.size; ++_i199) { - ByteBuffer _key192; // required - ByteBuffer _val193; // optional - _key192 = iprot.readBinary(); - _val193 = iprot.readBinary(); - struct.attributes.put(_key192, _val193); + ByteBuffer _key200; + ByteBuffer _val201; + _key200 = iprot.readBinary(); + _val201 = iprot.readBinary(); + struct.attributes.put(_key200, _val201); } } struct.setAttributesIsSet(true); @@ -21592,7 +24350,7 @@ public class Hbase { } - public static class getRowWithColumnsTs_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + public static class getRowWithColumnsTs_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getRowWithColumnsTs_result"); private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.LIST, (short)0); @@ -21698,7 +24456,7 @@ public class Hbase { */ public getRowWithColumnsTs_result(getRowWithColumnsTs_result other) { if (other.isSetSuccess()) { - List __this__success = new ArrayList(); + List __this__success = new ArrayList(other.success.size()); for (TRowResult other_element : other.success) { __this__success.add(new TRowResult(other_element)); } @@ -21869,30 +24627,30 @@ public class Hbase { return 0; } + @Override public int compareTo(getRowWithColumnsTs_result other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; - getRowWithColumnsTs_result typedOther = (getRowWithColumnsTs_result)other; - lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(typedOther.isSetSuccess()); + lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(other.isSetSuccess()); if (lastComparison != 0) { return lastComparison; } if (isSetSuccess()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, typedOther.success); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetIo()).compareTo(typedOther.isSetIo()); + lastComparison = Boolean.valueOf(isSetIo()).compareTo(other.isSetIo()); if (lastComparison != 0) { return lastComparison; } if (isSetIo()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.io, typedOther.io); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.io, other.io); if (lastComparison != 0) { return lastComparison; } @@ -21938,6 +24696,7 @@ public class Hbase { 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 { @@ -21977,14 +24736,14 @@ public class Hbase { case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list194 = iprot.readListBegin(); - struct.success = new ArrayList(_list194.size); - for (int _i195 = 0; _i195 < _list194.size; ++_i195) + org.apache.thrift.protocol.TList _list202 = iprot.readListBegin(); + struct.success = new ArrayList(_list202.size); + for (int _i203 = 0; _i203 < _list202.size; ++_i203) { - TRowResult _elem196; // optional - _elem196 = new TRowResult(); - _elem196.read(iprot); - struct.success.add(_elem196); + TRowResult _elem204; + _elem204 = new TRowResult(); + _elem204.read(iprot); + struct.success.add(_elem204); } iprot.readListEnd(); } @@ -22021,9 +24780,9 @@ public class Hbase { oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size())); - for (TRowResult _iter197 : struct.success) + for (TRowResult _iter205 : struct.success) { - _iter197.write(oprot); + _iter205.write(oprot); } oprot.writeListEnd(); } @@ -22062,9 +24821,9 @@ public class Hbase { if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (TRowResult _iter198 : struct.success) + for (TRowResult _iter206 : struct.success) { - _iter198.write(oprot); + _iter206.write(oprot); } } } @@ -22079,14 +24838,14 @@ public class Hbase { BitSet incoming = iprot.readBitSet(2); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list199 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); - struct.success = new ArrayList(_list199.size); - for (int _i200 = 0; _i200 < _list199.size; ++_i200) + org.apache.thrift.protocol.TList _list207 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); + struct.success = new ArrayList(_list207.size); + for (int _i208 = 0; _i208 < _list207.size; ++_i208) { - TRowResult _elem201; // optional - _elem201 = new TRowResult(); - _elem201.read(iprot); - struct.success.add(_elem201); + TRowResult _elem209; + _elem209 = new TRowResult(); + _elem209.read(iprot); + struct.success.add(_elem209); } } struct.setSuccessIsSet(true); @@ -22101,7 +24860,7 @@ public class Hbase { } - public static class getRows_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + public static class getRows_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getRows_args"); private static final org.apache.thrift.protocol.TField TABLE_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("tableName", org.apache.thrift.protocol.TType.STRING, (short)1); @@ -22239,14 +24998,14 @@ public class Hbase { this.tableName = other.tableName; } if (other.isSetRows()) { - List __this__rows = new ArrayList(); + List __this__rows = new ArrayList(other.rows.size()); for (ByteBuffer other_element : other.rows) { __this__rows.add(other_element); } this.rows = __this__rows; } if (other.isSetAttributes()) { - Map __this__attributes = new HashMap(); + Map __this__attributes = new HashMap(other.attributes.size()); for (Map.Entry other_element : other.attributes.entrySet()) { ByteBuffer other_element_key = other_element.getKey(); @@ -22508,40 +25267,40 @@ public class Hbase { return 0; } + @Override public int compareTo(getRows_args other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; - getRows_args typedOther = (getRows_args)other; - lastComparison = Boolean.valueOf(isSetTableName()).compareTo(typedOther.isSetTableName()); + lastComparison = Boolean.valueOf(isSetTableName()).compareTo(other.isSetTableName()); if (lastComparison != 0) { return lastComparison; } if (isSetTableName()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tableName, typedOther.tableName); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tableName, other.tableName); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetRows()).compareTo(typedOther.isSetRows()); + lastComparison = Boolean.valueOf(isSetRows()).compareTo(other.isSetRows()); if (lastComparison != 0) { return lastComparison; } if (isSetRows()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.rows, typedOther.rows); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.rows, other.rows); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetAttributes()).compareTo(typedOther.isSetAttributes()); + lastComparison = Boolean.valueOf(isSetAttributes()).compareTo(other.isSetAttributes()); if (lastComparison != 0) { return lastComparison; } if (isSetAttributes()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.attributes, typedOther.attributes); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.attributes, other.attributes); if (lastComparison != 0) { return lastComparison; } @@ -22595,6 +25354,7 @@ public class Hbase { 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 { @@ -22642,13 +25402,13 @@ public class Hbase { case 2: // ROWS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list202 = iprot.readListBegin(); - struct.rows = new ArrayList(_list202.size); - for (int _i203 = 0; _i203 < _list202.size; ++_i203) + org.apache.thrift.protocol.TList _list210 = iprot.readListBegin(); + struct.rows = new ArrayList(_list210.size); + for (int _i211 = 0; _i211 < _list210.size; ++_i211) { - ByteBuffer _elem204; // optional - _elem204 = iprot.readBinary(); - struct.rows.add(_elem204); + ByteBuffer _elem212; + _elem212 = iprot.readBinary(); + struct.rows.add(_elem212); } iprot.readListEnd(); } @@ -22660,15 +25420,15 @@ public class Hbase { case 3: // ATTRIBUTES if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { { - org.apache.thrift.protocol.TMap _map205 = iprot.readMapBegin(); - struct.attributes = new HashMap(2*_map205.size); - for (int _i206 = 0; _i206 < _map205.size; ++_i206) + org.apache.thrift.protocol.TMap _map213 = iprot.readMapBegin(); + struct.attributes = new HashMap(2*_map213.size); + for (int _i214 = 0; _i214 < _map213.size; ++_i214) { - ByteBuffer _key207; // required - ByteBuffer _val208; // optional - _key207 = iprot.readBinary(); - _val208 = iprot.readBinary(); - struct.attributes.put(_key207, _val208); + ByteBuffer _key215; + ByteBuffer _val216; + _key215 = iprot.readBinary(); + _val216 = iprot.readBinary(); + struct.attributes.put(_key215, _val216); } iprot.readMapEnd(); } @@ -22701,9 +25461,9 @@ public class Hbase { oprot.writeFieldBegin(ROWS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.rows.size())); - for (ByteBuffer _iter209 : struct.rows) + for (ByteBuffer _iter217 : struct.rows) { - oprot.writeBinary(_iter209); + oprot.writeBinary(_iter217); } oprot.writeListEnd(); } @@ -22713,10 +25473,10 @@ public class Hbase { oprot.writeFieldBegin(ATTRIBUTES_FIELD_DESC); { oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.attributes.size())); - for (Map.Entry _iter210 : struct.attributes.entrySet()) + for (Map.Entry _iter218 : struct.attributes.entrySet()) { - oprot.writeBinary(_iter210.getKey()); - oprot.writeBinary(_iter210.getValue()); + oprot.writeBinary(_iter218.getKey()); + oprot.writeBinary(_iter218.getValue()); } oprot.writeMapEnd(); } @@ -22756,19 +25516,19 @@ public class Hbase { if (struct.isSetRows()) { { oprot.writeI32(struct.rows.size()); - for (ByteBuffer _iter211 : struct.rows) + for (ByteBuffer _iter219 : struct.rows) { - oprot.writeBinary(_iter211); + oprot.writeBinary(_iter219); } } } if (struct.isSetAttributes()) { { oprot.writeI32(struct.attributes.size()); - for (Map.Entry _iter212 : struct.attributes.entrySet()) + for (Map.Entry _iter220 : struct.attributes.entrySet()) { - oprot.writeBinary(_iter212.getKey()); - oprot.writeBinary(_iter212.getValue()); + oprot.writeBinary(_iter220.getKey()); + oprot.writeBinary(_iter220.getValue()); } } } @@ -22784,28 +25544,28 @@ public class Hbase { } if (incoming.get(1)) { { - org.apache.thrift.protocol.TList _list213 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); - struct.rows = new ArrayList(_list213.size); - for (int _i214 = 0; _i214 < _list213.size; ++_i214) + org.apache.thrift.protocol.TList _list221 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); + struct.rows = new ArrayList(_list221.size); + for (int _i222 = 0; _i222 < _list221.size; ++_i222) { - ByteBuffer _elem215; // optional - _elem215 = iprot.readBinary(); - struct.rows.add(_elem215); + ByteBuffer _elem223; + _elem223 = iprot.readBinary(); + struct.rows.add(_elem223); } } struct.setRowsIsSet(true); } if (incoming.get(2)) { { - org.apache.thrift.protocol.TMap _map216 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32()); - struct.attributes = new HashMap(2*_map216.size); - for (int _i217 = 0; _i217 < _map216.size; ++_i217) + org.apache.thrift.protocol.TMap _map224 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32()); + struct.attributes = new HashMap(2*_map224.size); + for (int _i225 = 0; _i225 < _map224.size; ++_i225) { - ByteBuffer _key218; // required - ByteBuffer _val219; // optional - _key218 = iprot.readBinary(); - _val219 = iprot.readBinary(); - struct.attributes.put(_key218, _val219); + ByteBuffer _key226; + ByteBuffer _val227; + _key226 = iprot.readBinary(); + _val227 = iprot.readBinary(); + struct.attributes.put(_key226, _val227); } } struct.setAttributesIsSet(true); @@ -22815,7 +25575,7 @@ public class Hbase { } - public static class getRows_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + public static class getRows_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getRows_result"); private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.LIST, (short)0); @@ -22921,7 +25681,7 @@ public class Hbase { */ public getRows_result(getRows_result other) { if (other.isSetSuccess()) { - List __this__success = new ArrayList(); + List __this__success = new ArrayList(other.success.size()); for (TRowResult other_element : other.success) { __this__success.add(new TRowResult(other_element)); } @@ -23092,30 +25852,30 @@ public class Hbase { return 0; } + @Override public int compareTo(getRows_result other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; - getRows_result typedOther = (getRows_result)other; - lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(typedOther.isSetSuccess()); + lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(other.isSetSuccess()); if (lastComparison != 0) { return lastComparison; } if (isSetSuccess()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, typedOther.success); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetIo()).compareTo(typedOther.isSetIo()); + lastComparison = Boolean.valueOf(isSetIo()).compareTo(other.isSetIo()); if (lastComparison != 0) { return lastComparison; } if (isSetIo()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.io, typedOther.io); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.io, other.io); if (lastComparison != 0) { return lastComparison; } @@ -23161,6 +25921,7 @@ public class Hbase { 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 { @@ -23200,14 +25961,14 @@ public class Hbase { case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list220 = iprot.readListBegin(); - struct.success = new ArrayList(_list220.size); - for (int _i221 = 0; _i221 < _list220.size; ++_i221) + org.apache.thrift.protocol.TList _list228 = iprot.readListBegin(); + struct.success = new ArrayList(_list228.size); + for (int _i229 = 0; _i229 < _list228.size; ++_i229) { - TRowResult _elem222; // optional - _elem222 = new TRowResult(); - _elem222.read(iprot); - struct.success.add(_elem222); + TRowResult _elem230; + _elem230 = new TRowResult(); + _elem230.read(iprot); + struct.success.add(_elem230); } iprot.readListEnd(); } @@ -23244,9 +26005,9 @@ public class Hbase { oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size())); - for (TRowResult _iter223 : struct.success) + for (TRowResult _iter231 : struct.success) { - _iter223.write(oprot); + _iter231.write(oprot); } oprot.writeListEnd(); } @@ -23285,9 +26046,9 @@ public class Hbase { if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (TRowResult _iter224 : struct.success) + for (TRowResult _iter232 : struct.success) { - _iter224.write(oprot); + _iter232.write(oprot); } } } @@ -23302,14 +26063,14 @@ public class Hbase { BitSet incoming = iprot.readBitSet(2); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list225 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); - struct.success = new ArrayList(_list225.size); - for (int _i226 = 0; _i226 < _list225.size; ++_i226) + org.apache.thrift.protocol.TList _list233 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); + struct.success = new ArrayList(_list233.size); + for (int _i234 = 0; _i234 < _list233.size; ++_i234) { - TRowResult _elem227; // optional - _elem227 = new TRowResult(); - _elem227.read(iprot); - struct.success.add(_elem227); + TRowResult _elem235; + _elem235 = new TRowResult(); + _elem235.read(iprot); + struct.success.add(_elem235); } } struct.setSuccessIsSet(true); @@ -23324,7 +26085,7 @@ public class Hbase { } - public static class getRowsWithColumns_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + public static class getRowsWithColumns_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getRowsWithColumns_args"); private static final org.apache.thrift.protocol.TField TABLE_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("tableName", org.apache.thrift.protocol.TType.STRING, (short)1); @@ -23478,21 +26239,21 @@ public class Hbase { this.tableName = other.tableName; } if (other.isSetRows()) { - List __this__rows = new ArrayList(); + List __this__rows = new ArrayList(other.rows.size()); for (ByteBuffer other_element : other.rows) { __this__rows.add(other_element); } this.rows = __this__rows; } if (other.isSetColumns()) { - List __this__columns = new ArrayList(); + List __this__columns = new ArrayList(other.columns.size()); for (ByteBuffer other_element : other.columns) { __this__columns.add(other_element); } this.columns = __this__columns; } if (other.isSetAttributes()) { - Map __this__attributes = new HashMap(); + Map __this__attributes = new HashMap(other.attributes.size()); for (Map.Entry other_element : other.attributes.entrySet()) { ByteBuffer other_element_key = other_element.getKey(); @@ -23822,50 +26583,50 @@ public class Hbase { return 0; } + @Override public int compareTo(getRowsWithColumns_args other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; - getRowsWithColumns_args typedOther = (getRowsWithColumns_args)other; - lastComparison = Boolean.valueOf(isSetTableName()).compareTo(typedOther.isSetTableName()); + lastComparison = Boolean.valueOf(isSetTableName()).compareTo(other.isSetTableName()); if (lastComparison != 0) { return lastComparison; } if (isSetTableName()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tableName, typedOther.tableName); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tableName, other.tableName); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetRows()).compareTo(typedOther.isSetRows()); + lastComparison = Boolean.valueOf(isSetRows()).compareTo(other.isSetRows()); if (lastComparison != 0) { return lastComparison; } if (isSetRows()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.rows, typedOther.rows); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.rows, other.rows); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetColumns()).compareTo(typedOther.isSetColumns()); + lastComparison = Boolean.valueOf(isSetColumns()).compareTo(other.isSetColumns()); if (lastComparison != 0) { return lastComparison; } if (isSetColumns()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.columns, typedOther.columns); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.columns, other.columns); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetAttributes()).compareTo(typedOther.isSetAttributes()); + lastComparison = Boolean.valueOf(isSetAttributes()).compareTo(other.isSetAttributes()); if (lastComparison != 0) { return lastComparison; } if (isSetAttributes()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.attributes, typedOther.attributes); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.attributes, other.attributes); if (lastComparison != 0) { return lastComparison; } @@ -23927,6 +26688,7 @@ public class Hbase { 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 { @@ -23974,13 +26736,13 @@ public class Hbase { case 2: // ROWS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list228 = iprot.readListBegin(); - struct.rows = new ArrayList(_list228.size); - for (int _i229 = 0; _i229 < _list228.size; ++_i229) + org.apache.thrift.protocol.TList _list236 = iprot.readListBegin(); + struct.rows = new ArrayList(_list236.size); + for (int _i237 = 0; _i237 < _list236.size; ++_i237) { - ByteBuffer _elem230; // optional - _elem230 = iprot.readBinary(); - struct.rows.add(_elem230); + ByteBuffer _elem238; + _elem238 = iprot.readBinary(); + struct.rows.add(_elem238); } iprot.readListEnd(); } @@ -23992,13 +26754,13 @@ public class Hbase { case 3: // COLUMNS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list231 = iprot.readListBegin(); - struct.columns = new ArrayList(_list231.size); - for (int _i232 = 0; _i232 < _list231.size; ++_i232) + org.apache.thrift.protocol.TList _list239 = iprot.readListBegin(); + struct.columns = new ArrayList(_list239.size); + for (int _i240 = 0; _i240 < _list239.size; ++_i240) { - ByteBuffer _elem233; // optional - _elem233 = iprot.readBinary(); - struct.columns.add(_elem233); + ByteBuffer _elem241; + _elem241 = iprot.readBinary(); + struct.columns.add(_elem241); } iprot.readListEnd(); } @@ -24010,15 +26772,15 @@ public class Hbase { case 4: // ATTRIBUTES if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { { - org.apache.thrift.protocol.TMap _map234 = iprot.readMapBegin(); - struct.attributes = new HashMap(2*_map234.size); - for (int _i235 = 0; _i235 < _map234.size; ++_i235) + org.apache.thrift.protocol.TMap _map242 = iprot.readMapBegin(); + struct.attributes = new HashMap(2*_map242.size); + for (int _i243 = 0; _i243 < _map242.size; ++_i243) { - ByteBuffer _key236; // required - ByteBuffer _val237; // optional - _key236 = iprot.readBinary(); - _val237 = iprot.readBinary(); - struct.attributes.put(_key236, _val237); + ByteBuffer _key244; + ByteBuffer _val245; + _key244 = iprot.readBinary(); + _val245 = iprot.readBinary(); + struct.attributes.put(_key244, _val245); } iprot.readMapEnd(); } @@ -24051,9 +26813,9 @@ public class Hbase { oprot.writeFieldBegin(ROWS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.rows.size())); - for (ByteBuffer _iter238 : struct.rows) + for (ByteBuffer _iter246 : struct.rows) { - oprot.writeBinary(_iter238); + oprot.writeBinary(_iter246); } oprot.writeListEnd(); } @@ -24063,9 +26825,9 @@ public class Hbase { oprot.writeFieldBegin(COLUMNS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.columns.size())); - for (ByteBuffer _iter239 : struct.columns) + for (ByteBuffer _iter247 : struct.columns) { - oprot.writeBinary(_iter239); + oprot.writeBinary(_iter247); } oprot.writeListEnd(); } @@ -24075,10 +26837,10 @@ public class Hbase { oprot.writeFieldBegin(ATTRIBUTES_FIELD_DESC); { oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.attributes.size())); - for (Map.Entry _iter240 : struct.attributes.entrySet()) + for (Map.Entry _iter248 : struct.attributes.entrySet()) { - oprot.writeBinary(_iter240.getKey()); - oprot.writeBinary(_iter240.getValue()); + oprot.writeBinary(_iter248.getKey()); + oprot.writeBinary(_iter248.getValue()); } oprot.writeMapEnd(); } @@ -24121,28 +26883,28 @@ public class Hbase { if (struct.isSetRows()) { { oprot.writeI32(struct.rows.size()); - for (ByteBuffer _iter241 : struct.rows) + for (ByteBuffer _iter249 : struct.rows) { - oprot.writeBinary(_iter241); + oprot.writeBinary(_iter249); } } } if (struct.isSetColumns()) { { oprot.writeI32(struct.columns.size()); - for (ByteBuffer _iter242 : struct.columns) + for (ByteBuffer _iter250 : struct.columns) { - oprot.writeBinary(_iter242); + oprot.writeBinary(_iter250); } } } if (struct.isSetAttributes()) { { oprot.writeI32(struct.attributes.size()); - for (Map.Entry _iter243 : struct.attributes.entrySet()) + for (Map.Entry _iter251 : struct.attributes.entrySet()) { - oprot.writeBinary(_iter243.getKey()); - oprot.writeBinary(_iter243.getValue()); + oprot.writeBinary(_iter251.getKey()); + oprot.writeBinary(_iter251.getValue()); } } } @@ -24158,41 +26920,41 @@ public class Hbase { } if (incoming.get(1)) { { - org.apache.thrift.protocol.TList _list244 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); - struct.rows = new ArrayList(_list244.size); - for (int _i245 = 0; _i245 < _list244.size; ++_i245) + org.apache.thrift.protocol.TList _list252 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); + struct.rows = new ArrayList(_list252.size); + for (int _i253 = 0; _i253 < _list252.size; ++_i253) { - ByteBuffer _elem246; // optional - _elem246 = iprot.readBinary(); - struct.rows.add(_elem246); + ByteBuffer _elem254; + _elem254 = iprot.readBinary(); + struct.rows.add(_elem254); } } struct.setRowsIsSet(true); } if (incoming.get(2)) { { - org.apache.thrift.protocol.TList _list247 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); - struct.columns = new ArrayList(_list247.size); - for (int _i248 = 0; _i248 < _list247.size; ++_i248) + org.apache.thrift.protocol.TList _list255 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); + struct.columns = new ArrayList(_list255.size); + for (int _i256 = 0; _i256 < _list255.size; ++_i256) { - ByteBuffer _elem249; // optional - _elem249 = iprot.readBinary(); - struct.columns.add(_elem249); + ByteBuffer _elem257; + _elem257 = iprot.readBinary(); + struct.columns.add(_elem257); } } struct.setColumnsIsSet(true); } if (incoming.get(3)) { { - org.apache.thrift.protocol.TMap _map250 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32()); - struct.attributes = new HashMap(2*_map250.size); - for (int _i251 = 0; _i251 < _map250.size; ++_i251) + org.apache.thrift.protocol.TMap _map258 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32()); + struct.attributes = new HashMap(2*_map258.size); + for (int _i259 = 0; _i259 < _map258.size; ++_i259) { - ByteBuffer _key252; // required - ByteBuffer _val253; // optional - _key252 = iprot.readBinary(); - _val253 = iprot.readBinary(); - struct.attributes.put(_key252, _val253); + ByteBuffer _key260; + ByteBuffer _val261; + _key260 = iprot.readBinary(); + _val261 = iprot.readBinary(); + struct.attributes.put(_key260, _val261); } } struct.setAttributesIsSet(true); @@ -24202,7 +26964,7 @@ public class Hbase { } - public static class getRowsWithColumns_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + public static class getRowsWithColumns_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getRowsWithColumns_result"); private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.LIST, (short)0); @@ -24308,7 +27070,7 @@ public class Hbase { */ public getRowsWithColumns_result(getRowsWithColumns_result other) { if (other.isSetSuccess()) { - List __this__success = new ArrayList(); + List __this__success = new ArrayList(other.success.size()); for (TRowResult other_element : other.success) { __this__success.add(new TRowResult(other_element)); } @@ -24479,30 +27241,30 @@ public class Hbase { return 0; } + @Override public int compareTo(getRowsWithColumns_result other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; - getRowsWithColumns_result typedOther = (getRowsWithColumns_result)other; - lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(typedOther.isSetSuccess()); + lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(other.isSetSuccess()); if (lastComparison != 0) { return lastComparison; } if (isSetSuccess()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, typedOther.success); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetIo()).compareTo(typedOther.isSetIo()); + lastComparison = Boolean.valueOf(isSetIo()).compareTo(other.isSetIo()); if (lastComparison != 0) { return lastComparison; } if (isSetIo()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.io, typedOther.io); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.io, other.io); if (lastComparison != 0) { return lastComparison; } @@ -24548,6 +27310,7 @@ public class Hbase { 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 { @@ -24587,14 +27350,14 @@ public class Hbase { case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list254 = iprot.readListBegin(); - struct.success = new ArrayList(_list254.size); - for (int _i255 = 0; _i255 < _list254.size; ++_i255) + org.apache.thrift.protocol.TList _list262 = iprot.readListBegin(); + struct.success = new ArrayList(_list262.size); + for (int _i263 = 0; _i263 < _list262.size; ++_i263) { - TRowResult _elem256; // optional - _elem256 = new TRowResult(); - _elem256.read(iprot); - struct.success.add(_elem256); + TRowResult _elem264; + _elem264 = new TRowResult(); + _elem264.read(iprot); + struct.success.add(_elem264); } iprot.readListEnd(); } @@ -24631,9 +27394,9 @@ public class Hbase { oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size())); - for (TRowResult _iter257 : struct.success) + for (TRowResult _iter265 : struct.success) { - _iter257.write(oprot); + _iter265.write(oprot); } oprot.writeListEnd(); } @@ -24672,9 +27435,9 @@ public class Hbase { if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (TRowResult _iter258 : struct.success) + for (TRowResult _iter266 : struct.success) { - _iter258.write(oprot); + _iter266.write(oprot); } } } @@ -24689,14 +27452,14 @@ public class Hbase { BitSet incoming = iprot.readBitSet(2); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list259 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); - struct.success = new ArrayList(_list259.size); - for (int _i260 = 0; _i260 < _list259.size; ++_i260) + org.apache.thrift.protocol.TList _list267 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); + struct.success = new ArrayList(_list267.size); + for (int _i268 = 0; _i268 < _list267.size; ++_i268) { - TRowResult _elem261; // optional - _elem261 = new TRowResult(); - _elem261.read(iprot); - struct.success.add(_elem261); + TRowResult _elem269; + _elem269 = new TRowResult(); + _elem269.read(iprot); + struct.success.add(_elem269); } } struct.setSuccessIsSet(true); @@ -24711,7 +27474,7 @@ public class Hbase { } - public static class getRowsTs_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + public static class getRowsTs_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getRowsTs_args"); private static final org.apache.thrift.protocol.TField TABLE_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("tableName", org.apache.thrift.protocol.TType.STRING, (short)1); @@ -24823,7 +27586,7 @@ public class Hbase { // isset id assignments private static final int __TIMESTAMP_ISSET_ID = 0; - private BitSet __isset_bit_vector = new BitSet(1); + 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); @@ -24863,13 +27626,12 @@ public class Hbase { * Performs a deep copy on other. */ public getRowsTs_args(getRowsTs_args other) { - __isset_bit_vector.clear(); - __isset_bit_vector.or(other.__isset_bit_vector); + __isset_bitfield = other.__isset_bitfield; if (other.isSetTableName()) { this.tableName = other.tableName; } if (other.isSetRows()) { - List __this__rows = new ArrayList(); + List __this__rows = new ArrayList(other.rows.size()); for (ByteBuffer other_element : other.rows) { __this__rows.add(other_element); } @@ -24877,7 +27639,7 @@ public class Hbase { } this.timestamp = other.timestamp; if (other.isSetAttributes()) { - Map __this__attributes = new HashMap(); + Map __this__attributes = new HashMap(other.attributes.size()); for (Map.Entry other_element : other.attributes.entrySet()) { ByteBuffer other_element_key = other_element.getKey(); @@ -25008,16 +27770,16 @@ public class Hbase { } public void unsetTimestamp() { - __isset_bit_vector.clear(__TIMESTAMP_ISSET_ID); + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __TIMESTAMP_ISSET_ID); } /** Returns true if field timestamp is set (has been assigned a value) and false otherwise */ public boolean isSetTimestamp() { - return __isset_bit_vector.get(__TIMESTAMP_ISSET_ID); + return EncodingUtils.testBit(__isset_bitfield, __TIMESTAMP_ISSET_ID); } public void setTimestampIsSet(boolean value) { - __isset_bit_vector.set(__TIMESTAMP_ISSET_ID, value); + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __TIMESTAMP_ISSET_ID, value); } public int getAttributesSize() { @@ -25192,50 +27954,50 @@ public class Hbase { return 0; } + @Override public int compareTo(getRowsTs_args other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; - getRowsTs_args typedOther = (getRowsTs_args)other; - lastComparison = Boolean.valueOf(isSetTableName()).compareTo(typedOther.isSetTableName()); + lastComparison = Boolean.valueOf(isSetTableName()).compareTo(other.isSetTableName()); if (lastComparison != 0) { return lastComparison; } if (isSetTableName()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tableName, typedOther.tableName); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tableName, other.tableName); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetRows()).compareTo(typedOther.isSetRows()); + lastComparison = Boolean.valueOf(isSetRows()).compareTo(other.isSetRows()); if (lastComparison != 0) { return lastComparison; } if (isSetRows()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.rows, typedOther.rows); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.rows, other.rows); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetTimestamp()).compareTo(typedOther.isSetTimestamp()); + lastComparison = Boolean.valueOf(isSetTimestamp()).compareTo(other.isSetTimestamp()); if (lastComparison != 0) { return lastComparison; } if (isSetTimestamp()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.timestamp, typedOther.timestamp); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.timestamp, other.timestamp); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetAttributes()).compareTo(typedOther.isSetAttributes()); + lastComparison = Boolean.valueOf(isSetAttributes()).compareTo(other.isSetAttributes()); if (lastComparison != 0) { return lastComparison; } if (isSetAttributes()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.attributes, typedOther.attributes); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.attributes, other.attributes); if (lastComparison != 0) { return lastComparison; } @@ -25293,6 +28055,7 @@ public class Hbase { 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 { @@ -25306,7 +28069,7 @@ public class Hbase { 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_bit_vector = new BitSet(1); + __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); @@ -25342,13 +28105,13 @@ public class Hbase { case 2: // ROWS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list262 = iprot.readListBegin(); - struct.rows = new ArrayList(_list262.size); - for (int _i263 = 0; _i263 < _list262.size; ++_i263) + org.apache.thrift.protocol.TList _list270 = iprot.readListBegin(); + struct.rows = new ArrayList(_list270.size); + for (int _i271 = 0; _i271 < _list270.size; ++_i271) { - ByteBuffer _elem264; // optional - _elem264 = iprot.readBinary(); - struct.rows.add(_elem264); + ByteBuffer _elem272; + _elem272 = iprot.readBinary(); + struct.rows.add(_elem272); } iprot.readListEnd(); } @@ -25368,15 +28131,15 @@ public class Hbase { case 4: // ATTRIBUTES if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { { - org.apache.thrift.protocol.TMap _map265 = iprot.readMapBegin(); - struct.attributes = new HashMap(2*_map265.size); - for (int _i266 = 0; _i266 < _map265.size; ++_i266) + org.apache.thrift.protocol.TMap _map273 = iprot.readMapBegin(); + struct.attributes = new HashMap(2*_map273.size); + for (int _i274 = 0; _i274 < _map273.size; ++_i274) { - ByteBuffer _key267; // required - ByteBuffer _val268; // optional - _key267 = iprot.readBinary(); - _val268 = iprot.readBinary(); - struct.attributes.put(_key267, _val268); + ByteBuffer _key275; + ByteBuffer _val276; + _key275 = iprot.readBinary(); + _val276 = iprot.readBinary(); + struct.attributes.put(_key275, _val276); } iprot.readMapEnd(); } @@ -25409,9 +28172,9 @@ public class Hbase { oprot.writeFieldBegin(ROWS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.rows.size())); - for (ByteBuffer _iter269 : struct.rows) + for (ByteBuffer _iter277 : struct.rows) { - oprot.writeBinary(_iter269); + oprot.writeBinary(_iter277); } oprot.writeListEnd(); } @@ -25424,10 +28187,10 @@ public class Hbase { oprot.writeFieldBegin(ATTRIBUTES_FIELD_DESC); { oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.attributes.size())); - for (Map.Entry _iter270 : struct.attributes.entrySet()) + for (Map.Entry _iter278 : struct.attributes.entrySet()) { - oprot.writeBinary(_iter270.getKey()); - oprot.writeBinary(_iter270.getValue()); + oprot.writeBinary(_iter278.getKey()); + oprot.writeBinary(_iter278.getValue()); } oprot.writeMapEnd(); } @@ -25470,9 +28233,9 @@ public class Hbase { if (struct.isSetRows()) { { oprot.writeI32(struct.rows.size()); - for (ByteBuffer _iter271 : struct.rows) + for (ByteBuffer _iter279 : struct.rows) { - oprot.writeBinary(_iter271); + oprot.writeBinary(_iter279); } } } @@ -25482,10 +28245,10 @@ public class Hbase { if (struct.isSetAttributes()) { { oprot.writeI32(struct.attributes.size()); - for (Map.Entry _iter272 : struct.attributes.entrySet()) + for (Map.Entry _iter280 : struct.attributes.entrySet()) { - oprot.writeBinary(_iter272.getKey()); - oprot.writeBinary(_iter272.getValue()); + oprot.writeBinary(_iter280.getKey()); + oprot.writeBinary(_iter280.getValue()); } } } @@ -25501,13 +28264,13 @@ public class Hbase { } if (incoming.get(1)) { { - org.apache.thrift.protocol.TList _list273 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); - struct.rows = new ArrayList(_list273.size); - for (int _i274 = 0; _i274 < _list273.size; ++_i274) + org.apache.thrift.protocol.TList _list281 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); + struct.rows = new ArrayList(_list281.size); + for (int _i282 = 0; _i282 < _list281.size; ++_i282) { - ByteBuffer _elem275; // optional - _elem275 = iprot.readBinary(); - struct.rows.add(_elem275); + ByteBuffer _elem283; + _elem283 = iprot.readBinary(); + struct.rows.add(_elem283); } } struct.setRowsIsSet(true); @@ -25518,15 +28281,15 @@ public class Hbase { } if (incoming.get(3)) { { - org.apache.thrift.protocol.TMap _map276 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32()); - struct.attributes = new HashMap(2*_map276.size); - for (int _i277 = 0; _i277 < _map276.size; ++_i277) + org.apache.thrift.protocol.TMap _map284 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32()); + struct.attributes = new HashMap(2*_map284.size); + for (int _i285 = 0; _i285 < _map284.size; ++_i285) { - ByteBuffer _key278; // required - ByteBuffer _val279; // optional - _key278 = iprot.readBinary(); - _val279 = iprot.readBinary(); - struct.attributes.put(_key278, _val279); + ByteBuffer _key286; + ByteBuffer _val287; + _key286 = iprot.readBinary(); + _val287 = iprot.readBinary(); + struct.attributes.put(_key286, _val287); } } struct.setAttributesIsSet(true); @@ -25536,7 +28299,7 @@ public class Hbase { } - public static class getRowsTs_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + public static class getRowsTs_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getRowsTs_result"); private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.LIST, (short)0); @@ -25642,7 +28405,7 @@ public class Hbase { */ public getRowsTs_result(getRowsTs_result other) { if (other.isSetSuccess()) { - List __this__success = new ArrayList(); + List __this__success = new ArrayList(other.success.size()); for (TRowResult other_element : other.success) { __this__success.add(new TRowResult(other_element)); } @@ -25813,30 +28576,30 @@ public class Hbase { return 0; } + @Override public int compareTo(getRowsTs_result other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; - getRowsTs_result typedOther = (getRowsTs_result)other; - lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(typedOther.isSetSuccess()); + lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(other.isSetSuccess()); if (lastComparison != 0) { return lastComparison; } if (isSetSuccess()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, typedOther.success); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetIo()).compareTo(typedOther.isSetIo()); + lastComparison = Boolean.valueOf(isSetIo()).compareTo(other.isSetIo()); if (lastComparison != 0) { return lastComparison; } if (isSetIo()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.io, typedOther.io); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.io, other.io); if (lastComparison != 0) { return lastComparison; } @@ -25882,6 +28645,7 @@ public class Hbase { 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 { @@ -25921,14 +28685,14 @@ public class Hbase { case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list280 = iprot.readListBegin(); - struct.success = new ArrayList(_list280.size); - for (int _i281 = 0; _i281 < _list280.size; ++_i281) + org.apache.thrift.protocol.TList _list288 = iprot.readListBegin(); + struct.success = new ArrayList(_list288.size); + for (int _i289 = 0; _i289 < _list288.size; ++_i289) { - TRowResult _elem282; // optional - _elem282 = new TRowResult(); - _elem282.read(iprot); - struct.success.add(_elem282); + TRowResult _elem290; + _elem290 = new TRowResult(); + _elem290.read(iprot); + struct.success.add(_elem290); } iprot.readListEnd(); } @@ -25965,9 +28729,9 @@ public class Hbase { oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size())); - for (TRowResult _iter283 : struct.success) + for (TRowResult _iter291 : struct.success) { - _iter283.write(oprot); + _iter291.write(oprot); } oprot.writeListEnd(); } @@ -26006,9 +28770,9 @@ public class Hbase { if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (TRowResult _iter284 : struct.success) + for (TRowResult _iter292 : struct.success) { - _iter284.write(oprot); + _iter292.write(oprot); } } } @@ -26023,14 +28787,14 @@ public class Hbase { BitSet incoming = iprot.readBitSet(2); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list285 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); - struct.success = new ArrayList(_list285.size); - for (int _i286 = 0; _i286 < _list285.size; ++_i286) + org.apache.thrift.protocol.TList _list293 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); + struct.success = new ArrayList(_list293.size); + for (int _i294 = 0; _i294 < _list293.size; ++_i294) { - TRowResult _elem287; // optional - _elem287 = new TRowResult(); - _elem287.read(iprot); - struct.success.add(_elem287); + TRowResult _elem295; + _elem295 = new TRowResult(); + _elem295.read(iprot); + struct.success.add(_elem295); } } struct.setSuccessIsSet(true); @@ -26045,7 +28809,7 @@ public class Hbase { } - public static class getRowsWithColumnsTs_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + public static class getRowsWithColumnsTs_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getRowsWithColumnsTs_args"); private static final org.apache.thrift.protocol.TField TABLE_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("tableName", org.apache.thrift.protocol.TType.STRING, (short)1); @@ -26162,7 +28926,7 @@ public class Hbase { // isset id assignments private static final int __TIMESTAMP_ISSET_ID = 0; - private BitSet __isset_bit_vector = new BitSet(1); + 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); @@ -26207,20 +28971,19 @@ public class Hbase { * Performs a deep copy on other. */ public getRowsWithColumnsTs_args(getRowsWithColumnsTs_args other) { - __isset_bit_vector.clear(); - __isset_bit_vector.or(other.__isset_bit_vector); + __isset_bitfield = other.__isset_bitfield; if (other.isSetTableName()) { this.tableName = other.tableName; } if (other.isSetRows()) { - List __this__rows = new ArrayList(); + List __this__rows = new ArrayList(other.rows.size()); for (ByteBuffer other_element : other.rows) { __this__rows.add(other_element); } this.rows = __this__rows; } if (other.isSetColumns()) { - List __this__columns = new ArrayList(); + List __this__columns = new ArrayList(other.columns.size()); for (ByteBuffer other_element : other.columns) { __this__columns.add(other_element); } @@ -26228,7 +28991,7 @@ public class Hbase { } this.timestamp = other.timestamp; if (other.isSetAttributes()) { - Map __this__attributes = new HashMap(); + Map __this__attributes = new HashMap(other.attributes.size()); for (Map.Entry other_element : other.attributes.entrySet()) { ByteBuffer other_element_key = other_element.getKey(); @@ -26399,16 +29162,16 @@ public class Hbase { } public void unsetTimestamp() { - __isset_bit_vector.clear(__TIMESTAMP_ISSET_ID); + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __TIMESTAMP_ISSET_ID); } /** Returns true if field timestamp is set (has been assigned a value) and false otherwise */ public boolean isSetTimestamp() { - return __isset_bit_vector.get(__TIMESTAMP_ISSET_ID); + return EncodingUtils.testBit(__isset_bitfield, __TIMESTAMP_ISSET_ID); } public void setTimestampIsSet(boolean value) { - __isset_bit_vector.set(__TIMESTAMP_ISSET_ID, value); + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __TIMESTAMP_ISSET_ID, value); } public int getAttributesSize() { @@ -26605,60 +29368,60 @@ public class Hbase { return 0; } + @Override public int compareTo(getRowsWithColumnsTs_args other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; - getRowsWithColumnsTs_args typedOther = (getRowsWithColumnsTs_args)other; - lastComparison = Boolean.valueOf(isSetTableName()).compareTo(typedOther.isSetTableName()); + lastComparison = Boolean.valueOf(isSetTableName()).compareTo(other.isSetTableName()); if (lastComparison != 0) { return lastComparison; } if (isSetTableName()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tableName, typedOther.tableName); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tableName, other.tableName); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetRows()).compareTo(typedOther.isSetRows()); + lastComparison = Boolean.valueOf(isSetRows()).compareTo(other.isSetRows()); if (lastComparison != 0) { return lastComparison; } if (isSetRows()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.rows, typedOther.rows); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.rows, other.rows); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetColumns()).compareTo(typedOther.isSetColumns()); + lastComparison = Boolean.valueOf(isSetColumns()).compareTo(other.isSetColumns()); if (lastComparison != 0) { return lastComparison; } if (isSetColumns()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.columns, typedOther.columns); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.columns, other.columns); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetTimestamp()).compareTo(typedOther.isSetTimestamp()); + lastComparison = Boolean.valueOf(isSetTimestamp()).compareTo(other.isSetTimestamp()); if (lastComparison != 0) { return lastComparison; } if (isSetTimestamp()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.timestamp, typedOther.timestamp); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.timestamp, other.timestamp); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetAttributes()).compareTo(typedOther.isSetAttributes()); + lastComparison = Boolean.valueOf(isSetAttributes()).compareTo(other.isSetAttributes()); if (lastComparison != 0) { return lastComparison; } if (isSetAttributes()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.attributes, typedOther.attributes); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.attributes, other.attributes); if (lastComparison != 0) { return lastComparison; } @@ -26724,6 +29487,7 @@ public class Hbase { 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 { @@ -26737,7 +29501,7 @@ public class Hbase { 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_bit_vector = new BitSet(1); + __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); @@ -26773,13 +29537,13 @@ public class Hbase { case 2: // ROWS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list288 = iprot.readListBegin(); - struct.rows = new ArrayList(_list288.size); - for (int _i289 = 0; _i289 < _list288.size; ++_i289) + org.apache.thrift.protocol.TList _list296 = iprot.readListBegin(); + struct.rows = new ArrayList(_list296.size); + for (int _i297 = 0; _i297 < _list296.size; ++_i297) { - ByteBuffer _elem290; // optional - _elem290 = iprot.readBinary(); - struct.rows.add(_elem290); + ByteBuffer _elem298; + _elem298 = iprot.readBinary(); + struct.rows.add(_elem298); } iprot.readListEnd(); } @@ -26791,13 +29555,13 @@ public class Hbase { case 3: // COLUMNS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list291 = iprot.readListBegin(); - struct.columns = new ArrayList(_list291.size); - for (int _i292 = 0; _i292 < _list291.size; ++_i292) + org.apache.thrift.protocol.TList _list299 = iprot.readListBegin(); + struct.columns = new ArrayList(_list299.size); + for (int _i300 = 0; _i300 < _list299.size; ++_i300) { - ByteBuffer _elem293; // optional - _elem293 = iprot.readBinary(); - struct.columns.add(_elem293); + ByteBuffer _elem301; + _elem301 = iprot.readBinary(); + struct.columns.add(_elem301); } iprot.readListEnd(); } @@ -26817,15 +29581,15 @@ public class Hbase { case 5: // ATTRIBUTES if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { { - org.apache.thrift.protocol.TMap _map294 = iprot.readMapBegin(); - struct.attributes = new HashMap(2*_map294.size); - for (int _i295 = 0; _i295 < _map294.size; ++_i295) + org.apache.thrift.protocol.TMap _map302 = iprot.readMapBegin(); + struct.attributes = new HashMap(2*_map302.size); + for (int _i303 = 0; _i303 < _map302.size; ++_i303) { - ByteBuffer _key296; // required - ByteBuffer _val297; // optional - _key296 = iprot.readBinary(); - _val297 = iprot.readBinary(); - struct.attributes.put(_key296, _val297); + ByteBuffer _key304; + ByteBuffer _val305; + _key304 = iprot.readBinary(); + _val305 = iprot.readBinary(); + struct.attributes.put(_key304, _val305); } iprot.readMapEnd(); } @@ -26858,9 +29622,9 @@ public class Hbase { oprot.writeFieldBegin(ROWS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.rows.size())); - for (ByteBuffer _iter298 : struct.rows) + for (ByteBuffer _iter306 : struct.rows) { - oprot.writeBinary(_iter298); + oprot.writeBinary(_iter306); } oprot.writeListEnd(); } @@ -26870,9 +29634,9 @@ public class Hbase { oprot.writeFieldBegin(COLUMNS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.columns.size())); - for (ByteBuffer _iter299 : struct.columns) + for (ByteBuffer _iter307 : struct.columns) { - oprot.writeBinary(_iter299); + oprot.writeBinary(_iter307); } oprot.writeListEnd(); } @@ -26885,10 +29649,10 @@ public class Hbase { oprot.writeFieldBegin(ATTRIBUTES_FIELD_DESC); { oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.attributes.size())); - for (Map.Entry _iter300 : struct.attributes.entrySet()) + for (Map.Entry _iter308 : struct.attributes.entrySet()) { - oprot.writeBinary(_iter300.getKey()); - oprot.writeBinary(_iter300.getValue()); + oprot.writeBinary(_iter308.getKey()); + oprot.writeBinary(_iter308.getValue()); } oprot.writeMapEnd(); } @@ -26934,18 +29698,18 @@ public class Hbase { if (struct.isSetRows()) { { oprot.writeI32(struct.rows.size()); - for (ByteBuffer _iter301 : struct.rows) + for (ByteBuffer _iter309 : struct.rows) { - oprot.writeBinary(_iter301); + oprot.writeBinary(_iter309); } } } if (struct.isSetColumns()) { { oprot.writeI32(struct.columns.size()); - for (ByteBuffer _iter302 : struct.columns) + for (ByteBuffer _iter310 : struct.columns) { - oprot.writeBinary(_iter302); + oprot.writeBinary(_iter310); } } } @@ -26955,10 +29719,10 @@ public class Hbase { if (struct.isSetAttributes()) { { oprot.writeI32(struct.attributes.size()); - for (Map.Entry _iter303 : struct.attributes.entrySet()) + for (Map.Entry _iter311 : struct.attributes.entrySet()) { - oprot.writeBinary(_iter303.getKey()); - oprot.writeBinary(_iter303.getValue()); + oprot.writeBinary(_iter311.getKey()); + oprot.writeBinary(_iter311.getValue()); } } } @@ -26974,26 +29738,26 @@ public class Hbase { } if (incoming.get(1)) { { - org.apache.thrift.protocol.TList _list304 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); - struct.rows = new ArrayList(_list304.size); - for (int _i305 = 0; _i305 < _list304.size; ++_i305) + org.apache.thrift.protocol.TList _list312 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); + struct.rows = new ArrayList(_list312.size); + for (int _i313 = 0; _i313 < _list312.size; ++_i313) { - ByteBuffer _elem306; // optional - _elem306 = iprot.readBinary(); - struct.rows.add(_elem306); + ByteBuffer _elem314; + _elem314 = iprot.readBinary(); + struct.rows.add(_elem314); } } struct.setRowsIsSet(true); } if (incoming.get(2)) { { - org.apache.thrift.protocol.TList _list307 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); - struct.columns = new ArrayList(_list307.size); - for (int _i308 = 0; _i308 < _list307.size; ++_i308) + org.apache.thrift.protocol.TList _list315 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); + struct.columns = new ArrayList(_list315.size); + for (int _i316 = 0; _i316 < _list315.size; ++_i316) { - ByteBuffer _elem309; // optional - _elem309 = iprot.readBinary(); - struct.columns.add(_elem309); + ByteBuffer _elem317; + _elem317 = iprot.readBinary(); + struct.columns.add(_elem317); } } struct.setColumnsIsSet(true); @@ -27004,15 +29768,15 @@ public class Hbase { } if (incoming.get(4)) { { - org.apache.thrift.protocol.TMap _map310 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32()); - struct.attributes = new HashMap(2*_map310.size); - for (int _i311 = 0; _i311 < _map310.size; ++_i311) + org.apache.thrift.protocol.TMap _map318 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32()); + struct.attributes = new HashMap(2*_map318.size); + for (int _i319 = 0; _i319 < _map318.size; ++_i319) { - ByteBuffer _key312; // required - ByteBuffer _val313; // optional - _key312 = iprot.readBinary(); - _val313 = iprot.readBinary(); - struct.attributes.put(_key312, _val313); + ByteBuffer _key320; + ByteBuffer _val321; + _key320 = iprot.readBinary(); + _val321 = iprot.readBinary(); + struct.attributes.put(_key320, _val321); } } struct.setAttributesIsSet(true); @@ -27022,7 +29786,7 @@ public class Hbase { } - public static class getRowsWithColumnsTs_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + public static class getRowsWithColumnsTs_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getRowsWithColumnsTs_result"); private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.LIST, (short)0); @@ -27128,7 +29892,7 @@ public class Hbase { */ public getRowsWithColumnsTs_result(getRowsWithColumnsTs_result other) { if (other.isSetSuccess()) { - List __this__success = new ArrayList(); + List __this__success = new ArrayList(other.success.size()); for (TRowResult other_element : other.success) { __this__success.add(new TRowResult(other_element)); } @@ -27299,30 +30063,30 @@ public class Hbase { return 0; } + @Override public int compareTo(getRowsWithColumnsTs_result other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; - getRowsWithColumnsTs_result typedOther = (getRowsWithColumnsTs_result)other; - lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(typedOther.isSetSuccess()); + lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(other.isSetSuccess()); if (lastComparison != 0) { return lastComparison; } if (isSetSuccess()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, typedOther.success); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetIo()).compareTo(typedOther.isSetIo()); + lastComparison = Boolean.valueOf(isSetIo()).compareTo(other.isSetIo()); if (lastComparison != 0) { return lastComparison; } if (isSetIo()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.io, typedOther.io); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.io, other.io); if (lastComparison != 0) { return lastComparison; } @@ -27368,6 +30132,7 @@ public class Hbase { 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 { @@ -27407,14 +30172,14 @@ public class Hbase { case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list314 = iprot.readListBegin(); - struct.success = new ArrayList(_list314.size); - for (int _i315 = 0; _i315 < _list314.size; ++_i315) + org.apache.thrift.protocol.TList _list322 = iprot.readListBegin(); + struct.success = new ArrayList(_list322.size); + for (int _i323 = 0; _i323 < _list322.size; ++_i323) { - TRowResult _elem316; // optional - _elem316 = new TRowResult(); - _elem316.read(iprot); - struct.success.add(_elem316); + TRowResult _elem324; + _elem324 = new TRowResult(); + _elem324.read(iprot); + struct.success.add(_elem324); } iprot.readListEnd(); } @@ -27451,9 +30216,9 @@ public class Hbase { oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size())); - for (TRowResult _iter317 : struct.success) + for (TRowResult _iter325 : struct.success) { - _iter317.write(oprot); + _iter325.write(oprot); } oprot.writeListEnd(); } @@ -27492,9 +30257,9 @@ public class Hbase { if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (TRowResult _iter318 : struct.success) + for (TRowResult _iter326 : struct.success) { - _iter318.write(oprot); + _iter326.write(oprot); } } } @@ -27509,14 +30274,14 @@ public class Hbase { BitSet incoming = iprot.readBitSet(2); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list319 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); - struct.success = new ArrayList(_list319.size); - for (int _i320 = 0; _i320 < _list319.size; ++_i320) + org.apache.thrift.protocol.TList _list327 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); + struct.success = new ArrayList(_list327.size); + for (int _i328 = 0; _i328 < _list327.size; ++_i328) { - TRowResult _elem321; // optional - _elem321 = new TRowResult(); - _elem321.read(iprot); - struct.success.add(_elem321); + TRowResult _elem329; + _elem329 = new TRowResult(); + _elem329.read(iprot); + struct.success.add(_elem329); } } struct.setSuccessIsSet(true); @@ -27531,7 +30296,7 @@ public class Hbase { } - public static class mutateRow_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + public static class mutateRow_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("mutateRow_args"); private static final org.apache.thrift.protocol.TField TABLE_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("tableName", org.apache.thrift.protocol.TType.STRING, (short)1); @@ -27687,14 +30452,14 @@ public class Hbase { this.row = other.row; } if (other.isSetMutations()) { - List __this__mutations = new ArrayList(); + List __this__mutations = new ArrayList(other.mutations.size()); for (Mutation other_element : other.mutations) { __this__mutations.add(new Mutation(other_element)); } this.mutations = __this__mutations; } if (other.isSetAttributes()) { - Map __this__attributes = new HashMap(); + Map __this__attributes = new HashMap(other.attributes.size()); for (Map.Entry other_element : other.attributes.entrySet()) { ByteBuffer other_element_key = other_element.getKey(); @@ -28019,50 +30784,50 @@ public class Hbase { return 0; } + @Override public int compareTo(mutateRow_args other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; - mutateRow_args typedOther = (mutateRow_args)other; - lastComparison = Boolean.valueOf(isSetTableName()).compareTo(typedOther.isSetTableName()); + lastComparison = Boolean.valueOf(isSetTableName()).compareTo(other.isSetTableName()); if (lastComparison != 0) { return lastComparison; } if (isSetTableName()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tableName, typedOther.tableName); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tableName, other.tableName); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetRow()).compareTo(typedOther.isSetRow()); + lastComparison = Boolean.valueOf(isSetRow()).compareTo(other.isSetRow()); if (lastComparison != 0) { return lastComparison; } if (isSetRow()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.row, typedOther.row); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.row, other.row); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetMutations()).compareTo(typedOther.isSetMutations()); + lastComparison = Boolean.valueOf(isSetMutations()).compareTo(other.isSetMutations()); if (lastComparison != 0) { return lastComparison; } if (isSetMutations()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.mutations, typedOther.mutations); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.mutations, other.mutations); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetAttributes()).compareTo(typedOther.isSetAttributes()); + lastComparison = Boolean.valueOf(isSetAttributes()).compareTo(other.isSetAttributes()); if (lastComparison != 0) { return lastComparison; } if (isSetAttributes()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.attributes, typedOther.attributes); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.attributes, other.attributes); if (lastComparison != 0) { return lastComparison; } @@ -28124,6 +30889,7 @@ public class Hbase { 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 { @@ -28179,14 +30945,14 @@ public class Hbase { case 3: // MUTATIONS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list322 = iprot.readListBegin(); - struct.mutations = new ArrayList(_list322.size); - for (int _i323 = 0; _i323 < _list322.size; ++_i323) + org.apache.thrift.protocol.TList _list330 = iprot.readListBegin(); + struct.mutations = new ArrayList(_list330.size); + for (int _i331 = 0; _i331 < _list330.size; ++_i331) { - Mutation _elem324; // optional - _elem324 = new Mutation(); - _elem324.read(iprot); - struct.mutations.add(_elem324); + Mutation _elem332; + _elem332 = new Mutation(); + _elem332.read(iprot); + struct.mutations.add(_elem332); } iprot.readListEnd(); } @@ -28198,15 +30964,15 @@ public class Hbase { case 4: // ATTRIBUTES if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { { - org.apache.thrift.protocol.TMap _map325 = iprot.readMapBegin(); - struct.attributes = new HashMap(2*_map325.size); - for (int _i326 = 0; _i326 < _map325.size; ++_i326) + org.apache.thrift.protocol.TMap _map333 = iprot.readMapBegin(); + struct.attributes = new HashMap(2*_map333.size); + for (int _i334 = 0; _i334 < _map333.size; ++_i334) { - ByteBuffer _key327; // required - ByteBuffer _val328; // optional - _key327 = iprot.readBinary(); - _val328 = iprot.readBinary(); - struct.attributes.put(_key327, _val328); + ByteBuffer _key335; + ByteBuffer _val336; + _key335 = iprot.readBinary(); + _val336 = iprot.readBinary(); + struct.attributes.put(_key335, _val336); } iprot.readMapEnd(); } @@ -28244,9 +31010,9 @@ public class Hbase { oprot.writeFieldBegin(MUTATIONS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.mutations.size())); - for (Mutation _iter329 : struct.mutations) + for (Mutation _iter337 : struct.mutations) { - _iter329.write(oprot); + _iter337.write(oprot); } oprot.writeListEnd(); } @@ -28256,10 +31022,10 @@ public class Hbase { oprot.writeFieldBegin(ATTRIBUTES_FIELD_DESC); { oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.attributes.size())); - for (Map.Entry _iter330 : struct.attributes.entrySet()) + for (Map.Entry _iter338 : struct.attributes.entrySet()) { - oprot.writeBinary(_iter330.getKey()); - oprot.writeBinary(_iter330.getValue()); + oprot.writeBinary(_iter338.getKey()); + oprot.writeBinary(_iter338.getValue()); } oprot.writeMapEnd(); } @@ -28305,19 +31071,19 @@ public class Hbase { if (struct.isSetMutations()) { { oprot.writeI32(struct.mutations.size()); - for (Mutation _iter331 : struct.mutations) + for (Mutation _iter339 : struct.mutations) { - _iter331.write(oprot); + _iter339.write(oprot); } } } if (struct.isSetAttributes()) { { oprot.writeI32(struct.attributes.size()); - for (Map.Entry _iter332 : struct.attributes.entrySet()) + for (Map.Entry _iter340 : struct.attributes.entrySet()) { - oprot.writeBinary(_iter332.getKey()); - oprot.writeBinary(_iter332.getValue()); + oprot.writeBinary(_iter340.getKey()); + oprot.writeBinary(_iter340.getValue()); } } } @@ -28337,29 +31103,29 @@ public class Hbase { } if (incoming.get(2)) { { - org.apache.thrift.protocol.TList _list333 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); - struct.mutations = new ArrayList(_list333.size); - for (int _i334 = 0; _i334 < _list333.size; ++_i334) + org.apache.thrift.protocol.TList _list341 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); + struct.mutations = new ArrayList(_list341.size); + for (int _i342 = 0; _i342 < _list341.size; ++_i342) { - Mutation _elem335; // optional - _elem335 = new Mutation(); - _elem335.read(iprot); - struct.mutations.add(_elem335); + Mutation _elem343; + _elem343 = new Mutation(); + _elem343.read(iprot); + struct.mutations.add(_elem343); } } struct.setMutationsIsSet(true); } if (incoming.get(3)) { { - org.apache.thrift.protocol.TMap _map336 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32()); - struct.attributes = new HashMap(2*_map336.size); - for (int _i337 = 0; _i337 < _map336.size; ++_i337) + org.apache.thrift.protocol.TMap _map344 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32()); + struct.attributes = new HashMap(2*_map344.size); + for (int _i345 = 0; _i345 < _map344.size; ++_i345) { - ByteBuffer _key338; // required - ByteBuffer _val339; // optional - _key338 = iprot.readBinary(); - _val339 = iprot.readBinary(); - struct.attributes.put(_key338, _val339); + ByteBuffer _key346; + ByteBuffer _val347; + _key346 = iprot.readBinary(); + _val347 = iprot.readBinary(); + struct.attributes.put(_key346, _val347); } } struct.setAttributesIsSet(true); @@ -28369,7 +31135,7 @@ public class Hbase { } - public static class mutateRow_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + public static class mutateRow_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("mutateRow_result"); private static final org.apache.thrift.protocol.TField IO_FIELD_DESC = new org.apache.thrift.protocol.TField("io", org.apache.thrift.protocol.TType.STRUCT, (short)1); @@ -28626,30 +31392,30 @@ public class Hbase { return 0; } + @Override public int compareTo(mutateRow_result other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; - mutateRow_result typedOther = (mutateRow_result)other; - lastComparison = Boolean.valueOf(isSetIo()).compareTo(typedOther.isSetIo()); + lastComparison = Boolean.valueOf(isSetIo()).compareTo(other.isSetIo()); if (lastComparison != 0) { return lastComparison; } if (isSetIo()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.io, typedOther.io); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.io, other.io); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetIa()).compareTo(typedOther.isSetIa()); + lastComparison = Boolean.valueOf(isSetIa()).compareTo(other.isSetIa()); if (lastComparison != 0) { return lastComparison; } if (isSetIa()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.ia, typedOther.ia); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.ia, other.ia); if (lastComparison != 0) { return lastComparison; } @@ -28695,6 +31461,7 @@ public class Hbase { 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 { @@ -28826,7 +31593,7 @@ public class Hbase { } - public static class mutateRowTs_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + public static class mutateRowTs_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("mutateRowTs_args"); private static final org.apache.thrift.protocol.TField TABLE_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("tableName", org.apache.thrift.protocol.TType.STRING, (short)1); @@ -28949,7 +31716,7 @@ public class Hbase { // isset id assignments private static final int __TIMESTAMP_ISSET_ID = 0; - private BitSet __isset_bit_vector = new BitSet(1); + 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); @@ -28993,8 +31760,7 @@ public class Hbase { * Performs a deep copy on other. */ public mutateRowTs_args(mutateRowTs_args other) { - __isset_bit_vector.clear(); - __isset_bit_vector.or(other.__isset_bit_vector); + __isset_bitfield = other.__isset_bitfield; if (other.isSetTableName()) { this.tableName = other.tableName; } @@ -29002,7 +31768,7 @@ public class Hbase { this.row = other.row; } if (other.isSetMutations()) { - List __this__mutations = new ArrayList(); + List __this__mutations = new ArrayList(other.mutations.size()); for (Mutation other_element : other.mutations) { __this__mutations.add(new Mutation(other_element)); } @@ -29010,7 +31776,7 @@ public class Hbase { } this.timestamp = other.timestamp; if (other.isSetAttributes()) { - Map __this__attributes = new HashMap(); + Map __this__attributes = new HashMap(other.attributes.size()); for (Map.Entry other_element : other.attributes.entrySet()) { ByteBuffer other_element_key = other_element.getKey(); @@ -29182,16 +31948,16 @@ public class Hbase { } public void unsetTimestamp() { - __isset_bit_vector.clear(__TIMESTAMP_ISSET_ID); + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __TIMESTAMP_ISSET_ID); } /** Returns true if field timestamp is set (has been assigned a value) and false otherwise */ public boolean isSetTimestamp() { - return __isset_bit_vector.get(__TIMESTAMP_ISSET_ID); + return EncodingUtils.testBit(__isset_bitfield, __TIMESTAMP_ISSET_ID); } public void setTimestampIsSet(boolean value) { - __isset_bit_vector.set(__TIMESTAMP_ISSET_ID, value); + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __TIMESTAMP_ISSET_ID, value); } public int getAttributesSize() { @@ -29388,60 +32154,60 @@ public class Hbase { return 0; } + @Override public int compareTo(mutateRowTs_args other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; - mutateRowTs_args typedOther = (mutateRowTs_args)other; - lastComparison = Boolean.valueOf(isSetTableName()).compareTo(typedOther.isSetTableName()); + lastComparison = Boolean.valueOf(isSetTableName()).compareTo(other.isSetTableName()); if (lastComparison != 0) { return lastComparison; } if (isSetTableName()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tableName, typedOther.tableName); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tableName, other.tableName); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetRow()).compareTo(typedOther.isSetRow()); + lastComparison = Boolean.valueOf(isSetRow()).compareTo(other.isSetRow()); if (lastComparison != 0) { return lastComparison; } if (isSetRow()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.row, typedOther.row); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.row, other.row); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetMutations()).compareTo(typedOther.isSetMutations()); + lastComparison = Boolean.valueOf(isSetMutations()).compareTo(other.isSetMutations()); if (lastComparison != 0) { return lastComparison; } if (isSetMutations()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.mutations, typedOther.mutations); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.mutations, other.mutations); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetTimestamp()).compareTo(typedOther.isSetTimestamp()); + lastComparison = Boolean.valueOf(isSetTimestamp()).compareTo(other.isSetTimestamp()); if (lastComparison != 0) { return lastComparison; } if (isSetTimestamp()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.timestamp, typedOther.timestamp); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.timestamp, other.timestamp); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetAttributes()).compareTo(typedOther.isSetAttributes()); + lastComparison = Boolean.valueOf(isSetAttributes()).compareTo(other.isSetAttributes()); if (lastComparison != 0) { return lastComparison; } if (isSetAttributes()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.attributes, typedOther.attributes); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.attributes, other.attributes); if (lastComparison != 0) { return lastComparison; } @@ -29507,6 +32273,7 @@ public class Hbase { 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 { @@ -29520,7 +32287,7 @@ public class Hbase { 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_bit_vector = new BitSet(1); + __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); @@ -29564,14 +32331,14 @@ public class Hbase { case 3: // MUTATIONS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list340 = iprot.readListBegin(); - struct.mutations = new ArrayList(_list340.size); - for (int _i341 = 0; _i341 < _list340.size; ++_i341) + org.apache.thrift.protocol.TList _list348 = iprot.readListBegin(); + struct.mutations = new ArrayList(_list348.size); + for (int _i349 = 0; _i349 < _list348.size; ++_i349) { - Mutation _elem342; // optional - _elem342 = new Mutation(); - _elem342.read(iprot); - struct.mutations.add(_elem342); + Mutation _elem350; + _elem350 = new Mutation(); + _elem350.read(iprot); + struct.mutations.add(_elem350); } iprot.readListEnd(); } @@ -29591,15 +32358,15 @@ public class Hbase { case 5: // ATTRIBUTES if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { { - org.apache.thrift.protocol.TMap _map343 = iprot.readMapBegin(); - struct.attributes = new HashMap(2*_map343.size); - for (int _i344 = 0; _i344 < _map343.size; ++_i344) + org.apache.thrift.protocol.TMap _map351 = iprot.readMapBegin(); + struct.attributes = new HashMap(2*_map351.size); + for (int _i352 = 0; _i352 < _map351.size; ++_i352) { - ByteBuffer _key345; // required - ByteBuffer _val346; // optional - _key345 = iprot.readBinary(); - _val346 = iprot.readBinary(); - struct.attributes.put(_key345, _val346); + ByteBuffer _key353; + ByteBuffer _val354; + _key353 = iprot.readBinary(); + _val354 = iprot.readBinary(); + struct.attributes.put(_key353, _val354); } iprot.readMapEnd(); } @@ -29637,9 +32404,9 @@ public class Hbase { oprot.writeFieldBegin(MUTATIONS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.mutations.size())); - for (Mutation _iter347 : struct.mutations) + for (Mutation _iter355 : struct.mutations) { - _iter347.write(oprot); + _iter355.write(oprot); } oprot.writeListEnd(); } @@ -29652,10 +32419,10 @@ public class Hbase { oprot.writeFieldBegin(ATTRIBUTES_FIELD_DESC); { oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.attributes.size())); - for (Map.Entry _iter348 : struct.attributes.entrySet()) + for (Map.Entry _iter356 : struct.attributes.entrySet()) { - oprot.writeBinary(_iter348.getKey()); - oprot.writeBinary(_iter348.getValue()); + oprot.writeBinary(_iter356.getKey()); + oprot.writeBinary(_iter356.getValue()); } oprot.writeMapEnd(); } @@ -29704,9 +32471,9 @@ public class Hbase { if (struct.isSetMutations()) { { oprot.writeI32(struct.mutations.size()); - for (Mutation _iter349 : struct.mutations) + for (Mutation _iter357 : struct.mutations) { - _iter349.write(oprot); + _iter357.write(oprot); } } } @@ -29716,10 +32483,10 @@ public class Hbase { if (struct.isSetAttributes()) { { oprot.writeI32(struct.attributes.size()); - for (Map.Entry _iter350 : struct.attributes.entrySet()) + for (Map.Entry _iter358 : struct.attributes.entrySet()) { - oprot.writeBinary(_iter350.getKey()); - oprot.writeBinary(_iter350.getValue()); + oprot.writeBinary(_iter358.getKey()); + oprot.writeBinary(_iter358.getValue()); } } } @@ -29739,14 +32506,14 @@ public class Hbase { } if (incoming.get(2)) { { - org.apache.thrift.protocol.TList _list351 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); - struct.mutations = new ArrayList(_list351.size); - for (int _i352 = 0; _i352 < _list351.size; ++_i352) + org.apache.thrift.protocol.TList _list359 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); + struct.mutations = new ArrayList(_list359.size); + for (int _i360 = 0; _i360 < _list359.size; ++_i360) { - Mutation _elem353; // optional - _elem353 = new Mutation(); - _elem353.read(iprot); - struct.mutations.add(_elem353); + Mutation _elem361; + _elem361 = new Mutation(); + _elem361.read(iprot); + struct.mutations.add(_elem361); } } struct.setMutationsIsSet(true); @@ -29757,15 +32524,15 @@ public class Hbase { } if (incoming.get(4)) { { - org.apache.thrift.protocol.TMap _map354 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32()); - struct.attributes = new HashMap(2*_map354.size); - for (int _i355 = 0; _i355 < _map354.size; ++_i355) + org.apache.thrift.protocol.TMap _map362 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32()); + struct.attributes = new HashMap(2*_map362.size); + for (int _i363 = 0; _i363 < _map362.size; ++_i363) { - ByteBuffer _key356; // required - ByteBuffer _val357; // optional - _key356 = iprot.readBinary(); - _val357 = iprot.readBinary(); - struct.attributes.put(_key356, _val357); + ByteBuffer _key364; + ByteBuffer _val365; + _key364 = iprot.readBinary(); + _val365 = iprot.readBinary(); + struct.attributes.put(_key364, _val365); } } struct.setAttributesIsSet(true); @@ -29775,7 +32542,7 @@ public class Hbase { } - public static class mutateRowTs_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + public static class mutateRowTs_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("mutateRowTs_result"); private static final org.apache.thrift.protocol.TField IO_FIELD_DESC = new org.apache.thrift.protocol.TField("io", org.apache.thrift.protocol.TType.STRUCT, (short)1); @@ -30032,30 +32799,30 @@ public class Hbase { return 0; } + @Override public int compareTo(mutateRowTs_result other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; - mutateRowTs_result typedOther = (mutateRowTs_result)other; - lastComparison = Boolean.valueOf(isSetIo()).compareTo(typedOther.isSetIo()); + lastComparison = Boolean.valueOf(isSetIo()).compareTo(other.isSetIo()); if (lastComparison != 0) { return lastComparison; } if (isSetIo()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.io, typedOther.io); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.io, other.io); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetIa()).compareTo(typedOther.isSetIa()); + lastComparison = Boolean.valueOf(isSetIa()).compareTo(other.isSetIa()); if (lastComparison != 0) { return lastComparison; } if (isSetIa()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.ia, typedOther.ia); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.ia, other.ia); if (lastComparison != 0) { return lastComparison; } @@ -30101,6 +32868,7 @@ public class Hbase { 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 { @@ -30232,7 +33000,7 @@ public class Hbase { } - public static class mutateRows_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + public static class mutateRows_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("mutateRows_args"); private static final org.apache.thrift.protocol.TField TABLE_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("tableName", org.apache.thrift.protocol.TType.STRING, (short)1); @@ -30370,14 +33138,14 @@ public class Hbase { this.tableName = other.tableName; } if (other.isSetRowBatches()) { - List __this__rowBatches = new ArrayList(); + List __this__rowBatches = new ArrayList(other.rowBatches.size()); for (BatchMutation other_element : other.rowBatches) { __this__rowBatches.add(new BatchMutation(other_element)); } this.rowBatches = __this__rowBatches; } if (other.isSetAttributes()) { - Map __this__attributes = new HashMap(); + Map __this__attributes = new HashMap(other.attributes.size()); for (Map.Entry other_element : other.attributes.entrySet()) { ByteBuffer other_element_key = other_element.getKey(); @@ -30639,40 +33407,40 @@ public class Hbase { return 0; } + @Override public int compareTo(mutateRows_args other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; - mutateRows_args typedOther = (mutateRows_args)other; - lastComparison = Boolean.valueOf(isSetTableName()).compareTo(typedOther.isSetTableName()); + lastComparison = Boolean.valueOf(isSetTableName()).compareTo(other.isSetTableName()); if (lastComparison != 0) { return lastComparison; } if (isSetTableName()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tableName, typedOther.tableName); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tableName, other.tableName); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetRowBatches()).compareTo(typedOther.isSetRowBatches()); + lastComparison = Boolean.valueOf(isSetRowBatches()).compareTo(other.isSetRowBatches()); if (lastComparison != 0) { return lastComparison; } if (isSetRowBatches()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.rowBatches, typedOther.rowBatches); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.rowBatches, other.rowBatches); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetAttributes()).compareTo(typedOther.isSetAttributes()); + lastComparison = Boolean.valueOf(isSetAttributes()).compareTo(other.isSetAttributes()); if (lastComparison != 0) { return lastComparison; } if (isSetAttributes()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.attributes, typedOther.attributes); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.attributes, other.attributes); if (lastComparison != 0) { return lastComparison; } @@ -30726,6 +33494,7 @@ public class Hbase { 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 { @@ -30773,14 +33542,14 @@ public class Hbase { case 2: // ROW_BATCHES if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list358 = iprot.readListBegin(); - struct.rowBatches = new ArrayList(_list358.size); - for (int _i359 = 0; _i359 < _list358.size; ++_i359) + org.apache.thrift.protocol.TList _list366 = iprot.readListBegin(); + struct.rowBatches = new ArrayList(_list366.size); + for (int _i367 = 0; _i367 < _list366.size; ++_i367) { - BatchMutation _elem360; // optional - _elem360 = new BatchMutation(); - _elem360.read(iprot); - struct.rowBatches.add(_elem360); + BatchMutation _elem368; + _elem368 = new BatchMutation(); + _elem368.read(iprot); + struct.rowBatches.add(_elem368); } iprot.readListEnd(); } @@ -30792,15 +33561,15 @@ public class Hbase { case 3: // ATTRIBUTES if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { { - org.apache.thrift.protocol.TMap _map361 = iprot.readMapBegin(); - struct.attributes = new HashMap(2*_map361.size); - for (int _i362 = 0; _i362 < _map361.size; ++_i362) + org.apache.thrift.protocol.TMap _map369 = iprot.readMapBegin(); + struct.attributes = new HashMap(2*_map369.size); + for (int _i370 = 0; _i370 < _map369.size; ++_i370) { - ByteBuffer _key363; // required - ByteBuffer _val364; // optional - _key363 = iprot.readBinary(); - _val364 = iprot.readBinary(); - struct.attributes.put(_key363, _val364); + ByteBuffer _key371; + ByteBuffer _val372; + _key371 = iprot.readBinary(); + _val372 = iprot.readBinary(); + struct.attributes.put(_key371, _val372); } iprot.readMapEnd(); } @@ -30833,9 +33602,9 @@ public class Hbase { oprot.writeFieldBegin(ROW_BATCHES_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.rowBatches.size())); - for (BatchMutation _iter365 : struct.rowBatches) + for (BatchMutation _iter373 : struct.rowBatches) { - _iter365.write(oprot); + _iter373.write(oprot); } oprot.writeListEnd(); } @@ -30845,10 +33614,10 @@ public class Hbase { oprot.writeFieldBegin(ATTRIBUTES_FIELD_DESC); { oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.attributes.size())); - for (Map.Entry _iter366 : struct.attributes.entrySet()) + for (Map.Entry _iter374 : struct.attributes.entrySet()) { - oprot.writeBinary(_iter366.getKey()); - oprot.writeBinary(_iter366.getValue()); + oprot.writeBinary(_iter374.getKey()); + oprot.writeBinary(_iter374.getValue()); } oprot.writeMapEnd(); } @@ -30888,19 +33657,19 @@ public class Hbase { if (struct.isSetRowBatches()) { { oprot.writeI32(struct.rowBatches.size()); - for (BatchMutation _iter367 : struct.rowBatches) + for (BatchMutation _iter375 : struct.rowBatches) { - _iter367.write(oprot); + _iter375.write(oprot); } } } if (struct.isSetAttributes()) { { oprot.writeI32(struct.attributes.size()); - for (Map.Entry _iter368 : struct.attributes.entrySet()) + for (Map.Entry _iter376 : struct.attributes.entrySet()) { - oprot.writeBinary(_iter368.getKey()); - oprot.writeBinary(_iter368.getValue()); + oprot.writeBinary(_iter376.getKey()); + oprot.writeBinary(_iter376.getValue()); } } } @@ -30916,29 +33685,29 @@ public class Hbase { } if (incoming.get(1)) { { - org.apache.thrift.protocol.TList _list369 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); - struct.rowBatches = new ArrayList(_list369.size); - for (int _i370 = 0; _i370 < _list369.size; ++_i370) + org.apache.thrift.protocol.TList _list377 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); + struct.rowBatches = new ArrayList(_list377.size); + for (int _i378 = 0; _i378 < _list377.size; ++_i378) { - BatchMutation _elem371; // optional - _elem371 = new BatchMutation(); - _elem371.read(iprot); - struct.rowBatches.add(_elem371); + BatchMutation _elem379; + _elem379 = new BatchMutation(); + _elem379.read(iprot); + struct.rowBatches.add(_elem379); } } struct.setRowBatchesIsSet(true); } if (incoming.get(2)) { { - org.apache.thrift.protocol.TMap _map372 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32()); - struct.attributes = new HashMap(2*_map372.size); - for (int _i373 = 0; _i373 < _map372.size; ++_i373) + org.apache.thrift.protocol.TMap _map380 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32()); + struct.attributes = new HashMap(2*_map380.size); + for (int _i381 = 0; _i381 < _map380.size; ++_i381) { - ByteBuffer _key374; // required - ByteBuffer _val375; // optional - _key374 = iprot.readBinary(); - _val375 = iprot.readBinary(); - struct.attributes.put(_key374, _val375); + ByteBuffer _key382; + ByteBuffer _val383; + _key382 = iprot.readBinary(); + _val383 = iprot.readBinary(); + struct.attributes.put(_key382, _val383); } } struct.setAttributesIsSet(true); @@ -30948,7 +33717,7 @@ public class Hbase { } - public static class mutateRows_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + public static class mutateRows_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("mutateRows_result"); private static final org.apache.thrift.protocol.TField IO_FIELD_DESC = new org.apache.thrift.protocol.TField("io", org.apache.thrift.protocol.TType.STRUCT, (short)1); @@ -31205,30 +33974,30 @@ public class Hbase { return 0; } + @Override public int compareTo(mutateRows_result other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; - mutateRows_result typedOther = (mutateRows_result)other; - lastComparison = Boolean.valueOf(isSetIo()).compareTo(typedOther.isSetIo()); + lastComparison = Boolean.valueOf(isSetIo()).compareTo(other.isSetIo()); if (lastComparison != 0) { return lastComparison; } if (isSetIo()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.io, typedOther.io); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.io, other.io); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetIa()).compareTo(typedOther.isSetIa()); + lastComparison = Boolean.valueOf(isSetIa()).compareTo(other.isSetIa()); if (lastComparison != 0) { return lastComparison; } if (isSetIa()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.ia, typedOther.ia); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.ia, other.ia); if (lastComparison != 0) { return lastComparison; } @@ -31274,6 +34043,7 @@ public class Hbase { 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 { @@ -31405,7 +34175,7 @@ public class Hbase { } - public static class mutateRowsTs_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + public static class mutateRowsTs_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("mutateRowsTs_args"); private static final org.apache.thrift.protocol.TField TABLE_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("tableName", org.apache.thrift.protocol.TType.STRING, (short)1); @@ -31517,7 +34287,7 @@ public class Hbase { // isset id assignments private static final int __TIMESTAMP_ISSET_ID = 0; - private BitSet __isset_bit_vector = new BitSet(1); + 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); @@ -31557,13 +34327,12 @@ public class Hbase { * Performs a deep copy on other. */ public mutateRowsTs_args(mutateRowsTs_args other) { - __isset_bit_vector.clear(); - __isset_bit_vector.or(other.__isset_bit_vector); + __isset_bitfield = other.__isset_bitfield; if (other.isSetTableName()) { this.tableName = other.tableName; } if (other.isSetRowBatches()) { - List __this__rowBatches = new ArrayList(); + List __this__rowBatches = new ArrayList(other.rowBatches.size()); for (BatchMutation other_element : other.rowBatches) { __this__rowBatches.add(new BatchMutation(other_element)); } @@ -31571,7 +34340,7 @@ public class Hbase { } this.timestamp = other.timestamp; if (other.isSetAttributes()) { - Map __this__attributes = new HashMap(); + Map __this__attributes = new HashMap(other.attributes.size()); for (Map.Entry other_element : other.attributes.entrySet()) { ByteBuffer other_element_key = other_element.getKey(); @@ -31702,16 +34471,16 @@ public class Hbase { } public void unsetTimestamp() { - __isset_bit_vector.clear(__TIMESTAMP_ISSET_ID); + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __TIMESTAMP_ISSET_ID); } /** Returns true if field timestamp is set (has been assigned a value) and false otherwise */ public boolean isSetTimestamp() { - return __isset_bit_vector.get(__TIMESTAMP_ISSET_ID); + return EncodingUtils.testBit(__isset_bitfield, __TIMESTAMP_ISSET_ID); } public void setTimestampIsSet(boolean value) { - __isset_bit_vector.set(__TIMESTAMP_ISSET_ID, value); + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __TIMESTAMP_ISSET_ID, value); } public int getAttributesSize() { @@ -31886,50 +34655,50 @@ public class Hbase { return 0; } + @Override public int compareTo(mutateRowsTs_args other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; - mutateRowsTs_args typedOther = (mutateRowsTs_args)other; - lastComparison = Boolean.valueOf(isSetTableName()).compareTo(typedOther.isSetTableName()); + lastComparison = Boolean.valueOf(isSetTableName()).compareTo(other.isSetTableName()); if (lastComparison != 0) { return lastComparison; } if (isSetTableName()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tableName, typedOther.tableName); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tableName, other.tableName); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetRowBatches()).compareTo(typedOther.isSetRowBatches()); + lastComparison = Boolean.valueOf(isSetRowBatches()).compareTo(other.isSetRowBatches()); if (lastComparison != 0) { return lastComparison; } if (isSetRowBatches()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.rowBatches, typedOther.rowBatches); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.rowBatches, other.rowBatches); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetTimestamp()).compareTo(typedOther.isSetTimestamp()); + lastComparison = Boolean.valueOf(isSetTimestamp()).compareTo(other.isSetTimestamp()); if (lastComparison != 0) { return lastComparison; } if (isSetTimestamp()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.timestamp, typedOther.timestamp); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.timestamp, other.timestamp); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetAttributes()).compareTo(typedOther.isSetAttributes()); + lastComparison = Boolean.valueOf(isSetAttributes()).compareTo(other.isSetAttributes()); if (lastComparison != 0) { return lastComparison; } if (isSetAttributes()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.attributes, typedOther.attributes); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.attributes, other.attributes); if (lastComparison != 0) { return lastComparison; } @@ -31987,6 +34756,7 @@ public class Hbase { 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 { @@ -31999,6 +34769,8 @@ public class Hbase { 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); @@ -32034,14 +34806,14 @@ public class Hbase { case 2: // ROW_BATCHES if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list376 = iprot.readListBegin(); - struct.rowBatches = new ArrayList(_list376.size); - for (int _i377 = 0; _i377 < _list376.size; ++_i377) + org.apache.thrift.protocol.TList _list384 = iprot.readListBegin(); + struct.rowBatches = new ArrayList(_list384.size); + for (int _i385 = 0; _i385 < _list384.size; ++_i385) { - BatchMutation _elem378; // optional - _elem378 = new BatchMutation(); - _elem378.read(iprot); - struct.rowBatches.add(_elem378); + BatchMutation _elem386; + _elem386 = new BatchMutation(); + _elem386.read(iprot); + struct.rowBatches.add(_elem386); } iprot.readListEnd(); } @@ -32061,15 +34833,15 @@ public class Hbase { case 4: // ATTRIBUTES if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { { - org.apache.thrift.protocol.TMap _map379 = iprot.readMapBegin(); - struct.attributes = new HashMap(2*_map379.size); - for (int _i380 = 0; _i380 < _map379.size; ++_i380) + org.apache.thrift.protocol.TMap _map387 = iprot.readMapBegin(); + struct.attributes = new HashMap(2*_map387.size); + for (int _i388 = 0; _i388 < _map387.size; ++_i388) { - ByteBuffer _key381; // required - ByteBuffer _val382; // optional - _key381 = iprot.readBinary(); - _val382 = iprot.readBinary(); - struct.attributes.put(_key381, _val382); + ByteBuffer _key389; + ByteBuffer _val390; + _key389 = iprot.readBinary(); + _val390 = iprot.readBinary(); + struct.attributes.put(_key389, _val390); } iprot.readMapEnd(); } @@ -32102,9 +34874,9 @@ public class Hbase { oprot.writeFieldBegin(ROW_BATCHES_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.rowBatches.size())); - for (BatchMutation _iter383 : struct.rowBatches) + for (BatchMutation _iter391 : struct.rowBatches) { - _iter383.write(oprot); + _iter391.write(oprot); } oprot.writeListEnd(); } @@ -32117,10 +34889,10 @@ public class Hbase { oprot.writeFieldBegin(ATTRIBUTES_FIELD_DESC); { oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.attributes.size())); - for (Map.Entry _iter384 : struct.attributes.entrySet()) + for (Map.Entry _iter392 : struct.attributes.entrySet()) { - oprot.writeBinary(_iter384.getKey()); - oprot.writeBinary(_iter384.getValue()); + oprot.writeBinary(_iter392.getKey()); + oprot.writeBinary(_iter392.getValue()); } oprot.writeMapEnd(); } @@ -32163,9 +34935,9 @@ public class Hbase { if (struct.isSetRowBatches()) { { oprot.writeI32(struct.rowBatches.size()); - for (BatchMutation _iter385 : struct.rowBatches) + for (BatchMutation _iter393 : struct.rowBatches) { - _iter385.write(oprot); + _iter393.write(oprot); } } } @@ -32175,10 +34947,10 @@ public class Hbase { if (struct.isSetAttributes()) { { oprot.writeI32(struct.attributes.size()); - for (Map.Entry _iter386 : struct.attributes.entrySet()) + for (Map.Entry _iter394 : struct.attributes.entrySet()) { - oprot.writeBinary(_iter386.getKey()); - oprot.writeBinary(_iter386.getValue()); + oprot.writeBinary(_iter394.getKey()); + oprot.writeBinary(_iter394.getValue()); } } } @@ -32194,14 +34966,14 @@ public class Hbase { } if (incoming.get(1)) { { - org.apache.thrift.protocol.TList _list387 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); - struct.rowBatches = new ArrayList(_list387.size); - for (int _i388 = 0; _i388 < _list387.size; ++_i388) + org.apache.thrift.protocol.TList _list395 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); + struct.rowBatches = new ArrayList(_list395.size); + for (int _i396 = 0; _i396 < _list395.size; ++_i396) { - BatchMutation _elem389; // optional - _elem389 = new BatchMutation(); - _elem389.read(iprot); - struct.rowBatches.add(_elem389); + BatchMutation _elem397; + _elem397 = new BatchMutation(); + _elem397.read(iprot); + struct.rowBatches.add(_elem397); } } struct.setRowBatchesIsSet(true); @@ -32212,15 +34984,15 @@ public class Hbase { } if (incoming.get(3)) { { - org.apache.thrift.protocol.TMap _map390 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32()); - struct.attributes = new HashMap(2*_map390.size); - for (int _i391 = 0; _i391 < _map390.size; ++_i391) + org.apache.thrift.protocol.TMap _map398 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32()); + struct.attributes = new HashMap(2*_map398.size); + for (int _i399 = 0; _i399 < _map398.size; ++_i399) { - ByteBuffer _key392; // required - ByteBuffer _val393; // optional - _key392 = iprot.readBinary(); - _val393 = iprot.readBinary(); - struct.attributes.put(_key392, _val393); + ByteBuffer _key400; + ByteBuffer _val401; + _key400 = iprot.readBinary(); + _val401 = iprot.readBinary(); + struct.attributes.put(_key400, _val401); } } struct.setAttributesIsSet(true); @@ -32230,7 +35002,7 @@ public class Hbase { } - public static class mutateRowsTs_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + public static class mutateRowsTs_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("mutateRowsTs_result"); private static final org.apache.thrift.protocol.TField IO_FIELD_DESC = new org.apache.thrift.protocol.TField("io", org.apache.thrift.protocol.TType.STRUCT, (short)1); @@ -32487,30 +35259,30 @@ public class Hbase { return 0; } + @Override public int compareTo(mutateRowsTs_result other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; - mutateRowsTs_result typedOther = (mutateRowsTs_result)other; - lastComparison = Boolean.valueOf(isSetIo()).compareTo(typedOther.isSetIo()); + lastComparison = Boolean.valueOf(isSetIo()).compareTo(other.isSetIo()); if (lastComparison != 0) { return lastComparison; } if (isSetIo()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.io, typedOther.io); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.io, other.io); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetIa()).compareTo(typedOther.isSetIa()); + lastComparison = Boolean.valueOf(isSetIa()).compareTo(other.isSetIa()); if (lastComparison != 0) { return lastComparison; } if (isSetIa()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.ia, typedOther.ia); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.ia, other.ia); if (lastComparison != 0) { return lastComparison; } @@ -32556,6 +35328,7 @@ public class Hbase { 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 { @@ -32687,7 +35460,7 @@ public class Hbase { } - public static class atomicIncrement_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + public static class atomicIncrement_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("atomicIncrement_args"); private static final org.apache.thrift.protocol.TField TABLE_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("tableName", org.apache.thrift.protocol.TType.STRING, (short)1); @@ -32799,7 +35572,7 @@ public class Hbase { // isset id assignments private static final int __VALUE_ISSET_ID = 0; - private BitSet __isset_bit_vector = new BitSet(1); + 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); @@ -32836,8 +35609,7 @@ public class Hbase { * Performs a deep copy on other. */ public atomicIncrement_args(atomicIncrement_args other) { - __isset_bit_vector.clear(); - __isset_bit_vector.or(other.__isset_bit_vector); + __isset_bitfield = other.__isset_bitfield; if (other.isSetTableName()) { this.tableName = other.tableName; } @@ -33000,16 +35772,16 @@ public class Hbase { } public void unsetValue() { - __isset_bit_vector.clear(__VALUE_ISSET_ID); + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __VALUE_ISSET_ID); } /** Returns true if field value is set (has been assigned a value) and false otherwise */ public boolean isSetValue() { - return __isset_bit_vector.get(__VALUE_ISSET_ID); + return EncodingUtils.testBit(__isset_bitfield, __VALUE_ISSET_ID); } public void setValueIsSet(boolean value) { - __isset_bit_vector.set(__VALUE_ISSET_ID, value); + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __VALUE_ISSET_ID, value); } public void setFieldValue(_Fields field, Object value) { @@ -33143,50 +35915,50 @@ public class Hbase { return 0; } + @Override public int compareTo(atomicIncrement_args other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; - atomicIncrement_args typedOther = (atomicIncrement_args)other; - lastComparison = Boolean.valueOf(isSetTableName()).compareTo(typedOther.isSetTableName()); + lastComparison = Boolean.valueOf(isSetTableName()).compareTo(other.isSetTableName()); if (lastComparison != 0) { return lastComparison; } if (isSetTableName()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tableName, typedOther.tableName); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tableName, other.tableName); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetRow()).compareTo(typedOther.isSetRow()); + lastComparison = Boolean.valueOf(isSetRow()).compareTo(other.isSetRow()); if (lastComparison != 0) { return lastComparison; } if (isSetRow()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.row, typedOther.row); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.row, other.row); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetColumn()).compareTo(typedOther.isSetColumn()); + lastComparison = Boolean.valueOf(isSetColumn()).compareTo(other.isSetColumn()); if (lastComparison != 0) { return lastComparison; } if (isSetColumn()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.column, typedOther.column); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.column, other.column); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetValue()).compareTo(typedOther.isSetValue()); + lastComparison = Boolean.valueOf(isSetValue()).compareTo(other.isSetValue()); if (lastComparison != 0) { return lastComparison; } if (isSetValue()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.value, typedOther.value); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.value, other.value); if (lastComparison != 0) { return lastComparison; } @@ -33244,6 +36016,7 @@ public class Hbase { 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 { @@ -33257,7 +36030,7 @@ public class Hbase { 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_bit_vector = new BitSet(1); + __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); @@ -33417,7 +36190,7 @@ public class Hbase { } - public static class atomicIncrement_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + public static class atomicIncrement_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("atomicIncrement_result"); private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.I64, (short)0); @@ -33500,7 +36273,7 @@ public class Hbase { // isset id assignments private static final int __SUCCESS_ISSET_ID = 0; - private BitSet __isset_bit_vector = new BitSet(1); + 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); @@ -33533,8 +36306,7 @@ public class Hbase { * Performs a deep copy on other. */ public atomicIncrement_result(atomicIncrement_result other) { - __isset_bit_vector.clear(); - __isset_bit_vector.or(other.__isset_bit_vector); + __isset_bitfield = other.__isset_bitfield; this.success = other.success; if (other.isSetIo()) { this.io = new IOError(other.io); @@ -33567,16 +36339,16 @@ public class Hbase { } public void unsetSuccess() { - __isset_bit_vector.clear(__SUCCESS_ISSET_ID); + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __SUCCESS_ISSET_ID); } /** Returns true if field success is set (has been assigned a value) and false otherwise */ public boolean isSetSuccess() { - return __isset_bit_vector.get(__SUCCESS_ISSET_ID); + return EncodingUtils.testBit(__isset_bitfield, __SUCCESS_ISSET_ID); } public void setSuccessIsSet(boolean value) { - __isset_bit_vector.set(__SUCCESS_ISSET_ID, value); + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __SUCCESS_ISSET_ID, value); } public IOError getIo() { @@ -33736,40 +36508,40 @@ public class Hbase { return 0; } + @Override public int compareTo(atomicIncrement_result other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; - atomicIncrement_result typedOther = (atomicIncrement_result)other; - lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(typedOther.isSetSuccess()); + lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(other.isSetSuccess()); if (lastComparison != 0) { return lastComparison; } if (isSetSuccess()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, typedOther.success); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetIo()).compareTo(typedOther.isSetIo()); + lastComparison = Boolean.valueOf(isSetIo()).compareTo(other.isSetIo()); if (lastComparison != 0) { return lastComparison; } if (isSetIo()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.io, typedOther.io); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.io, other.io); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetIa()).compareTo(typedOther.isSetIa()); + lastComparison = Boolean.valueOf(isSetIa()).compareTo(other.isSetIa()); if (lastComparison != 0) { return lastComparison; } if (isSetIa()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.ia, typedOther.ia); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.ia, other.ia); if (lastComparison != 0) { return lastComparison; } @@ -33819,6 +36591,7 @@ public class Hbase { 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 { @@ -33832,7 +36605,7 @@ public class Hbase { 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_bit_vector = new BitSet(1); + __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); @@ -33898,9 +36671,11 @@ public class Hbase { struct.validate(); oprot.writeStructBegin(STRUCT_DESC); - oprot.writeFieldBegin(SUCCESS_FIELD_DESC); - oprot.writeI64(struct.success); - oprot.writeFieldEnd(); + if (struct.isSetSuccess()) { + oprot.writeFieldBegin(SUCCESS_FIELD_DESC); + oprot.writeI64(struct.success); + oprot.writeFieldEnd(); + } if (struct.io != null) { oprot.writeFieldBegin(IO_FIELD_DESC); struct.io.write(oprot); @@ -33973,7 +36748,7 @@ public class Hbase { } - public static class deleteAll_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + public static class deleteAll_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("deleteAll_args"); private static final org.apache.thrift.protocol.TField TABLE_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("tableName", org.apache.thrift.protocol.TType.STRING, (short)1); @@ -34131,7 +36906,7 @@ public class Hbase { this.column = other.column; } if (other.isSetAttributes()) { - Map __this__attributes = new HashMap(); + Map __this__attributes = new HashMap(other.attributes.size()); for (Map.Entry other_element : other.attributes.entrySet()) { ByteBuffer other_element_key = other_element.getKey(); @@ -34451,50 +37226,50 @@ public class Hbase { return 0; } + @Override public int compareTo(deleteAll_args other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; - deleteAll_args typedOther = (deleteAll_args)other; - lastComparison = Boolean.valueOf(isSetTableName()).compareTo(typedOther.isSetTableName()); + lastComparison = Boolean.valueOf(isSetTableName()).compareTo(other.isSetTableName()); if (lastComparison != 0) { return lastComparison; } if (isSetTableName()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tableName, typedOther.tableName); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tableName, other.tableName); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetRow()).compareTo(typedOther.isSetRow()); + lastComparison = Boolean.valueOf(isSetRow()).compareTo(other.isSetRow()); if (lastComparison != 0) { return lastComparison; } if (isSetRow()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.row, typedOther.row); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.row, other.row); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetColumn()).compareTo(typedOther.isSetColumn()); + lastComparison = Boolean.valueOf(isSetColumn()).compareTo(other.isSetColumn()); if (lastComparison != 0) { return lastComparison; } if (isSetColumn()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.column, typedOther.column); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.column, other.column); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetAttributes()).compareTo(typedOther.isSetAttributes()); + lastComparison = Boolean.valueOf(isSetAttributes()).compareTo(other.isSetAttributes()); if (lastComparison != 0) { return lastComparison; } if (isSetAttributes()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.attributes, typedOther.attributes); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.attributes, other.attributes); if (lastComparison != 0) { return lastComparison; } @@ -34556,6 +37331,7 @@ public class Hbase { 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 { @@ -34619,15 +37395,15 @@ public class Hbase { case 4: // ATTRIBUTES if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { { - org.apache.thrift.protocol.TMap _map394 = iprot.readMapBegin(); - struct.attributes = new HashMap(2*_map394.size); - for (int _i395 = 0; _i395 < _map394.size; ++_i395) + org.apache.thrift.protocol.TMap _map402 = iprot.readMapBegin(); + struct.attributes = new HashMap(2*_map402.size); + for (int _i403 = 0; _i403 < _map402.size; ++_i403) { - ByteBuffer _key396; // required - ByteBuffer _val397; // optional - _key396 = iprot.readBinary(); - _val397 = iprot.readBinary(); - struct.attributes.put(_key396, _val397); + ByteBuffer _key404; + ByteBuffer _val405; + _key404 = iprot.readBinary(); + _val405 = iprot.readBinary(); + struct.attributes.put(_key404, _val405); } iprot.readMapEnd(); } @@ -34670,10 +37446,10 @@ public class Hbase { oprot.writeFieldBegin(ATTRIBUTES_FIELD_DESC); { oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.attributes.size())); - for (Map.Entry _iter398 : struct.attributes.entrySet()) + for (Map.Entry _iter406 : struct.attributes.entrySet()) { - oprot.writeBinary(_iter398.getKey()); - oprot.writeBinary(_iter398.getValue()); + oprot.writeBinary(_iter406.getKey()); + oprot.writeBinary(_iter406.getValue()); } oprot.writeMapEnd(); } @@ -34722,10 +37498,10 @@ public class Hbase { if (struct.isSetAttributes()) { { oprot.writeI32(struct.attributes.size()); - for (Map.Entry _iter399 : struct.attributes.entrySet()) + for (Map.Entry _iter407 : struct.attributes.entrySet()) { - oprot.writeBinary(_iter399.getKey()); - oprot.writeBinary(_iter399.getValue()); + oprot.writeBinary(_iter407.getKey()); + oprot.writeBinary(_iter407.getValue()); } } } @@ -34749,15 +37525,15 @@ public class Hbase { } if (incoming.get(3)) { { - org.apache.thrift.protocol.TMap _map400 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32()); - struct.attributes = new HashMap(2*_map400.size); - for (int _i401 = 0; _i401 < _map400.size; ++_i401) + org.apache.thrift.protocol.TMap _map408 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32()); + struct.attributes = new HashMap(2*_map408.size); + for (int _i409 = 0; _i409 < _map408.size; ++_i409) { - ByteBuffer _key402; // required - ByteBuffer _val403; // optional - _key402 = iprot.readBinary(); - _val403 = iprot.readBinary(); - struct.attributes.put(_key402, _val403); + ByteBuffer _key410; + ByteBuffer _val411; + _key410 = iprot.readBinary(); + _val411 = iprot.readBinary(); + struct.attributes.put(_key410, _val411); } } struct.setAttributesIsSet(true); @@ -34767,7 +37543,7 @@ public class Hbase { } - public static class deleteAll_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + public static class deleteAll_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("deleteAll_result"); private static final org.apache.thrift.protocol.TField IO_FIELD_DESC = new org.apache.thrift.protocol.TField("io", org.apache.thrift.protocol.TType.STRUCT, (short)1); @@ -34965,20 +37741,20 @@ public class Hbase { return 0; } + @Override public int compareTo(deleteAll_result other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; - deleteAll_result typedOther = (deleteAll_result)other; - lastComparison = Boolean.valueOf(isSetIo()).compareTo(typedOther.isSetIo()); + lastComparison = Boolean.valueOf(isSetIo()).compareTo(other.isSetIo()); if (lastComparison != 0) { return lastComparison; } if (isSetIo()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.io, typedOther.io); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.io, other.io); if (lastComparison != 0) { return lastComparison; } @@ -35016,6 +37792,7 @@ public class Hbase { 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 { @@ -35122,7 +37899,7 @@ public class Hbase { } - public static class deleteAllTs_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + public static class deleteAllTs_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("deleteAllTs_args"); private static final org.apache.thrift.protocol.TField TABLE_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("tableName", org.apache.thrift.protocol.TType.STRING, (short)1); @@ -35245,7 +38022,7 @@ public class Hbase { // isset id assignments private static final int __TIMESTAMP_ISSET_ID = 0; - private BitSet __isset_bit_vector = new BitSet(1); + 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); @@ -35288,8 +38065,7 @@ public class Hbase { * Performs a deep copy on other. */ public deleteAllTs_args(deleteAllTs_args other) { - __isset_bit_vector.clear(); - __isset_bit_vector.or(other.__isset_bit_vector); + __isset_bitfield = other.__isset_bitfield; if (other.isSetTableName()) { this.tableName = other.tableName; } @@ -35301,7 +38077,7 @@ public class Hbase { } this.timestamp = other.timestamp; if (other.isSetAttributes()) { - Map __this__attributes = new HashMap(); + Map __this__attributes = new HashMap(other.attributes.size()); for (Map.Entry other_element : other.attributes.entrySet()) { ByteBuffer other_element_key = other_element.getKey(); @@ -35468,16 +38244,16 @@ public class Hbase { } public void unsetTimestamp() { - __isset_bit_vector.clear(__TIMESTAMP_ISSET_ID); + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __TIMESTAMP_ISSET_ID); } /** Returns true if field timestamp is set (has been assigned a value) and false otherwise */ public boolean isSetTimestamp() { - return __isset_bit_vector.get(__TIMESTAMP_ISSET_ID); + return EncodingUtils.testBit(__isset_bitfield, __TIMESTAMP_ISSET_ID); } public void setTimestampIsSet(boolean value) { - __isset_bit_vector.set(__TIMESTAMP_ISSET_ID, value); + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __TIMESTAMP_ISSET_ID, value); } public int getAttributesSize() { @@ -35674,60 +38450,60 @@ public class Hbase { return 0; } + @Override public int compareTo(deleteAllTs_args other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; - deleteAllTs_args typedOther = (deleteAllTs_args)other; - lastComparison = Boolean.valueOf(isSetTableName()).compareTo(typedOther.isSetTableName()); + lastComparison = Boolean.valueOf(isSetTableName()).compareTo(other.isSetTableName()); if (lastComparison != 0) { return lastComparison; } if (isSetTableName()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tableName, typedOther.tableName); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tableName, other.tableName); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetRow()).compareTo(typedOther.isSetRow()); + lastComparison = Boolean.valueOf(isSetRow()).compareTo(other.isSetRow()); if (lastComparison != 0) { return lastComparison; } if (isSetRow()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.row, typedOther.row); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.row, other.row); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetColumn()).compareTo(typedOther.isSetColumn()); + lastComparison = Boolean.valueOf(isSetColumn()).compareTo(other.isSetColumn()); if (lastComparison != 0) { return lastComparison; } if (isSetColumn()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.column, typedOther.column); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.column, other.column); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetTimestamp()).compareTo(typedOther.isSetTimestamp()); + lastComparison = Boolean.valueOf(isSetTimestamp()).compareTo(other.isSetTimestamp()); if (lastComparison != 0) { return lastComparison; } if (isSetTimestamp()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.timestamp, typedOther.timestamp); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.timestamp, other.timestamp); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetAttributes()).compareTo(typedOther.isSetAttributes()); + lastComparison = Boolean.valueOf(isSetAttributes()).compareTo(other.isSetAttributes()); if (lastComparison != 0) { return lastComparison; } if (isSetAttributes()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.attributes, typedOther.attributes); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.attributes, other.attributes); if (lastComparison != 0) { return lastComparison; } @@ -35793,6 +38569,7 @@ public class Hbase { 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 { @@ -35806,7 +38583,7 @@ public class Hbase { 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_bit_vector = new BitSet(1); + __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); @@ -35866,15 +38643,15 @@ public class Hbase { case 5: // ATTRIBUTES if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { { - org.apache.thrift.protocol.TMap _map404 = iprot.readMapBegin(); - struct.attributes = new HashMap(2*_map404.size); - for (int _i405 = 0; _i405 < _map404.size; ++_i405) + org.apache.thrift.protocol.TMap _map412 = iprot.readMapBegin(); + struct.attributes = new HashMap(2*_map412.size); + for (int _i413 = 0; _i413 < _map412.size; ++_i413) { - ByteBuffer _key406; // required - ByteBuffer _val407; // optional - _key406 = iprot.readBinary(); - _val407 = iprot.readBinary(); - struct.attributes.put(_key406, _val407); + ByteBuffer _key414; + ByteBuffer _val415; + _key414 = iprot.readBinary(); + _val415 = iprot.readBinary(); + struct.attributes.put(_key414, _val415); } iprot.readMapEnd(); } @@ -35920,10 +38697,10 @@ public class Hbase { oprot.writeFieldBegin(ATTRIBUTES_FIELD_DESC); { oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.attributes.size())); - for (Map.Entry _iter408 : struct.attributes.entrySet()) + for (Map.Entry _iter416 : struct.attributes.entrySet()) { - oprot.writeBinary(_iter408.getKey()); - oprot.writeBinary(_iter408.getValue()); + oprot.writeBinary(_iter416.getKey()); + oprot.writeBinary(_iter416.getValue()); } oprot.writeMapEnd(); } @@ -35978,10 +38755,10 @@ public class Hbase { if (struct.isSetAttributes()) { { oprot.writeI32(struct.attributes.size()); - for (Map.Entry _iter409 : struct.attributes.entrySet()) + for (Map.Entry _iter417 : struct.attributes.entrySet()) { - oprot.writeBinary(_iter409.getKey()); - oprot.writeBinary(_iter409.getValue()); + oprot.writeBinary(_iter417.getKey()); + oprot.writeBinary(_iter417.getValue()); } } } @@ -36009,15 +38786,15 @@ public class Hbase { } if (incoming.get(4)) { { - org.apache.thrift.protocol.TMap _map410 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32()); - struct.attributes = new HashMap(2*_map410.size); - for (int _i411 = 0; _i411 < _map410.size; ++_i411) + org.apache.thrift.protocol.TMap _map418 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32()); + struct.attributes = new HashMap(2*_map418.size); + for (int _i419 = 0; _i419 < _map418.size; ++_i419) { - ByteBuffer _key412; // required - ByteBuffer _val413; // optional - _key412 = iprot.readBinary(); - _val413 = iprot.readBinary(); - struct.attributes.put(_key412, _val413); + ByteBuffer _key420; + ByteBuffer _val421; + _key420 = iprot.readBinary(); + _val421 = iprot.readBinary(); + struct.attributes.put(_key420, _val421); } } struct.setAttributesIsSet(true); @@ -36027,7 +38804,7 @@ public class Hbase { } - public static class deleteAllTs_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + public static class deleteAllTs_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("deleteAllTs_result"); private static final org.apache.thrift.protocol.TField IO_FIELD_DESC = new org.apache.thrift.protocol.TField("io", org.apache.thrift.protocol.TType.STRUCT, (short)1); @@ -36225,20 +39002,20 @@ public class Hbase { return 0; } + @Override public int compareTo(deleteAllTs_result other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; - deleteAllTs_result typedOther = (deleteAllTs_result)other; - lastComparison = Boolean.valueOf(isSetIo()).compareTo(typedOther.isSetIo()); + lastComparison = Boolean.valueOf(isSetIo()).compareTo(other.isSetIo()); if (lastComparison != 0) { return lastComparison; } if (isSetIo()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.io, typedOther.io); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.io, other.io); if (lastComparison != 0) { return lastComparison; } @@ -36276,6 +39053,7 @@ public class Hbase { 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 { @@ -36382,7 +39160,7 @@ public class Hbase { } - public static class deleteAllRow_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + public static class deleteAllRow_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("deleteAllRow_args"); private static final org.apache.thrift.protocol.TField TABLE_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("tableName", org.apache.thrift.protocol.TType.STRING, (short)1); @@ -36522,7 +39300,7 @@ public class Hbase { this.row = other.row; } if (other.isSetAttributes()) { - Map __this__attributes = new HashMap(); + Map __this__attributes = new HashMap(other.attributes.size()); for (Map.Entry other_element : other.attributes.entrySet()) { ByteBuffer other_element_key = other_element.getKey(); @@ -36779,40 +39557,40 @@ public class Hbase { return 0; } + @Override public int compareTo(deleteAllRow_args other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; - deleteAllRow_args typedOther = (deleteAllRow_args)other; - lastComparison = Boolean.valueOf(isSetTableName()).compareTo(typedOther.isSetTableName()); + lastComparison = Boolean.valueOf(isSetTableName()).compareTo(other.isSetTableName()); if (lastComparison != 0) { return lastComparison; } if (isSetTableName()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tableName, typedOther.tableName); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tableName, other.tableName); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetRow()).compareTo(typedOther.isSetRow()); + lastComparison = Boolean.valueOf(isSetRow()).compareTo(other.isSetRow()); if (lastComparison != 0) { return lastComparison; } if (isSetRow()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.row, typedOther.row); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.row, other.row); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetAttributes()).compareTo(typedOther.isSetAttributes()); + lastComparison = Boolean.valueOf(isSetAttributes()).compareTo(other.isSetAttributes()); if (lastComparison != 0) { return lastComparison; } if (isSetAttributes()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.attributes, typedOther.attributes); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.attributes, other.attributes); if (lastComparison != 0) { return lastComparison; } @@ -36866,6 +39644,7 @@ public class Hbase { 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 { @@ -36921,15 +39700,15 @@ public class Hbase { case 3: // ATTRIBUTES if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { { - org.apache.thrift.protocol.TMap _map414 = iprot.readMapBegin(); - struct.attributes = new HashMap(2*_map414.size); - for (int _i415 = 0; _i415 < _map414.size; ++_i415) + org.apache.thrift.protocol.TMap _map422 = iprot.readMapBegin(); + struct.attributes = new HashMap(2*_map422.size); + for (int _i423 = 0; _i423 < _map422.size; ++_i423) { - ByteBuffer _key416; // required - ByteBuffer _val417; // optional - _key416 = iprot.readBinary(); - _val417 = iprot.readBinary(); - struct.attributes.put(_key416, _val417); + ByteBuffer _key424; + ByteBuffer _val425; + _key424 = iprot.readBinary(); + _val425 = iprot.readBinary(); + struct.attributes.put(_key424, _val425); } iprot.readMapEnd(); } @@ -36967,10 +39746,10 @@ public class Hbase { oprot.writeFieldBegin(ATTRIBUTES_FIELD_DESC); { oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.attributes.size())); - for (Map.Entry _iter418 : struct.attributes.entrySet()) + for (Map.Entry _iter426 : struct.attributes.entrySet()) { - oprot.writeBinary(_iter418.getKey()); - oprot.writeBinary(_iter418.getValue()); + oprot.writeBinary(_iter426.getKey()); + oprot.writeBinary(_iter426.getValue()); } oprot.writeMapEnd(); } @@ -37013,10 +39792,10 @@ public class Hbase { if (struct.isSetAttributes()) { { oprot.writeI32(struct.attributes.size()); - for (Map.Entry _iter419 : struct.attributes.entrySet()) + for (Map.Entry _iter427 : struct.attributes.entrySet()) { - oprot.writeBinary(_iter419.getKey()); - oprot.writeBinary(_iter419.getValue()); + oprot.writeBinary(_iter427.getKey()); + oprot.writeBinary(_iter427.getValue()); } } } @@ -37036,15 +39815,15 @@ public class Hbase { } if (incoming.get(2)) { { - org.apache.thrift.protocol.TMap _map420 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32()); - struct.attributes = new HashMap(2*_map420.size); - for (int _i421 = 0; _i421 < _map420.size; ++_i421) + org.apache.thrift.protocol.TMap _map428 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32()); + struct.attributes = new HashMap(2*_map428.size); + for (int _i429 = 0; _i429 < _map428.size; ++_i429) { - ByteBuffer _key422; // required - ByteBuffer _val423; // optional - _key422 = iprot.readBinary(); - _val423 = iprot.readBinary(); - struct.attributes.put(_key422, _val423); + ByteBuffer _key430; + ByteBuffer _val431; + _key430 = iprot.readBinary(); + _val431 = iprot.readBinary(); + struct.attributes.put(_key430, _val431); } } struct.setAttributesIsSet(true); @@ -37054,7 +39833,7 @@ public class Hbase { } - public static class deleteAllRow_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + public static class deleteAllRow_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("deleteAllRow_result"); private static final org.apache.thrift.protocol.TField IO_FIELD_DESC = new org.apache.thrift.protocol.TField("io", org.apache.thrift.protocol.TType.STRUCT, (short)1); @@ -37252,20 +40031,20 @@ public class Hbase { return 0; } + @Override public int compareTo(deleteAllRow_result other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; - deleteAllRow_result typedOther = (deleteAllRow_result)other; - lastComparison = Boolean.valueOf(isSetIo()).compareTo(typedOther.isSetIo()); + lastComparison = Boolean.valueOf(isSetIo()).compareTo(other.isSetIo()); if (lastComparison != 0) { return lastComparison; } if (isSetIo()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.io, typedOther.io); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.io, other.io); if (lastComparison != 0) { return lastComparison; } @@ -37303,6 +40082,7 @@ public class Hbase { 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 { @@ -37409,7 +40189,7 @@ public class Hbase { } - public static class increment_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + public static class increment_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("increment_args"); private static final org.apache.thrift.protocol.TField INCREMENT_FIELD_DESC = new org.apache.thrift.protocol.TField("increment", org.apache.thrift.protocol.TType.STRUCT, (short)1); @@ -37619,20 +40399,20 @@ public class Hbase { return 0; } + @Override public int compareTo(increment_args other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; - increment_args typedOther = (increment_args)other; - lastComparison = Boolean.valueOf(isSetIncrement()).compareTo(typedOther.isSetIncrement()); + lastComparison = Boolean.valueOf(isSetIncrement()).compareTo(other.isSetIncrement()); if (lastComparison != 0) { return lastComparison; } if (isSetIncrement()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.increment, typedOther.increment); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.increment, other.increment); if (lastComparison != 0) { return lastComparison; } @@ -37670,6 +40450,10 @@ public class Hbase { public void validate() throws org.apache.thrift.TException { // check for required fields + // check for sub-struct validity + if (increment != null) { + increment.validate(); + } } private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { @@ -37776,7 +40560,7 @@ public class Hbase { } - public static class increment_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + public static class increment_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("increment_result"); private static final org.apache.thrift.protocol.TField IO_FIELD_DESC = new org.apache.thrift.protocol.TField("io", org.apache.thrift.protocol.TType.STRUCT, (short)1); @@ -37974,20 +40758,20 @@ public class Hbase { return 0; } + @Override public int compareTo(increment_result other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; - increment_result typedOther = (increment_result)other; - lastComparison = Boolean.valueOf(isSetIo()).compareTo(typedOther.isSetIo()); + lastComparison = Boolean.valueOf(isSetIo()).compareTo(other.isSetIo()); if (lastComparison != 0) { return lastComparison; } if (isSetIo()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.io, typedOther.io); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.io, other.io); if (lastComparison != 0) { return lastComparison; } @@ -38025,6 +40809,7 @@ public class Hbase { 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 { @@ -38131,7 +40916,7 @@ public class Hbase { } - public static class incrementRows_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + public static class incrementRows_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("incrementRows_args"); private static final org.apache.thrift.protocol.TField INCREMENTS_FIELD_DESC = new org.apache.thrift.protocol.TField("increments", org.apache.thrift.protocol.TType.LIST, (short)1); @@ -38234,7 +41019,7 @@ public class Hbase { */ public incrementRows_args(incrementRows_args other) { if (other.isSetIncrements()) { - List __this__increments = new ArrayList(); + List __this__increments = new ArrayList(other.increments.size()); for (TIncrement other_element : other.increments) { __this__increments.add(new TIncrement(other_element)); } @@ -38361,20 +41146,20 @@ public class Hbase { return 0; } + @Override public int compareTo(incrementRows_args other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; - incrementRows_args typedOther = (incrementRows_args)other; - lastComparison = Boolean.valueOf(isSetIncrements()).compareTo(typedOther.isSetIncrements()); + lastComparison = Boolean.valueOf(isSetIncrements()).compareTo(other.isSetIncrements()); if (lastComparison != 0) { return lastComparison; } if (isSetIncrements()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.increments, typedOther.increments); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.increments, other.increments); if (lastComparison != 0) { return lastComparison; } @@ -38412,6 +41197,7 @@ public class Hbase { 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 { @@ -38451,14 +41237,14 @@ public class Hbase { case 1: // INCREMENTS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list424 = iprot.readListBegin(); - struct.increments = new ArrayList(_list424.size); - for (int _i425 = 0; _i425 < _list424.size; ++_i425) + org.apache.thrift.protocol.TList _list432 = iprot.readListBegin(); + struct.increments = new ArrayList(_list432.size); + for (int _i433 = 0; _i433 < _list432.size; ++_i433) { - TIncrement _elem426; // optional - _elem426 = new TIncrement(); - _elem426.read(iprot); - struct.increments.add(_elem426); + TIncrement _elem434; + _elem434 = new TIncrement(); + _elem434.read(iprot); + struct.increments.add(_elem434); } iprot.readListEnd(); } @@ -38486,9 +41272,9 @@ public class Hbase { oprot.writeFieldBegin(INCREMENTS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.increments.size())); - for (TIncrement _iter427 : struct.increments) + for (TIncrement _iter435 : struct.increments) { - _iter427.write(oprot); + _iter435.write(oprot); } oprot.writeListEnd(); } @@ -38519,9 +41305,9 @@ public class Hbase { if (struct.isSetIncrements()) { { oprot.writeI32(struct.increments.size()); - for (TIncrement _iter428 : struct.increments) + for (TIncrement _iter436 : struct.increments) { - _iter428.write(oprot); + _iter436.write(oprot); } } } @@ -38533,14 +41319,14 @@ public class Hbase { BitSet incoming = iprot.readBitSet(1); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list429 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); - struct.increments = new ArrayList(_list429.size); - for (int _i430 = 0; _i430 < _list429.size; ++_i430) + org.apache.thrift.protocol.TList _list437 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); + struct.increments = new ArrayList(_list437.size); + for (int _i438 = 0; _i438 < _list437.size; ++_i438) { - TIncrement _elem431; // optional - _elem431 = new TIncrement(); - _elem431.read(iprot); - struct.increments.add(_elem431); + TIncrement _elem439; + _elem439 = new TIncrement(); + _elem439.read(iprot); + struct.increments.add(_elem439); } } struct.setIncrementsIsSet(true); @@ -38550,7 +41336,7 @@ public class Hbase { } - public static class incrementRows_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + public static class incrementRows_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("incrementRows_result"); private static final org.apache.thrift.protocol.TField IO_FIELD_DESC = new org.apache.thrift.protocol.TField("io", org.apache.thrift.protocol.TType.STRUCT, (short)1); @@ -38748,20 +41534,20 @@ public class Hbase { return 0; } + @Override public int compareTo(incrementRows_result other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; - incrementRows_result typedOther = (incrementRows_result)other; - lastComparison = Boolean.valueOf(isSetIo()).compareTo(typedOther.isSetIo()); + lastComparison = Boolean.valueOf(isSetIo()).compareTo(other.isSetIo()); if (lastComparison != 0) { return lastComparison; } if (isSetIo()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.io, typedOther.io); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.io, other.io); if (lastComparison != 0) { return lastComparison; } @@ -38799,6 +41585,7 @@ public class Hbase { 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 { @@ -38905,7 +41692,7 @@ public class Hbase { } - public static class deleteAllRowTs_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + public static class deleteAllRowTs_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("deleteAllRowTs_args"); private static final org.apache.thrift.protocol.TField TABLE_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("tableName", org.apache.thrift.protocol.TType.STRING, (short)1); @@ -39017,7 +41804,7 @@ public class Hbase { // isset id assignments private static final int __TIMESTAMP_ISSET_ID = 0; - private BitSet __isset_bit_vector = new BitSet(1); + 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); @@ -39056,8 +41843,7 @@ public class Hbase { * Performs a deep copy on other. */ public deleteAllRowTs_args(deleteAllRowTs_args other) { - __isset_bit_vector.clear(); - __isset_bit_vector.or(other.__isset_bit_vector); + __isset_bitfield = other.__isset_bitfield; if (other.isSetTableName()) { this.tableName = other.tableName; } @@ -39066,7 +41852,7 @@ public class Hbase { } this.timestamp = other.timestamp; if (other.isSetAttributes()) { - Map __this__attributes = new HashMap(); + Map __this__attributes = new HashMap(other.attributes.size()); for (Map.Entry other_element : other.attributes.entrySet()) { ByteBuffer other_element_key = other_element.getKey(); @@ -39192,16 +41978,16 @@ public class Hbase { } public void unsetTimestamp() { - __isset_bit_vector.clear(__TIMESTAMP_ISSET_ID); + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __TIMESTAMP_ISSET_ID); } /** Returns true if field timestamp is set (has been assigned a value) and false otherwise */ public boolean isSetTimestamp() { - return __isset_bit_vector.get(__TIMESTAMP_ISSET_ID); + return EncodingUtils.testBit(__isset_bitfield, __TIMESTAMP_ISSET_ID); } public void setTimestampIsSet(boolean value) { - __isset_bit_vector.set(__TIMESTAMP_ISSET_ID, value); + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __TIMESTAMP_ISSET_ID, value); } public int getAttributesSize() { @@ -39376,50 +42162,50 @@ public class Hbase { return 0; } + @Override public int compareTo(deleteAllRowTs_args other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; - deleteAllRowTs_args typedOther = (deleteAllRowTs_args)other; - lastComparison = Boolean.valueOf(isSetTableName()).compareTo(typedOther.isSetTableName()); + lastComparison = Boolean.valueOf(isSetTableName()).compareTo(other.isSetTableName()); if (lastComparison != 0) { return lastComparison; } if (isSetTableName()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tableName, typedOther.tableName); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tableName, other.tableName); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetRow()).compareTo(typedOther.isSetRow()); + lastComparison = Boolean.valueOf(isSetRow()).compareTo(other.isSetRow()); if (lastComparison != 0) { return lastComparison; } if (isSetRow()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.row, typedOther.row); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.row, other.row); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetTimestamp()).compareTo(typedOther.isSetTimestamp()); + lastComparison = Boolean.valueOf(isSetTimestamp()).compareTo(other.isSetTimestamp()); if (lastComparison != 0) { return lastComparison; } if (isSetTimestamp()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.timestamp, typedOther.timestamp); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.timestamp, other.timestamp); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetAttributes()).compareTo(typedOther.isSetAttributes()); + lastComparison = Boolean.valueOf(isSetAttributes()).compareTo(other.isSetAttributes()); if (lastComparison != 0) { return lastComparison; } if (isSetAttributes()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.attributes, typedOther.attributes); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.attributes, other.attributes); if (lastComparison != 0) { return lastComparison; } @@ -39477,6 +42263,7 @@ public class Hbase { 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 { @@ -39490,7 +42277,7 @@ public class Hbase { 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_bit_vector = new BitSet(1); + __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); @@ -39542,15 +42329,15 @@ public class Hbase { case 4: // ATTRIBUTES if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { { - org.apache.thrift.protocol.TMap _map432 = iprot.readMapBegin(); - struct.attributes = new HashMap(2*_map432.size); - for (int _i433 = 0; _i433 < _map432.size; ++_i433) + org.apache.thrift.protocol.TMap _map440 = iprot.readMapBegin(); + struct.attributes = new HashMap(2*_map440.size); + for (int _i441 = 0; _i441 < _map440.size; ++_i441) { - ByteBuffer _key434; // required - ByteBuffer _val435; // optional - _key434 = iprot.readBinary(); - _val435 = iprot.readBinary(); - struct.attributes.put(_key434, _val435); + ByteBuffer _key442; + ByteBuffer _val443; + _key442 = iprot.readBinary(); + _val443 = iprot.readBinary(); + struct.attributes.put(_key442, _val443); } iprot.readMapEnd(); } @@ -39591,10 +42378,10 @@ public class Hbase { oprot.writeFieldBegin(ATTRIBUTES_FIELD_DESC); { oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.attributes.size())); - for (Map.Entry _iter436 : struct.attributes.entrySet()) + for (Map.Entry _iter444 : struct.attributes.entrySet()) { - oprot.writeBinary(_iter436.getKey()); - oprot.writeBinary(_iter436.getValue()); + oprot.writeBinary(_iter444.getKey()); + oprot.writeBinary(_iter444.getValue()); } oprot.writeMapEnd(); } @@ -39643,10 +42430,10 @@ public class Hbase { if (struct.isSetAttributes()) { { oprot.writeI32(struct.attributes.size()); - for (Map.Entry _iter437 : struct.attributes.entrySet()) + for (Map.Entry _iter445 : struct.attributes.entrySet()) { - oprot.writeBinary(_iter437.getKey()); - oprot.writeBinary(_iter437.getValue()); + oprot.writeBinary(_iter445.getKey()); + oprot.writeBinary(_iter445.getValue()); } } } @@ -39670,15 +42457,15 @@ public class Hbase { } if (incoming.get(3)) { { - org.apache.thrift.protocol.TMap _map438 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32()); - struct.attributes = new HashMap(2*_map438.size); - for (int _i439 = 0; _i439 < _map438.size; ++_i439) + org.apache.thrift.protocol.TMap _map446 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32()); + struct.attributes = new HashMap(2*_map446.size); + for (int _i447 = 0; _i447 < _map446.size; ++_i447) { - ByteBuffer _key440; // required - ByteBuffer _val441; // optional - _key440 = iprot.readBinary(); - _val441 = iprot.readBinary(); - struct.attributes.put(_key440, _val441); + ByteBuffer _key448; + ByteBuffer _val449; + _key448 = iprot.readBinary(); + _val449 = iprot.readBinary(); + struct.attributes.put(_key448, _val449); } } struct.setAttributesIsSet(true); @@ -39688,7 +42475,7 @@ public class Hbase { } - public static class deleteAllRowTs_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + public static class deleteAllRowTs_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("deleteAllRowTs_result"); private static final org.apache.thrift.protocol.TField IO_FIELD_DESC = new org.apache.thrift.protocol.TField("io", org.apache.thrift.protocol.TType.STRUCT, (short)1); @@ -39886,20 +42673,20 @@ public class Hbase { return 0; } + @Override public int compareTo(deleteAllRowTs_result other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; - deleteAllRowTs_result typedOther = (deleteAllRowTs_result)other; - lastComparison = Boolean.valueOf(isSetIo()).compareTo(typedOther.isSetIo()); + lastComparison = Boolean.valueOf(isSetIo()).compareTo(other.isSetIo()); if (lastComparison != 0) { return lastComparison; } if (isSetIo()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.io, typedOther.io); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.io, other.io); if (lastComparison != 0) { return lastComparison; } @@ -39937,6 +42724,7 @@ public class Hbase { 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 { @@ -40043,7 +42831,7 @@ public class Hbase { } - public static class scannerOpenWithScan_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + public static class scannerOpenWithScan_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("scannerOpenWithScan_args"); private static final org.apache.thrift.protocol.TField TABLE_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("tableName", org.apache.thrift.protocol.TType.STRING, (short)1); @@ -40183,7 +42971,7 @@ public class Hbase { this.scan = new TScan(other.scan); } if (other.isSetAttributes()) { - Map __this__attributes = new HashMap(); + Map __this__attributes = new HashMap(other.attributes.size()); for (Map.Entry other_element : other.attributes.entrySet()) { ByteBuffer other_element_key = other_element.getKey(); @@ -40430,40 +43218,40 @@ public class Hbase { return 0; } + @Override public int compareTo(scannerOpenWithScan_args other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; - scannerOpenWithScan_args typedOther = (scannerOpenWithScan_args)other; - lastComparison = Boolean.valueOf(isSetTableName()).compareTo(typedOther.isSetTableName()); + lastComparison = Boolean.valueOf(isSetTableName()).compareTo(other.isSetTableName()); if (lastComparison != 0) { return lastComparison; } if (isSetTableName()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tableName, typedOther.tableName); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tableName, other.tableName); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetScan()).compareTo(typedOther.isSetScan()); + lastComparison = Boolean.valueOf(isSetScan()).compareTo(other.isSetScan()); if (lastComparison != 0) { return lastComparison; } if (isSetScan()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.scan, typedOther.scan); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.scan, other.scan); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetAttributes()).compareTo(typedOther.isSetAttributes()); + lastComparison = Boolean.valueOf(isSetAttributes()).compareTo(other.isSetAttributes()); if (lastComparison != 0) { return lastComparison; } if (isSetAttributes()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.attributes, typedOther.attributes); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.attributes, other.attributes); if (lastComparison != 0) { return lastComparison; } @@ -40517,6 +43305,10 @@ public class Hbase { public void validate() throws org.apache.thrift.TException { // check for required fields + // check for sub-struct validity + if (scan != null) { + scan.validate(); + } } private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { @@ -40573,15 +43365,15 @@ public class Hbase { case 3: // ATTRIBUTES if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { { - org.apache.thrift.protocol.TMap _map442 = iprot.readMapBegin(); - struct.attributes = new HashMap(2*_map442.size); - for (int _i443 = 0; _i443 < _map442.size; ++_i443) + org.apache.thrift.protocol.TMap _map450 = iprot.readMapBegin(); + struct.attributes = new HashMap(2*_map450.size); + for (int _i451 = 0; _i451 < _map450.size; ++_i451) { - ByteBuffer _key444; // required - ByteBuffer _val445; // optional - _key444 = iprot.readBinary(); - _val445 = iprot.readBinary(); - struct.attributes.put(_key444, _val445); + ByteBuffer _key452; + ByteBuffer _val453; + _key452 = iprot.readBinary(); + _val453 = iprot.readBinary(); + struct.attributes.put(_key452, _val453); } iprot.readMapEnd(); } @@ -40619,10 +43411,10 @@ public class Hbase { oprot.writeFieldBegin(ATTRIBUTES_FIELD_DESC); { oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.attributes.size())); - for (Map.Entry _iter446 : struct.attributes.entrySet()) + for (Map.Entry _iter454 : struct.attributes.entrySet()) { - oprot.writeBinary(_iter446.getKey()); - oprot.writeBinary(_iter446.getValue()); + oprot.writeBinary(_iter454.getKey()); + oprot.writeBinary(_iter454.getValue()); } oprot.writeMapEnd(); } @@ -40665,10 +43457,10 @@ public class Hbase { if (struct.isSetAttributes()) { { oprot.writeI32(struct.attributes.size()); - for (Map.Entry _iter447 : struct.attributes.entrySet()) + for (Map.Entry _iter455 : struct.attributes.entrySet()) { - oprot.writeBinary(_iter447.getKey()); - oprot.writeBinary(_iter447.getValue()); + oprot.writeBinary(_iter455.getKey()); + oprot.writeBinary(_iter455.getValue()); } } } @@ -40689,15 +43481,15 @@ public class Hbase { } if (incoming.get(2)) { { - org.apache.thrift.protocol.TMap _map448 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32()); - struct.attributes = new HashMap(2*_map448.size); - for (int _i449 = 0; _i449 < _map448.size; ++_i449) + org.apache.thrift.protocol.TMap _map456 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32()); + struct.attributes = new HashMap(2*_map456.size); + for (int _i457 = 0; _i457 < _map456.size; ++_i457) { - ByteBuffer _key450; // required - ByteBuffer _val451; // optional - _key450 = iprot.readBinary(); - _val451 = iprot.readBinary(); - struct.attributes.put(_key450, _val451); + ByteBuffer _key458; + ByteBuffer _val459; + _key458 = iprot.readBinary(); + _val459 = iprot.readBinary(); + struct.attributes.put(_key458, _val459); } } struct.setAttributesIsSet(true); @@ -40707,7 +43499,7 @@ public class Hbase { } - public static class scannerOpenWithScan_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + public static class scannerOpenWithScan_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("scannerOpenWithScan_result"); private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.I32, (short)0); @@ -40785,7 +43577,7 @@ public class Hbase { // isset id assignments private static final int __SUCCESS_ISSET_ID = 0; - private BitSet __isset_bit_vector = new BitSet(1); + 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); @@ -40814,8 +43606,7 @@ public class Hbase { * Performs a deep copy on other. */ public scannerOpenWithScan_result(scannerOpenWithScan_result other) { - __isset_bit_vector.clear(); - __isset_bit_vector.or(other.__isset_bit_vector); + __isset_bitfield = other.__isset_bitfield; this.success = other.success; if (other.isSetIo()) { this.io = new IOError(other.io); @@ -40844,16 +43635,16 @@ public class Hbase { } public void unsetSuccess() { - __isset_bit_vector.clear(__SUCCESS_ISSET_ID); + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __SUCCESS_ISSET_ID); } /** Returns true if field success is set (has been assigned a value) and false otherwise */ public boolean isSetSuccess() { - return __isset_bit_vector.get(__SUCCESS_ISSET_ID); + return EncodingUtils.testBit(__isset_bitfield, __SUCCESS_ISSET_ID); } public void setSuccessIsSet(boolean value) { - __isset_bit_vector.set(__SUCCESS_ISSET_ID, value); + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __SUCCESS_ISSET_ID, value); } public IOError getIo() { @@ -40967,30 +43758,30 @@ public class Hbase { return 0; } + @Override public int compareTo(scannerOpenWithScan_result other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; - scannerOpenWithScan_result typedOther = (scannerOpenWithScan_result)other; - lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(typedOther.isSetSuccess()); + lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(other.isSetSuccess()); if (lastComparison != 0) { return lastComparison; } if (isSetSuccess()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, typedOther.success); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetIo()).compareTo(typedOther.isSetIo()); + lastComparison = Boolean.valueOf(isSetIo()).compareTo(other.isSetIo()); if (lastComparison != 0) { return lastComparison; } if (isSetIo()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.io, typedOther.io); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.io, other.io); if (lastComparison != 0) { return lastComparison; } @@ -41032,6 +43823,7 @@ public class Hbase { 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 { @@ -41045,7 +43837,7 @@ public class Hbase { 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_bit_vector = new BitSet(1); + __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); @@ -41102,9 +43894,11 @@ public class Hbase { struct.validate(); oprot.writeStructBegin(STRUCT_DESC); - oprot.writeFieldBegin(SUCCESS_FIELD_DESC); - oprot.writeI32(struct.success); - oprot.writeFieldEnd(); + if (struct.isSetSuccess()) { + oprot.writeFieldBegin(SUCCESS_FIELD_DESC); + oprot.writeI32(struct.success); + oprot.writeFieldEnd(); + } if (struct.io != null) { oprot.writeFieldBegin(IO_FIELD_DESC); struct.io.write(oprot); @@ -41161,7 +43955,7 @@ public class Hbase { } - public static class scannerOpen_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + public static class scannerOpen_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("scannerOpen_args"); private static final org.apache.thrift.protocol.TField TABLE_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("tableName", org.apache.thrift.protocol.TType.STRING, (short)1); @@ -41323,14 +44117,14 @@ public class Hbase { this.startRow = other.startRow; } if (other.isSetColumns()) { - List __this__columns = new ArrayList(); + List __this__columns = new ArrayList(other.columns.size()); for (ByteBuffer other_element : other.columns) { __this__columns.add(other_element); } this.columns = __this__columns; } if (other.isSetAttributes()) { - Map __this__attributes = new HashMap(); + Map __this__attributes = new HashMap(other.attributes.size()); for (Map.Entry other_element : other.attributes.entrySet()) { ByteBuffer other_element_key = other_element.getKey(); @@ -41661,50 +44455,50 @@ public class Hbase { return 0; } + @Override public int compareTo(scannerOpen_args other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; - scannerOpen_args typedOther = (scannerOpen_args)other; - lastComparison = Boolean.valueOf(isSetTableName()).compareTo(typedOther.isSetTableName()); + lastComparison = Boolean.valueOf(isSetTableName()).compareTo(other.isSetTableName()); if (lastComparison != 0) { return lastComparison; } if (isSetTableName()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tableName, typedOther.tableName); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tableName, other.tableName); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetStartRow()).compareTo(typedOther.isSetStartRow()); + lastComparison = Boolean.valueOf(isSetStartRow()).compareTo(other.isSetStartRow()); if (lastComparison != 0) { return lastComparison; } if (isSetStartRow()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.startRow, typedOther.startRow); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.startRow, other.startRow); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetColumns()).compareTo(typedOther.isSetColumns()); + lastComparison = Boolean.valueOf(isSetColumns()).compareTo(other.isSetColumns()); if (lastComparison != 0) { return lastComparison; } if (isSetColumns()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.columns, typedOther.columns); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.columns, other.columns); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetAttributes()).compareTo(typedOther.isSetAttributes()); + lastComparison = Boolean.valueOf(isSetAttributes()).compareTo(other.isSetAttributes()); if (lastComparison != 0) { return lastComparison; } if (isSetAttributes()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.attributes, typedOther.attributes); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.attributes, other.attributes); if (lastComparison != 0) { return lastComparison; } @@ -41766,6 +44560,7 @@ public class Hbase { 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 { @@ -41821,13 +44616,13 @@ public class Hbase { case 3: // COLUMNS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list452 = iprot.readListBegin(); - struct.columns = new ArrayList(_list452.size); - for (int _i453 = 0; _i453 < _list452.size; ++_i453) + org.apache.thrift.protocol.TList _list460 = iprot.readListBegin(); + struct.columns = new ArrayList(_list460.size); + for (int _i461 = 0; _i461 < _list460.size; ++_i461) { - ByteBuffer _elem454; // optional - _elem454 = iprot.readBinary(); - struct.columns.add(_elem454); + ByteBuffer _elem462; + _elem462 = iprot.readBinary(); + struct.columns.add(_elem462); } iprot.readListEnd(); } @@ -41839,15 +44634,15 @@ public class Hbase { case 4: // ATTRIBUTES if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { { - org.apache.thrift.protocol.TMap _map455 = iprot.readMapBegin(); - struct.attributes = new HashMap(2*_map455.size); - for (int _i456 = 0; _i456 < _map455.size; ++_i456) + org.apache.thrift.protocol.TMap _map463 = iprot.readMapBegin(); + struct.attributes = new HashMap(2*_map463.size); + for (int _i464 = 0; _i464 < _map463.size; ++_i464) { - ByteBuffer _key457; // required - ByteBuffer _val458; // optional - _key457 = iprot.readBinary(); - _val458 = iprot.readBinary(); - struct.attributes.put(_key457, _val458); + ByteBuffer _key465; + ByteBuffer _val466; + _key465 = iprot.readBinary(); + _val466 = iprot.readBinary(); + struct.attributes.put(_key465, _val466); } iprot.readMapEnd(); } @@ -41885,9 +44680,9 @@ public class Hbase { oprot.writeFieldBegin(COLUMNS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.columns.size())); - for (ByteBuffer _iter459 : struct.columns) + for (ByteBuffer _iter467 : struct.columns) { - oprot.writeBinary(_iter459); + oprot.writeBinary(_iter467); } oprot.writeListEnd(); } @@ -41897,10 +44692,10 @@ public class Hbase { oprot.writeFieldBegin(ATTRIBUTES_FIELD_DESC); { oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.attributes.size())); - for (Map.Entry _iter460 : struct.attributes.entrySet()) + for (Map.Entry _iter468 : struct.attributes.entrySet()) { - oprot.writeBinary(_iter460.getKey()); - oprot.writeBinary(_iter460.getValue()); + oprot.writeBinary(_iter468.getKey()); + oprot.writeBinary(_iter468.getValue()); } oprot.writeMapEnd(); } @@ -41946,19 +44741,19 @@ public class Hbase { if (struct.isSetColumns()) { { oprot.writeI32(struct.columns.size()); - for (ByteBuffer _iter461 : struct.columns) + for (ByteBuffer _iter469 : struct.columns) { - oprot.writeBinary(_iter461); + oprot.writeBinary(_iter469); } } } if (struct.isSetAttributes()) { { oprot.writeI32(struct.attributes.size()); - for (Map.Entry _iter462 : struct.attributes.entrySet()) + for (Map.Entry _iter470 : struct.attributes.entrySet()) { - oprot.writeBinary(_iter462.getKey()); - oprot.writeBinary(_iter462.getValue()); + oprot.writeBinary(_iter470.getKey()); + oprot.writeBinary(_iter470.getValue()); } } } @@ -41978,28 +44773,28 @@ public class Hbase { } if (incoming.get(2)) { { - org.apache.thrift.protocol.TList _list463 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); - struct.columns = new ArrayList(_list463.size); - for (int _i464 = 0; _i464 < _list463.size; ++_i464) + org.apache.thrift.protocol.TList _list471 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); + struct.columns = new ArrayList(_list471.size); + for (int _i472 = 0; _i472 < _list471.size; ++_i472) { - ByteBuffer _elem465; // optional - _elem465 = iprot.readBinary(); - struct.columns.add(_elem465); + ByteBuffer _elem473; + _elem473 = iprot.readBinary(); + struct.columns.add(_elem473); } } struct.setColumnsIsSet(true); } if (incoming.get(3)) { { - org.apache.thrift.protocol.TMap _map466 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32()); - struct.attributes = new HashMap(2*_map466.size); - for (int _i467 = 0; _i467 < _map466.size; ++_i467) + org.apache.thrift.protocol.TMap _map474 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32()); + struct.attributes = new HashMap(2*_map474.size); + for (int _i475 = 0; _i475 < _map474.size; ++_i475) { - ByteBuffer _key468; // required - ByteBuffer _val469; // optional - _key468 = iprot.readBinary(); - _val469 = iprot.readBinary(); - struct.attributes.put(_key468, _val469); + ByteBuffer _key476; + ByteBuffer _val477; + _key476 = iprot.readBinary(); + _val477 = iprot.readBinary(); + struct.attributes.put(_key476, _val477); } } struct.setAttributesIsSet(true); @@ -42009,7 +44804,7 @@ public class Hbase { } - public static class scannerOpen_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + public static class scannerOpen_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("scannerOpen_result"); private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.I32, (short)0); @@ -42087,7 +44882,7 @@ public class Hbase { // isset id assignments private static final int __SUCCESS_ISSET_ID = 0; - private BitSet __isset_bit_vector = new BitSet(1); + 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); @@ -42116,8 +44911,7 @@ public class Hbase { * Performs a deep copy on other. */ public scannerOpen_result(scannerOpen_result other) { - __isset_bit_vector.clear(); - __isset_bit_vector.or(other.__isset_bit_vector); + __isset_bitfield = other.__isset_bitfield; this.success = other.success; if (other.isSetIo()) { this.io = new IOError(other.io); @@ -42146,16 +44940,16 @@ public class Hbase { } public void unsetSuccess() { - __isset_bit_vector.clear(__SUCCESS_ISSET_ID); + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __SUCCESS_ISSET_ID); } /** Returns true if field success is set (has been assigned a value) and false otherwise */ public boolean isSetSuccess() { - return __isset_bit_vector.get(__SUCCESS_ISSET_ID); + return EncodingUtils.testBit(__isset_bitfield, __SUCCESS_ISSET_ID); } public void setSuccessIsSet(boolean value) { - __isset_bit_vector.set(__SUCCESS_ISSET_ID, value); + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __SUCCESS_ISSET_ID, value); } public IOError getIo() { @@ -42269,30 +45063,30 @@ public class Hbase { return 0; } + @Override public int compareTo(scannerOpen_result other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; - scannerOpen_result typedOther = (scannerOpen_result)other; - lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(typedOther.isSetSuccess()); + lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(other.isSetSuccess()); if (lastComparison != 0) { return lastComparison; } if (isSetSuccess()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, typedOther.success); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetIo()).compareTo(typedOther.isSetIo()); + lastComparison = Boolean.valueOf(isSetIo()).compareTo(other.isSetIo()); if (lastComparison != 0) { return lastComparison; } if (isSetIo()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.io, typedOther.io); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.io, other.io); if (lastComparison != 0) { return lastComparison; } @@ -42334,6 +45128,7 @@ public class Hbase { 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 { @@ -42347,7 +45142,7 @@ public class Hbase { 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_bit_vector = new BitSet(1); + __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); @@ -42404,9 +45199,11 @@ public class Hbase { struct.validate(); oprot.writeStructBegin(STRUCT_DESC); - oprot.writeFieldBegin(SUCCESS_FIELD_DESC); - oprot.writeI32(struct.success); - oprot.writeFieldEnd(); + if (struct.isSetSuccess()) { + oprot.writeFieldBegin(SUCCESS_FIELD_DESC); + oprot.writeI32(struct.success); + oprot.writeFieldEnd(); + } if (struct.io != null) { oprot.writeFieldBegin(IO_FIELD_DESC); struct.io.write(oprot); @@ -42463,7 +45260,7 @@ public class Hbase { } - public static class scannerOpenWithStop_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + public static class scannerOpenWithStop_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("scannerOpenWithStop_args"); private static final org.apache.thrift.protocol.TField TABLE_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("tableName", org.apache.thrift.protocol.TType.STRING, (short)1); @@ -42645,14 +45442,14 @@ public class Hbase { this.stopRow = other.stopRow; } if (other.isSetColumns()) { - List __this__columns = new ArrayList(); + List __this__columns = new ArrayList(other.columns.size()); for (ByteBuffer other_element : other.columns) { __this__columns.add(other_element); } this.columns = __this__columns; } if (other.isSetAttributes()) { - Map __this__attributes = new HashMap(); + Map __this__attributes = new HashMap(other.attributes.size()); for (Map.Entry other_element : other.attributes.entrySet()) { ByteBuffer other_element_key = other_element.getKey(); @@ -43048,60 +45845,60 @@ public class Hbase { return 0; } + @Override public int compareTo(scannerOpenWithStop_args other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; - scannerOpenWithStop_args typedOther = (scannerOpenWithStop_args)other; - lastComparison = Boolean.valueOf(isSetTableName()).compareTo(typedOther.isSetTableName()); + lastComparison = Boolean.valueOf(isSetTableName()).compareTo(other.isSetTableName()); if (lastComparison != 0) { return lastComparison; } if (isSetTableName()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tableName, typedOther.tableName); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tableName, other.tableName); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetStartRow()).compareTo(typedOther.isSetStartRow()); + lastComparison = Boolean.valueOf(isSetStartRow()).compareTo(other.isSetStartRow()); if (lastComparison != 0) { return lastComparison; } if (isSetStartRow()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.startRow, typedOther.startRow); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.startRow, other.startRow); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetStopRow()).compareTo(typedOther.isSetStopRow()); + lastComparison = Boolean.valueOf(isSetStopRow()).compareTo(other.isSetStopRow()); if (lastComparison != 0) { return lastComparison; } if (isSetStopRow()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.stopRow, typedOther.stopRow); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.stopRow, other.stopRow); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetColumns()).compareTo(typedOther.isSetColumns()); + lastComparison = Boolean.valueOf(isSetColumns()).compareTo(other.isSetColumns()); if (lastComparison != 0) { return lastComparison; } if (isSetColumns()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.columns, typedOther.columns); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.columns, other.columns); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetAttributes()).compareTo(typedOther.isSetAttributes()); + lastComparison = Boolean.valueOf(isSetAttributes()).compareTo(other.isSetAttributes()); if (lastComparison != 0) { return lastComparison; } if (isSetAttributes()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.attributes, typedOther.attributes); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.attributes, other.attributes); if (lastComparison != 0) { return lastComparison; } @@ -43171,6 +45968,7 @@ public class Hbase { 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 { @@ -43234,13 +46032,13 @@ public class Hbase { case 4: // COLUMNS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list470 = iprot.readListBegin(); - struct.columns = new ArrayList(_list470.size); - for (int _i471 = 0; _i471 < _list470.size; ++_i471) + org.apache.thrift.protocol.TList _list478 = iprot.readListBegin(); + struct.columns = new ArrayList(_list478.size); + for (int _i479 = 0; _i479 < _list478.size; ++_i479) { - ByteBuffer _elem472; // optional - _elem472 = iprot.readBinary(); - struct.columns.add(_elem472); + ByteBuffer _elem480; + _elem480 = iprot.readBinary(); + struct.columns.add(_elem480); } iprot.readListEnd(); } @@ -43252,15 +46050,15 @@ public class Hbase { case 5: // ATTRIBUTES if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { { - org.apache.thrift.protocol.TMap _map473 = iprot.readMapBegin(); - struct.attributes = new HashMap(2*_map473.size); - for (int _i474 = 0; _i474 < _map473.size; ++_i474) + org.apache.thrift.protocol.TMap _map481 = iprot.readMapBegin(); + struct.attributes = new HashMap(2*_map481.size); + for (int _i482 = 0; _i482 < _map481.size; ++_i482) { - ByteBuffer _key475; // required - ByteBuffer _val476; // optional - _key475 = iprot.readBinary(); - _val476 = iprot.readBinary(); - struct.attributes.put(_key475, _val476); + ByteBuffer _key483; + ByteBuffer _val484; + _key483 = iprot.readBinary(); + _val484 = iprot.readBinary(); + struct.attributes.put(_key483, _val484); } iprot.readMapEnd(); } @@ -43303,9 +46101,9 @@ public class Hbase { oprot.writeFieldBegin(COLUMNS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.columns.size())); - for (ByteBuffer _iter477 : struct.columns) + for (ByteBuffer _iter485 : struct.columns) { - oprot.writeBinary(_iter477); + oprot.writeBinary(_iter485); } oprot.writeListEnd(); } @@ -43315,10 +46113,10 @@ public class Hbase { oprot.writeFieldBegin(ATTRIBUTES_FIELD_DESC); { oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.attributes.size())); - for (Map.Entry _iter478 : struct.attributes.entrySet()) + for (Map.Entry _iter486 : struct.attributes.entrySet()) { - oprot.writeBinary(_iter478.getKey()); - oprot.writeBinary(_iter478.getValue()); + oprot.writeBinary(_iter486.getKey()); + oprot.writeBinary(_iter486.getValue()); } oprot.writeMapEnd(); } @@ -43370,19 +46168,19 @@ public class Hbase { if (struct.isSetColumns()) { { oprot.writeI32(struct.columns.size()); - for (ByteBuffer _iter479 : struct.columns) + for (ByteBuffer _iter487 : struct.columns) { - oprot.writeBinary(_iter479); + oprot.writeBinary(_iter487); } } } if (struct.isSetAttributes()) { { oprot.writeI32(struct.attributes.size()); - for (Map.Entry _iter480 : struct.attributes.entrySet()) + for (Map.Entry _iter488 : struct.attributes.entrySet()) { - oprot.writeBinary(_iter480.getKey()); - oprot.writeBinary(_iter480.getValue()); + oprot.writeBinary(_iter488.getKey()); + oprot.writeBinary(_iter488.getValue()); } } } @@ -43406,28 +46204,28 @@ public class Hbase { } if (incoming.get(3)) { { - org.apache.thrift.protocol.TList _list481 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); - struct.columns = new ArrayList(_list481.size); - for (int _i482 = 0; _i482 < _list481.size; ++_i482) + org.apache.thrift.protocol.TList _list489 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); + struct.columns = new ArrayList(_list489.size); + for (int _i490 = 0; _i490 < _list489.size; ++_i490) { - ByteBuffer _elem483; // optional - _elem483 = iprot.readBinary(); - struct.columns.add(_elem483); + ByteBuffer _elem491; + _elem491 = iprot.readBinary(); + struct.columns.add(_elem491); } } struct.setColumnsIsSet(true); } if (incoming.get(4)) { { - org.apache.thrift.protocol.TMap _map484 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32()); - struct.attributes = new HashMap(2*_map484.size); - for (int _i485 = 0; _i485 < _map484.size; ++_i485) + org.apache.thrift.protocol.TMap _map492 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32()); + struct.attributes = new HashMap(2*_map492.size); + for (int _i493 = 0; _i493 < _map492.size; ++_i493) { - ByteBuffer _key486; // required - ByteBuffer _val487; // optional - _key486 = iprot.readBinary(); - _val487 = iprot.readBinary(); - struct.attributes.put(_key486, _val487); + ByteBuffer _key494; + ByteBuffer _val495; + _key494 = iprot.readBinary(); + _val495 = iprot.readBinary(); + struct.attributes.put(_key494, _val495); } } struct.setAttributesIsSet(true); @@ -43437,7 +46235,7 @@ public class Hbase { } - public static class scannerOpenWithStop_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + public static class scannerOpenWithStop_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("scannerOpenWithStop_result"); private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.I32, (short)0); @@ -43515,7 +46313,7 @@ public class Hbase { // isset id assignments private static final int __SUCCESS_ISSET_ID = 0; - private BitSet __isset_bit_vector = new BitSet(1); + 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); @@ -43544,8 +46342,7 @@ public class Hbase { * Performs a deep copy on other. */ public scannerOpenWithStop_result(scannerOpenWithStop_result other) { - __isset_bit_vector.clear(); - __isset_bit_vector.or(other.__isset_bit_vector); + __isset_bitfield = other.__isset_bitfield; this.success = other.success; if (other.isSetIo()) { this.io = new IOError(other.io); @@ -43574,16 +46371,16 @@ public class Hbase { } public void unsetSuccess() { - __isset_bit_vector.clear(__SUCCESS_ISSET_ID); + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __SUCCESS_ISSET_ID); } /** Returns true if field success is set (has been assigned a value) and false otherwise */ public boolean isSetSuccess() { - return __isset_bit_vector.get(__SUCCESS_ISSET_ID); + return EncodingUtils.testBit(__isset_bitfield, __SUCCESS_ISSET_ID); } public void setSuccessIsSet(boolean value) { - __isset_bit_vector.set(__SUCCESS_ISSET_ID, value); + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __SUCCESS_ISSET_ID, value); } public IOError getIo() { @@ -43697,30 +46494,30 @@ public class Hbase { return 0; } + @Override public int compareTo(scannerOpenWithStop_result other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; - scannerOpenWithStop_result typedOther = (scannerOpenWithStop_result)other; - lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(typedOther.isSetSuccess()); + lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(other.isSetSuccess()); if (lastComparison != 0) { return lastComparison; } if (isSetSuccess()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, typedOther.success); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetIo()).compareTo(typedOther.isSetIo()); + lastComparison = Boolean.valueOf(isSetIo()).compareTo(other.isSetIo()); if (lastComparison != 0) { return lastComparison; } if (isSetIo()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.io, typedOther.io); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.io, other.io); if (lastComparison != 0) { return lastComparison; } @@ -43762,6 +46559,7 @@ public class Hbase { 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 { @@ -43775,7 +46573,7 @@ public class Hbase { 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_bit_vector = new BitSet(1); + __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); @@ -43832,9 +46630,11 @@ public class Hbase { struct.validate(); oprot.writeStructBegin(STRUCT_DESC); - oprot.writeFieldBegin(SUCCESS_FIELD_DESC); - oprot.writeI32(struct.success); - oprot.writeFieldEnd(); + if (struct.isSetSuccess()) { + oprot.writeFieldBegin(SUCCESS_FIELD_DESC); + oprot.writeI32(struct.success); + oprot.writeFieldEnd(); + } if (struct.io != null) { oprot.writeFieldBegin(IO_FIELD_DESC); struct.io.write(oprot); @@ -43891,7 +46691,7 @@ public class Hbase { } - public static class scannerOpenWithPrefix_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + public static class scannerOpenWithPrefix_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("scannerOpenWithPrefix_args"); private static final org.apache.thrift.protocol.TField TABLE_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("tableName", org.apache.thrift.protocol.TType.STRING, (short)1); @@ -44047,14 +46847,14 @@ public class Hbase { this.startAndPrefix = other.startAndPrefix; } if (other.isSetColumns()) { - List __this__columns = new ArrayList(); + List __this__columns = new ArrayList(other.columns.size()); for (ByteBuffer other_element : other.columns) { __this__columns.add(other_element); } this.columns = __this__columns; } if (other.isSetAttributes()) { - Map __this__attributes = new HashMap(); + Map __this__attributes = new HashMap(other.attributes.size()); for (Map.Entry other_element : other.attributes.entrySet()) { ByteBuffer other_element_key = other_element.getKey(); @@ -44379,50 +47179,50 @@ public class Hbase { return 0; } + @Override public int compareTo(scannerOpenWithPrefix_args other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; - scannerOpenWithPrefix_args typedOther = (scannerOpenWithPrefix_args)other; - lastComparison = Boolean.valueOf(isSetTableName()).compareTo(typedOther.isSetTableName()); + lastComparison = Boolean.valueOf(isSetTableName()).compareTo(other.isSetTableName()); if (lastComparison != 0) { return lastComparison; } if (isSetTableName()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tableName, typedOther.tableName); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tableName, other.tableName); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetStartAndPrefix()).compareTo(typedOther.isSetStartAndPrefix()); + lastComparison = Boolean.valueOf(isSetStartAndPrefix()).compareTo(other.isSetStartAndPrefix()); if (lastComparison != 0) { return lastComparison; } if (isSetStartAndPrefix()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.startAndPrefix, typedOther.startAndPrefix); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.startAndPrefix, other.startAndPrefix); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetColumns()).compareTo(typedOther.isSetColumns()); + lastComparison = Boolean.valueOf(isSetColumns()).compareTo(other.isSetColumns()); if (lastComparison != 0) { return lastComparison; } if (isSetColumns()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.columns, typedOther.columns); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.columns, other.columns); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetAttributes()).compareTo(typedOther.isSetAttributes()); + lastComparison = Boolean.valueOf(isSetAttributes()).compareTo(other.isSetAttributes()); if (lastComparison != 0) { return lastComparison; } if (isSetAttributes()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.attributes, typedOther.attributes); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.attributes, other.attributes); if (lastComparison != 0) { return lastComparison; } @@ -44484,6 +47284,7 @@ public class Hbase { 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 { @@ -44539,13 +47340,13 @@ public class Hbase { case 3: // COLUMNS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list488 = iprot.readListBegin(); - struct.columns = new ArrayList(_list488.size); - for (int _i489 = 0; _i489 < _list488.size; ++_i489) + org.apache.thrift.protocol.TList _list496 = iprot.readListBegin(); + struct.columns = new ArrayList(_list496.size); + for (int _i497 = 0; _i497 < _list496.size; ++_i497) { - ByteBuffer _elem490; // optional - _elem490 = iprot.readBinary(); - struct.columns.add(_elem490); + ByteBuffer _elem498; + _elem498 = iprot.readBinary(); + struct.columns.add(_elem498); } iprot.readListEnd(); } @@ -44557,15 +47358,15 @@ public class Hbase { case 4: // ATTRIBUTES if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { { - org.apache.thrift.protocol.TMap _map491 = iprot.readMapBegin(); - struct.attributes = new HashMap(2*_map491.size); - for (int _i492 = 0; _i492 < _map491.size; ++_i492) + org.apache.thrift.protocol.TMap _map499 = iprot.readMapBegin(); + struct.attributes = new HashMap(2*_map499.size); + for (int _i500 = 0; _i500 < _map499.size; ++_i500) { - ByteBuffer _key493; // required - ByteBuffer _val494; // optional - _key493 = iprot.readBinary(); - _val494 = iprot.readBinary(); - struct.attributes.put(_key493, _val494); + ByteBuffer _key501; + ByteBuffer _val502; + _key501 = iprot.readBinary(); + _val502 = iprot.readBinary(); + struct.attributes.put(_key501, _val502); } iprot.readMapEnd(); } @@ -44603,9 +47404,9 @@ public class Hbase { oprot.writeFieldBegin(COLUMNS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.columns.size())); - for (ByteBuffer _iter495 : struct.columns) + for (ByteBuffer _iter503 : struct.columns) { - oprot.writeBinary(_iter495); + oprot.writeBinary(_iter503); } oprot.writeListEnd(); } @@ -44615,10 +47416,10 @@ public class Hbase { oprot.writeFieldBegin(ATTRIBUTES_FIELD_DESC); { oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.attributes.size())); - for (Map.Entry _iter496 : struct.attributes.entrySet()) + for (Map.Entry _iter504 : struct.attributes.entrySet()) { - oprot.writeBinary(_iter496.getKey()); - oprot.writeBinary(_iter496.getValue()); + oprot.writeBinary(_iter504.getKey()); + oprot.writeBinary(_iter504.getValue()); } oprot.writeMapEnd(); } @@ -44664,19 +47465,19 @@ public class Hbase { if (struct.isSetColumns()) { { oprot.writeI32(struct.columns.size()); - for (ByteBuffer _iter497 : struct.columns) + for (ByteBuffer _iter505 : struct.columns) { - oprot.writeBinary(_iter497); + oprot.writeBinary(_iter505); } } } if (struct.isSetAttributes()) { { oprot.writeI32(struct.attributes.size()); - for (Map.Entry _iter498 : struct.attributes.entrySet()) + for (Map.Entry _iter506 : struct.attributes.entrySet()) { - oprot.writeBinary(_iter498.getKey()); - oprot.writeBinary(_iter498.getValue()); + oprot.writeBinary(_iter506.getKey()); + oprot.writeBinary(_iter506.getValue()); } } } @@ -44696,28 +47497,28 @@ public class Hbase { } if (incoming.get(2)) { { - org.apache.thrift.protocol.TList _list499 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); - struct.columns = new ArrayList(_list499.size); - for (int _i500 = 0; _i500 < _list499.size; ++_i500) + org.apache.thrift.protocol.TList _list507 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); + struct.columns = new ArrayList(_list507.size); + for (int _i508 = 0; _i508 < _list507.size; ++_i508) { - ByteBuffer _elem501; // optional - _elem501 = iprot.readBinary(); - struct.columns.add(_elem501); + ByteBuffer _elem509; + _elem509 = iprot.readBinary(); + struct.columns.add(_elem509); } } struct.setColumnsIsSet(true); } if (incoming.get(3)) { { - org.apache.thrift.protocol.TMap _map502 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32()); - struct.attributes = new HashMap(2*_map502.size); - for (int _i503 = 0; _i503 < _map502.size; ++_i503) + org.apache.thrift.protocol.TMap _map510 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32()); + struct.attributes = new HashMap(2*_map510.size); + for (int _i511 = 0; _i511 < _map510.size; ++_i511) { - ByteBuffer _key504; // required - ByteBuffer _val505; // optional - _key504 = iprot.readBinary(); - _val505 = iprot.readBinary(); - struct.attributes.put(_key504, _val505); + ByteBuffer _key512; + ByteBuffer _val513; + _key512 = iprot.readBinary(); + _val513 = iprot.readBinary(); + struct.attributes.put(_key512, _val513); } } struct.setAttributesIsSet(true); @@ -44727,7 +47528,7 @@ public class Hbase { } - public static class scannerOpenWithPrefix_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + public static class scannerOpenWithPrefix_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("scannerOpenWithPrefix_result"); private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.I32, (short)0); @@ -44805,7 +47606,7 @@ public class Hbase { // isset id assignments private static final int __SUCCESS_ISSET_ID = 0; - private BitSet __isset_bit_vector = new BitSet(1); + 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); @@ -44834,8 +47635,7 @@ public class Hbase { * Performs a deep copy on other. */ public scannerOpenWithPrefix_result(scannerOpenWithPrefix_result other) { - __isset_bit_vector.clear(); - __isset_bit_vector.or(other.__isset_bit_vector); + __isset_bitfield = other.__isset_bitfield; this.success = other.success; if (other.isSetIo()) { this.io = new IOError(other.io); @@ -44864,16 +47664,16 @@ public class Hbase { } public void unsetSuccess() { - __isset_bit_vector.clear(__SUCCESS_ISSET_ID); + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __SUCCESS_ISSET_ID); } /** Returns true if field success is set (has been assigned a value) and false otherwise */ public boolean isSetSuccess() { - return __isset_bit_vector.get(__SUCCESS_ISSET_ID); + return EncodingUtils.testBit(__isset_bitfield, __SUCCESS_ISSET_ID); } public void setSuccessIsSet(boolean value) { - __isset_bit_vector.set(__SUCCESS_ISSET_ID, value); + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __SUCCESS_ISSET_ID, value); } public IOError getIo() { @@ -44987,30 +47787,30 @@ public class Hbase { return 0; } + @Override public int compareTo(scannerOpenWithPrefix_result other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; - scannerOpenWithPrefix_result typedOther = (scannerOpenWithPrefix_result)other; - lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(typedOther.isSetSuccess()); + lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(other.isSetSuccess()); if (lastComparison != 0) { return lastComparison; } if (isSetSuccess()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, typedOther.success); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetIo()).compareTo(typedOther.isSetIo()); + lastComparison = Boolean.valueOf(isSetIo()).compareTo(other.isSetIo()); if (lastComparison != 0) { return lastComparison; } if (isSetIo()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.io, typedOther.io); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.io, other.io); if (lastComparison != 0) { return lastComparison; } @@ -45052,6 +47852,7 @@ public class Hbase { 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 { @@ -45065,7 +47866,7 @@ public class Hbase { 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_bit_vector = new BitSet(1); + __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); @@ -45122,9 +47923,11 @@ public class Hbase { struct.validate(); oprot.writeStructBegin(STRUCT_DESC); - oprot.writeFieldBegin(SUCCESS_FIELD_DESC); - oprot.writeI32(struct.success); - oprot.writeFieldEnd(); + if (struct.isSetSuccess()) { + oprot.writeFieldBegin(SUCCESS_FIELD_DESC); + oprot.writeI32(struct.success); + oprot.writeFieldEnd(); + } if (struct.io != null) { oprot.writeFieldBegin(IO_FIELD_DESC); struct.io.write(oprot); @@ -45181,7 +47984,7 @@ public class Hbase { } - public static class scannerOpenTs_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + public static class scannerOpenTs_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("scannerOpenTs_args"); private static final org.apache.thrift.protocol.TField TABLE_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("tableName", org.apache.thrift.protocol.TType.STRING, (short)1); @@ -45310,7 +48113,7 @@ public class Hbase { // isset id assignments private static final int __TIMESTAMP_ISSET_ID = 0; - private BitSet __isset_bit_vector = new BitSet(1); + 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); @@ -45354,8 +48157,7 @@ public class Hbase { * Performs a deep copy on other. */ public scannerOpenTs_args(scannerOpenTs_args other) { - __isset_bit_vector.clear(); - __isset_bit_vector.or(other.__isset_bit_vector); + __isset_bitfield = other.__isset_bitfield; if (other.isSetTableName()) { this.tableName = other.tableName; } @@ -45363,7 +48165,7 @@ public class Hbase { this.startRow = other.startRow; } if (other.isSetColumns()) { - List __this__columns = new ArrayList(); + List __this__columns = new ArrayList(other.columns.size()); for (ByteBuffer other_element : other.columns) { __this__columns.add(other_element); } @@ -45371,7 +48173,7 @@ public class Hbase { } this.timestamp = other.timestamp; if (other.isSetAttributes()) { - Map __this__attributes = new HashMap(); + Map __this__attributes = new HashMap(other.attributes.size()); for (Map.Entry other_element : other.attributes.entrySet()) { ByteBuffer other_element_key = other_element.getKey(); @@ -45549,16 +48351,16 @@ public class Hbase { } public void unsetTimestamp() { - __isset_bit_vector.clear(__TIMESTAMP_ISSET_ID); + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __TIMESTAMP_ISSET_ID); } /** Returns true if field timestamp is set (has been assigned a value) and false otherwise */ public boolean isSetTimestamp() { - return __isset_bit_vector.get(__TIMESTAMP_ISSET_ID); + return EncodingUtils.testBit(__isset_bitfield, __TIMESTAMP_ISSET_ID); } public void setTimestampIsSet(boolean value) { - __isset_bit_vector.set(__TIMESTAMP_ISSET_ID, value); + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __TIMESTAMP_ISSET_ID, value); } public int getAttributesSize() { @@ -45755,60 +48557,60 @@ public class Hbase { return 0; } + @Override public int compareTo(scannerOpenTs_args other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; - scannerOpenTs_args typedOther = (scannerOpenTs_args)other; - lastComparison = Boolean.valueOf(isSetTableName()).compareTo(typedOther.isSetTableName()); + lastComparison = Boolean.valueOf(isSetTableName()).compareTo(other.isSetTableName()); if (lastComparison != 0) { return lastComparison; } if (isSetTableName()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tableName, typedOther.tableName); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tableName, other.tableName); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetStartRow()).compareTo(typedOther.isSetStartRow()); + lastComparison = Boolean.valueOf(isSetStartRow()).compareTo(other.isSetStartRow()); if (lastComparison != 0) { return lastComparison; } if (isSetStartRow()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.startRow, typedOther.startRow); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.startRow, other.startRow); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetColumns()).compareTo(typedOther.isSetColumns()); + lastComparison = Boolean.valueOf(isSetColumns()).compareTo(other.isSetColumns()); if (lastComparison != 0) { return lastComparison; } if (isSetColumns()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.columns, typedOther.columns); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.columns, other.columns); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetTimestamp()).compareTo(typedOther.isSetTimestamp()); + lastComparison = Boolean.valueOf(isSetTimestamp()).compareTo(other.isSetTimestamp()); if (lastComparison != 0) { return lastComparison; } if (isSetTimestamp()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.timestamp, typedOther.timestamp); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.timestamp, other.timestamp); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetAttributes()).compareTo(typedOther.isSetAttributes()); + lastComparison = Boolean.valueOf(isSetAttributes()).compareTo(other.isSetAttributes()); if (lastComparison != 0) { return lastComparison; } if (isSetAttributes()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.attributes, typedOther.attributes); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.attributes, other.attributes); if (lastComparison != 0) { return lastComparison; } @@ -45874,6 +48676,7 @@ public class Hbase { 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 { @@ -45887,7 +48690,7 @@ public class Hbase { 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_bit_vector = new BitSet(1); + __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); @@ -45931,13 +48734,13 @@ public class Hbase { case 3: // COLUMNS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list506 = iprot.readListBegin(); - struct.columns = new ArrayList(_list506.size); - for (int _i507 = 0; _i507 < _list506.size; ++_i507) + org.apache.thrift.protocol.TList _list514 = iprot.readListBegin(); + struct.columns = new ArrayList(_list514.size); + for (int _i515 = 0; _i515 < _list514.size; ++_i515) { - ByteBuffer _elem508; // optional - _elem508 = iprot.readBinary(); - struct.columns.add(_elem508); + ByteBuffer _elem516; + _elem516 = iprot.readBinary(); + struct.columns.add(_elem516); } iprot.readListEnd(); } @@ -45957,15 +48760,15 @@ public class Hbase { case 5: // ATTRIBUTES if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { { - org.apache.thrift.protocol.TMap _map509 = iprot.readMapBegin(); - struct.attributes = new HashMap(2*_map509.size); - for (int _i510 = 0; _i510 < _map509.size; ++_i510) + org.apache.thrift.protocol.TMap _map517 = iprot.readMapBegin(); + struct.attributes = new HashMap(2*_map517.size); + for (int _i518 = 0; _i518 < _map517.size; ++_i518) { - ByteBuffer _key511; // required - ByteBuffer _val512; // optional - _key511 = iprot.readBinary(); - _val512 = iprot.readBinary(); - struct.attributes.put(_key511, _val512); + ByteBuffer _key519; + ByteBuffer _val520; + _key519 = iprot.readBinary(); + _val520 = iprot.readBinary(); + struct.attributes.put(_key519, _val520); } iprot.readMapEnd(); } @@ -46003,9 +48806,9 @@ public class Hbase { oprot.writeFieldBegin(COLUMNS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.columns.size())); - for (ByteBuffer _iter513 : struct.columns) + for (ByteBuffer _iter521 : struct.columns) { - oprot.writeBinary(_iter513); + oprot.writeBinary(_iter521); } oprot.writeListEnd(); } @@ -46018,10 +48821,10 @@ public class Hbase { oprot.writeFieldBegin(ATTRIBUTES_FIELD_DESC); { oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.attributes.size())); - for (Map.Entry _iter514 : struct.attributes.entrySet()) + for (Map.Entry _iter522 : struct.attributes.entrySet()) { - oprot.writeBinary(_iter514.getKey()); - oprot.writeBinary(_iter514.getValue()); + oprot.writeBinary(_iter522.getKey()); + oprot.writeBinary(_iter522.getValue()); } oprot.writeMapEnd(); } @@ -46070,9 +48873,9 @@ public class Hbase { if (struct.isSetColumns()) { { oprot.writeI32(struct.columns.size()); - for (ByteBuffer _iter515 : struct.columns) + for (ByteBuffer _iter523 : struct.columns) { - oprot.writeBinary(_iter515); + oprot.writeBinary(_iter523); } } } @@ -46082,10 +48885,10 @@ public class Hbase { if (struct.isSetAttributes()) { { oprot.writeI32(struct.attributes.size()); - for (Map.Entry _iter516 : struct.attributes.entrySet()) + for (Map.Entry _iter524 : struct.attributes.entrySet()) { - oprot.writeBinary(_iter516.getKey()); - oprot.writeBinary(_iter516.getValue()); + oprot.writeBinary(_iter524.getKey()); + oprot.writeBinary(_iter524.getValue()); } } } @@ -46105,13 +48908,13 @@ public class Hbase { } if (incoming.get(2)) { { - org.apache.thrift.protocol.TList _list517 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); - struct.columns = new ArrayList(_list517.size); - for (int _i518 = 0; _i518 < _list517.size; ++_i518) + org.apache.thrift.protocol.TList _list525 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); + struct.columns = new ArrayList(_list525.size); + for (int _i526 = 0; _i526 < _list525.size; ++_i526) { - ByteBuffer _elem519; // optional - _elem519 = iprot.readBinary(); - struct.columns.add(_elem519); + ByteBuffer _elem527; + _elem527 = iprot.readBinary(); + struct.columns.add(_elem527); } } struct.setColumnsIsSet(true); @@ -46122,15 +48925,15 @@ public class Hbase { } if (incoming.get(4)) { { - org.apache.thrift.protocol.TMap _map520 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32()); - struct.attributes = new HashMap(2*_map520.size); - for (int _i521 = 0; _i521 < _map520.size; ++_i521) + org.apache.thrift.protocol.TMap _map528 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32()); + struct.attributes = new HashMap(2*_map528.size); + for (int _i529 = 0; _i529 < _map528.size; ++_i529) { - ByteBuffer _key522; // required - ByteBuffer _val523; // optional - _key522 = iprot.readBinary(); - _val523 = iprot.readBinary(); - struct.attributes.put(_key522, _val523); + ByteBuffer _key530; + ByteBuffer _val531; + _key530 = iprot.readBinary(); + _val531 = iprot.readBinary(); + struct.attributes.put(_key530, _val531); } } struct.setAttributesIsSet(true); @@ -46140,7 +48943,7 @@ public class Hbase { } - public static class scannerOpenTs_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + public static class scannerOpenTs_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("scannerOpenTs_result"); private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.I32, (short)0); @@ -46218,7 +49021,7 @@ public class Hbase { // isset id assignments private static final int __SUCCESS_ISSET_ID = 0; - private BitSet __isset_bit_vector = new BitSet(1); + 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); @@ -46247,8 +49050,7 @@ public class Hbase { * Performs a deep copy on other. */ public scannerOpenTs_result(scannerOpenTs_result other) { - __isset_bit_vector.clear(); - __isset_bit_vector.or(other.__isset_bit_vector); + __isset_bitfield = other.__isset_bitfield; this.success = other.success; if (other.isSetIo()) { this.io = new IOError(other.io); @@ -46277,16 +49079,16 @@ public class Hbase { } public void unsetSuccess() { - __isset_bit_vector.clear(__SUCCESS_ISSET_ID); + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __SUCCESS_ISSET_ID); } /** Returns true if field success is set (has been assigned a value) and false otherwise */ public boolean isSetSuccess() { - return __isset_bit_vector.get(__SUCCESS_ISSET_ID); + return EncodingUtils.testBit(__isset_bitfield, __SUCCESS_ISSET_ID); } public void setSuccessIsSet(boolean value) { - __isset_bit_vector.set(__SUCCESS_ISSET_ID, value); + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __SUCCESS_ISSET_ID, value); } public IOError getIo() { @@ -46400,30 +49202,30 @@ public class Hbase { return 0; } + @Override public int compareTo(scannerOpenTs_result other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; - scannerOpenTs_result typedOther = (scannerOpenTs_result)other; - lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(typedOther.isSetSuccess()); + lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(other.isSetSuccess()); if (lastComparison != 0) { return lastComparison; } if (isSetSuccess()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, typedOther.success); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetIo()).compareTo(typedOther.isSetIo()); + lastComparison = Boolean.valueOf(isSetIo()).compareTo(other.isSetIo()); if (lastComparison != 0) { return lastComparison; } if (isSetIo()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.io, typedOther.io); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.io, other.io); if (lastComparison != 0) { return lastComparison; } @@ -46465,6 +49267,7 @@ public class Hbase { 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 { @@ -46478,7 +49281,7 @@ public class Hbase { 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_bit_vector = new BitSet(1); + __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); @@ -46535,9 +49338,11 @@ public class Hbase { struct.validate(); oprot.writeStructBegin(STRUCT_DESC); - oprot.writeFieldBegin(SUCCESS_FIELD_DESC); - oprot.writeI32(struct.success); - oprot.writeFieldEnd(); + if (struct.isSetSuccess()) { + oprot.writeFieldBegin(SUCCESS_FIELD_DESC); + oprot.writeI32(struct.success); + oprot.writeFieldEnd(); + } if (struct.io != null) { oprot.writeFieldBegin(IO_FIELD_DESC); struct.io.write(oprot); @@ -46594,7 +49399,7 @@ public class Hbase { } - public static class scannerOpenWithStopTs_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + public static class scannerOpenWithStopTs_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("scannerOpenWithStopTs_args"); private static final org.apache.thrift.protocol.TField TABLE_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("tableName", org.apache.thrift.protocol.TType.STRING, (short)1); @@ -46736,7 +49541,7 @@ public class Hbase { // isset id assignments private static final int __TIMESTAMP_ISSET_ID = 0; - private BitSet __isset_bit_vector = new BitSet(1); + 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); @@ -46784,8 +49589,7 @@ public class Hbase { * Performs a deep copy on other. */ public scannerOpenWithStopTs_args(scannerOpenWithStopTs_args other) { - __isset_bit_vector.clear(); - __isset_bit_vector.or(other.__isset_bit_vector); + __isset_bitfield = other.__isset_bitfield; if (other.isSetTableName()) { this.tableName = other.tableName; } @@ -46796,7 +49600,7 @@ public class Hbase { this.stopRow = other.stopRow; } if (other.isSetColumns()) { - List __this__columns = new ArrayList(); + List __this__columns = new ArrayList(other.columns.size()); for (ByteBuffer other_element : other.columns) { __this__columns.add(other_element); } @@ -46804,7 +49608,7 @@ public class Hbase { } this.timestamp = other.timestamp; if (other.isSetAttributes()) { - Map __this__attributes = new HashMap(); + Map __this__attributes = new HashMap(other.attributes.size()); for (Map.Entry other_element : other.attributes.entrySet()) { ByteBuffer other_element_key = other_element.getKey(); @@ -47025,16 +49829,16 @@ public class Hbase { } public void unsetTimestamp() { - __isset_bit_vector.clear(__TIMESTAMP_ISSET_ID); + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __TIMESTAMP_ISSET_ID); } /** Returns true if field timestamp is set (has been assigned a value) and false otherwise */ public boolean isSetTimestamp() { - return __isset_bit_vector.get(__TIMESTAMP_ISSET_ID); + return EncodingUtils.testBit(__isset_bitfield, __TIMESTAMP_ISSET_ID); } public void setTimestampIsSet(boolean value) { - __isset_bit_vector.set(__TIMESTAMP_ISSET_ID, value); + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __TIMESTAMP_ISSET_ID, value); } public int getAttributesSize() { @@ -47253,70 +50057,70 @@ public class Hbase { return 0; } + @Override public int compareTo(scannerOpenWithStopTs_args other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; - scannerOpenWithStopTs_args typedOther = (scannerOpenWithStopTs_args)other; - lastComparison = Boolean.valueOf(isSetTableName()).compareTo(typedOther.isSetTableName()); + lastComparison = Boolean.valueOf(isSetTableName()).compareTo(other.isSetTableName()); if (lastComparison != 0) { return lastComparison; } if (isSetTableName()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tableName, typedOther.tableName); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tableName, other.tableName); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetStartRow()).compareTo(typedOther.isSetStartRow()); + lastComparison = Boolean.valueOf(isSetStartRow()).compareTo(other.isSetStartRow()); if (lastComparison != 0) { return lastComparison; } if (isSetStartRow()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.startRow, typedOther.startRow); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.startRow, other.startRow); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetStopRow()).compareTo(typedOther.isSetStopRow()); + lastComparison = Boolean.valueOf(isSetStopRow()).compareTo(other.isSetStopRow()); if (lastComparison != 0) { return lastComparison; } if (isSetStopRow()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.stopRow, typedOther.stopRow); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.stopRow, other.stopRow); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetColumns()).compareTo(typedOther.isSetColumns()); + lastComparison = Boolean.valueOf(isSetColumns()).compareTo(other.isSetColumns()); if (lastComparison != 0) { return lastComparison; } if (isSetColumns()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.columns, typedOther.columns); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.columns, other.columns); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetTimestamp()).compareTo(typedOther.isSetTimestamp()); + lastComparison = Boolean.valueOf(isSetTimestamp()).compareTo(other.isSetTimestamp()); if (lastComparison != 0) { return lastComparison; } if (isSetTimestamp()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.timestamp, typedOther.timestamp); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.timestamp, other.timestamp); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetAttributes()).compareTo(typedOther.isSetAttributes()); + lastComparison = Boolean.valueOf(isSetAttributes()).compareTo(other.isSetAttributes()); if (lastComparison != 0) { return lastComparison; } if (isSetAttributes()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.attributes, typedOther.attributes); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.attributes, other.attributes); if (lastComparison != 0) { return lastComparison; } @@ -47390,6 +50194,7 @@ public class Hbase { 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 { @@ -47402,6 +50207,8 @@ public class Hbase { 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); @@ -47453,13 +50260,13 @@ public class Hbase { case 4: // COLUMNS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list524 = iprot.readListBegin(); - struct.columns = new ArrayList(_list524.size); - for (int _i525 = 0; _i525 < _list524.size; ++_i525) + org.apache.thrift.protocol.TList _list532 = iprot.readListBegin(); + struct.columns = new ArrayList(_list532.size); + for (int _i533 = 0; _i533 < _list532.size; ++_i533) { - ByteBuffer _elem526; // optional - _elem526 = iprot.readBinary(); - struct.columns.add(_elem526); + ByteBuffer _elem534; + _elem534 = iprot.readBinary(); + struct.columns.add(_elem534); } iprot.readListEnd(); } @@ -47479,15 +50286,15 @@ public class Hbase { case 6: // ATTRIBUTES if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { { - org.apache.thrift.protocol.TMap _map527 = iprot.readMapBegin(); - struct.attributes = new HashMap(2*_map527.size); - for (int _i528 = 0; _i528 < _map527.size; ++_i528) + org.apache.thrift.protocol.TMap _map535 = iprot.readMapBegin(); + struct.attributes = new HashMap(2*_map535.size); + for (int _i536 = 0; _i536 < _map535.size; ++_i536) { - ByteBuffer _key529; // required - ByteBuffer _val530; // optional - _key529 = iprot.readBinary(); - _val530 = iprot.readBinary(); - struct.attributes.put(_key529, _val530); + ByteBuffer _key537; + ByteBuffer _val538; + _key537 = iprot.readBinary(); + _val538 = iprot.readBinary(); + struct.attributes.put(_key537, _val538); } iprot.readMapEnd(); } @@ -47530,9 +50337,9 @@ public class Hbase { oprot.writeFieldBegin(COLUMNS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.columns.size())); - for (ByteBuffer _iter531 : struct.columns) + for (ByteBuffer _iter539 : struct.columns) { - oprot.writeBinary(_iter531); + oprot.writeBinary(_iter539); } oprot.writeListEnd(); } @@ -47545,10 +50352,10 @@ public class Hbase { oprot.writeFieldBegin(ATTRIBUTES_FIELD_DESC); { oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.attributes.size())); - for (Map.Entry _iter532 : struct.attributes.entrySet()) + for (Map.Entry _iter540 : struct.attributes.entrySet()) { - oprot.writeBinary(_iter532.getKey()); - oprot.writeBinary(_iter532.getValue()); + oprot.writeBinary(_iter540.getKey()); + oprot.writeBinary(_iter540.getValue()); } oprot.writeMapEnd(); } @@ -47603,9 +50410,9 @@ public class Hbase { if (struct.isSetColumns()) { { oprot.writeI32(struct.columns.size()); - for (ByteBuffer _iter533 : struct.columns) + for (ByteBuffer _iter541 : struct.columns) { - oprot.writeBinary(_iter533); + oprot.writeBinary(_iter541); } } } @@ -47615,10 +50422,10 @@ public class Hbase { if (struct.isSetAttributes()) { { oprot.writeI32(struct.attributes.size()); - for (Map.Entry _iter534 : struct.attributes.entrySet()) + for (Map.Entry _iter542 : struct.attributes.entrySet()) { - oprot.writeBinary(_iter534.getKey()); - oprot.writeBinary(_iter534.getValue()); + oprot.writeBinary(_iter542.getKey()); + oprot.writeBinary(_iter542.getValue()); } } } @@ -47642,13 +50449,13 @@ public class Hbase { } if (incoming.get(3)) { { - org.apache.thrift.protocol.TList _list535 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); - struct.columns = new ArrayList(_list535.size); - for (int _i536 = 0; _i536 < _list535.size; ++_i536) + org.apache.thrift.protocol.TList _list543 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); + struct.columns = new ArrayList(_list543.size); + for (int _i544 = 0; _i544 < _list543.size; ++_i544) { - ByteBuffer _elem537; // optional - _elem537 = iprot.readBinary(); - struct.columns.add(_elem537); + ByteBuffer _elem545; + _elem545 = iprot.readBinary(); + struct.columns.add(_elem545); } } struct.setColumnsIsSet(true); @@ -47659,15 +50466,15 @@ public class Hbase { } if (incoming.get(5)) { { - org.apache.thrift.protocol.TMap _map538 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32()); - struct.attributes = new HashMap(2*_map538.size); - for (int _i539 = 0; _i539 < _map538.size; ++_i539) + 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.attributes = new HashMap(2*_map546.size); + for (int _i547 = 0; _i547 < _map546.size; ++_i547) { - ByteBuffer _key540; // required - ByteBuffer _val541; // optional - _key540 = iprot.readBinary(); - _val541 = iprot.readBinary(); - struct.attributes.put(_key540, _val541); + ByteBuffer _key548; + ByteBuffer _val549; + _key548 = iprot.readBinary(); + _val549 = iprot.readBinary(); + struct.attributes.put(_key548, _val549); } } struct.setAttributesIsSet(true); @@ -47677,7 +50484,7 @@ public class Hbase { } - public static class scannerOpenWithStopTs_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + public static class scannerOpenWithStopTs_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("scannerOpenWithStopTs_result"); private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.I32, (short)0); @@ -47755,7 +50562,7 @@ public class Hbase { // isset id assignments private static final int __SUCCESS_ISSET_ID = 0; - private BitSet __isset_bit_vector = new BitSet(1); + 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); @@ -47784,8 +50591,7 @@ public class Hbase { * Performs a deep copy on other. */ public scannerOpenWithStopTs_result(scannerOpenWithStopTs_result other) { - __isset_bit_vector.clear(); - __isset_bit_vector.or(other.__isset_bit_vector); + __isset_bitfield = other.__isset_bitfield; this.success = other.success; if (other.isSetIo()) { this.io = new IOError(other.io); @@ -47814,16 +50620,16 @@ public class Hbase { } public void unsetSuccess() { - __isset_bit_vector.clear(__SUCCESS_ISSET_ID); + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __SUCCESS_ISSET_ID); } /** Returns true if field success is set (has been assigned a value) and false otherwise */ public boolean isSetSuccess() { - return __isset_bit_vector.get(__SUCCESS_ISSET_ID); + return EncodingUtils.testBit(__isset_bitfield, __SUCCESS_ISSET_ID); } public void setSuccessIsSet(boolean value) { - __isset_bit_vector.set(__SUCCESS_ISSET_ID, value); + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __SUCCESS_ISSET_ID, value); } public IOError getIo() { @@ -47937,30 +50743,30 @@ public class Hbase { return 0; } + @Override public int compareTo(scannerOpenWithStopTs_result other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; - scannerOpenWithStopTs_result typedOther = (scannerOpenWithStopTs_result)other; - lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(typedOther.isSetSuccess()); + lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(other.isSetSuccess()); if (lastComparison != 0) { return lastComparison; } if (isSetSuccess()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, typedOther.success); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetIo()).compareTo(typedOther.isSetIo()); + lastComparison = Boolean.valueOf(isSetIo()).compareTo(other.isSetIo()); if (lastComparison != 0) { return lastComparison; } if (isSetIo()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.io, typedOther.io); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.io, other.io); if (lastComparison != 0) { return lastComparison; } @@ -48002,6 +50808,7 @@ public class Hbase { 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 { @@ -48015,7 +50822,7 @@ public class Hbase { 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_bit_vector = new BitSet(1); + __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); @@ -48072,9 +50879,11 @@ public class Hbase { struct.validate(); oprot.writeStructBegin(STRUCT_DESC); - oprot.writeFieldBegin(SUCCESS_FIELD_DESC); - oprot.writeI32(struct.success); - oprot.writeFieldEnd(); + if (struct.isSetSuccess()) { + oprot.writeFieldBegin(SUCCESS_FIELD_DESC); + oprot.writeI32(struct.success); + oprot.writeFieldEnd(); + } if (struct.io != null) { oprot.writeFieldBegin(IO_FIELD_DESC); struct.io.write(oprot); @@ -48131,7 +50940,7 @@ public class Hbase { } - public static class scannerGet_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + public static class scannerGet_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("scannerGet_args"); private static final org.apache.thrift.protocol.TField ID_FIELD_DESC = new org.apache.thrift.protocol.TField("id", org.apache.thrift.protocol.TType.I32, (short)1); @@ -48210,7 +51019,7 @@ public class Hbase { // isset id assignments private static final int __ID_ISSET_ID = 0; - private BitSet __isset_bit_vector = new BitSet(1); + 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); @@ -48235,8 +51044,7 @@ public class Hbase { * Performs a deep copy on other. */ public scannerGet_args(scannerGet_args other) { - __isset_bit_vector.clear(); - __isset_bit_vector.or(other.__isset_bit_vector); + __isset_bitfield = other.__isset_bitfield; this.id = other.id; } @@ -48267,16 +51075,16 @@ public class Hbase { } public void unsetId() { - __isset_bit_vector.clear(__ID_ISSET_ID); + __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 __isset_bit_vector.get(__ID_ISSET_ID); + return EncodingUtils.testBit(__isset_bitfield, __ID_ISSET_ID); } public void setIdIsSet(boolean value) { - __isset_bit_vector.set(__ID_ISSET_ID, value); + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __ID_ISSET_ID, value); } public void setFieldValue(_Fields field, Object value) { @@ -48344,20 +51152,20 @@ public class Hbase { return 0; } + @Override public int compareTo(scannerGet_args other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; - scannerGet_args typedOther = (scannerGet_args)other; - lastComparison = Boolean.valueOf(isSetId()).compareTo(typedOther.isSetId()); + lastComparison = Boolean.valueOf(isSetId()).compareTo(other.isSetId()); if (lastComparison != 0) { return lastComparison; } if (isSetId()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.id, typedOther.id); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.id, other.id); if (lastComparison != 0) { return lastComparison; } @@ -48391,6 +51199,7 @@ public class Hbase { 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 { @@ -48404,7 +51213,7 @@ public class Hbase { 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_bit_vector = new BitSet(1); + __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); @@ -48495,7 +51304,7 @@ public class Hbase { } - public static class scannerGet_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + public static class scannerGet_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("scannerGet_result"); private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.LIST, (short)0); @@ -48610,7 +51419,7 @@ public class Hbase { */ public scannerGet_result(scannerGet_result other) { if (other.isSetSuccess()) { - List __this__success = new ArrayList(); + List __this__success = new ArrayList(other.success.size()); for (TRowResult other_element : other.success) { __this__success.add(new TRowResult(other_element)); } @@ -48831,40 +51640,40 @@ public class Hbase { return 0; } + @Override public int compareTo(scannerGet_result other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; - scannerGet_result typedOther = (scannerGet_result)other; - lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(typedOther.isSetSuccess()); + lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(other.isSetSuccess()); if (lastComparison != 0) { return lastComparison; } if (isSetSuccess()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, typedOther.success); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetIo()).compareTo(typedOther.isSetIo()); + lastComparison = Boolean.valueOf(isSetIo()).compareTo(other.isSetIo()); if (lastComparison != 0) { return lastComparison; } if (isSetIo()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.io, typedOther.io); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.io, other.io); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetIa()).compareTo(typedOther.isSetIa()); + lastComparison = Boolean.valueOf(isSetIa()).compareTo(other.isSetIa()); if (lastComparison != 0) { return lastComparison; } if (isSetIa()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.ia, typedOther.ia); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.ia, other.ia); if (lastComparison != 0) { return lastComparison; } @@ -48918,6 +51727,7 @@ public class Hbase { 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 { @@ -48957,14 +51767,14 @@ public class Hbase { 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 _list550 = iprot.readListBegin(); + struct.success = new ArrayList(_list550.size); + for (int _i551 = 0; _i551 < _list550.size; ++_i551) { - TRowResult _elem544; // optional - _elem544 = new TRowResult(); - _elem544.read(iprot); - struct.success.add(_elem544); + TRowResult _elem552; + _elem552 = new TRowResult(); + _elem552.read(iprot); + struct.success.add(_elem552); } iprot.readListEnd(); } @@ -49010,9 +51820,9 @@ public class Hbase { oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size())); - for (TRowResult _iter545 : struct.success) + for (TRowResult _iter553 : struct.success) { - _iter545.write(oprot); + _iter553.write(oprot); } oprot.writeListEnd(); } @@ -49059,9 +51869,9 @@ public class Hbase { if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (TRowResult _iter546 : struct.success) + for (TRowResult _iter554 : struct.success) { - _iter546.write(oprot); + _iter554.write(oprot); } } } @@ -49079,14 +51889,14 @@ public class Hbase { BitSet incoming = iprot.readBitSet(3); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list547 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); - struct.success = new ArrayList(_list547.size); - for (int _i548 = 0; _i548 < _list547.size; ++_i548) + org.apache.thrift.protocol.TList _list555 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); + struct.success = new ArrayList(_list555.size); + for (int _i556 = 0; _i556 < _list555.size; ++_i556) { - TRowResult _elem549; // optional - _elem549 = new TRowResult(); - _elem549.read(iprot); - struct.success.add(_elem549); + TRowResult _elem557; + _elem557 = new TRowResult(); + _elem557.read(iprot); + struct.success.add(_elem557); } } struct.setSuccessIsSet(true); @@ -49106,7 +51916,7 @@ public class Hbase { } - public static class scannerGetList_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + public static class scannerGetList_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("scannerGetList_args"); private static final org.apache.thrift.protocol.TField ID_FIELD_DESC = new org.apache.thrift.protocol.TField("id", org.apache.thrift.protocol.TType.I32, (short)1); @@ -49197,7 +52007,7 @@ public class Hbase { // isset id assignments private static final int __ID_ISSET_ID = 0; private static final int __NBROWS_ISSET_ID = 1; - private BitSet __isset_bit_vector = new BitSet(2); + 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); @@ -49227,8 +52037,7 @@ public class Hbase { * Performs a deep copy on other. */ public scannerGetList_args(scannerGetList_args other) { - __isset_bit_vector.clear(); - __isset_bit_vector.or(other.__isset_bit_vector); + __isset_bitfield = other.__isset_bitfield; this.id = other.id; this.nbRows = other.nbRows; } @@ -49262,16 +52071,16 @@ public class Hbase { } public void unsetId() { - __isset_bit_vector.clear(__ID_ISSET_ID); + __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 __isset_bit_vector.get(__ID_ISSET_ID); + return EncodingUtils.testBit(__isset_bitfield, __ID_ISSET_ID); } public void setIdIsSet(boolean value) { - __isset_bit_vector.set(__ID_ISSET_ID, value); + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __ID_ISSET_ID, value); } /** @@ -49291,16 +52100,16 @@ public class Hbase { } public void unsetNbRows() { - __isset_bit_vector.clear(__NBROWS_ISSET_ID); + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __NBROWS_ISSET_ID); } /** Returns true if field nbRows is set (has been assigned a value) and false otherwise */ public boolean isSetNbRows() { - return __isset_bit_vector.get(__NBROWS_ISSET_ID); + return EncodingUtils.testBit(__isset_bitfield, __NBROWS_ISSET_ID); } public void setNbRowsIsSet(boolean value) { - __isset_bit_vector.set(__NBROWS_ISSET_ID, value); + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __NBROWS_ISSET_ID, value); } public void setFieldValue(_Fields field, Object value) { @@ -49390,30 +52199,30 @@ public class Hbase { return 0; } + @Override public int compareTo(scannerGetList_args other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; - scannerGetList_args typedOther = (scannerGetList_args)other; - lastComparison = Boolean.valueOf(isSetId()).compareTo(typedOther.isSetId()); + lastComparison = Boolean.valueOf(isSetId()).compareTo(other.isSetId()); if (lastComparison != 0) { return lastComparison; } if (isSetId()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.id, typedOther.id); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.id, other.id); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetNbRows()).compareTo(typedOther.isSetNbRows()); + lastComparison = Boolean.valueOf(isSetNbRows()).compareTo(other.isSetNbRows()); if (lastComparison != 0) { return lastComparison; } if (isSetNbRows()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.nbRows, typedOther.nbRows); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.nbRows, other.nbRows); if (lastComparison != 0) { return lastComparison; } @@ -49451,6 +52260,7 @@ public class Hbase { 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 { @@ -49463,6 +52273,8 @@ public class Hbase { 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); @@ -49574,7 +52386,7 @@ public class Hbase { } - public static class scannerGetList_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + public static class scannerGetList_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("scannerGetList_result"); private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.LIST, (short)0); @@ -49689,7 +52501,7 @@ public class Hbase { */ public scannerGetList_result(scannerGetList_result other) { if (other.isSetSuccess()) { - List __this__success = new ArrayList(); + List __this__success = new ArrayList(other.success.size()); for (TRowResult other_element : other.success) { __this__success.add(new TRowResult(other_element)); } @@ -49910,40 +52722,40 @@ public class Hbase { return 0; } + @Override public int compareTo(scannerGetList_result other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; - scannerGetList_result typedOther = (scannerGetList_result)other; - lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(typedOther.isSetSuccess()); + lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(other.isSetSuccess()); if (lastComparison != 0) { return lastComparison; } if (isSetSuccess()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, typedOther.success); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetIo()).compareTo(typedOther.isSetIo()); + lastComparison = Boolean.valueOf(isSetIo()).compareTo(other.isSetIo()); if (lastComparison != 0) { return lastComparison; } if (isSetIo()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.io, typedOther.io); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.io, other.io); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetIa()).compareTo(typedOther.isSetIa()); + lastComparison = Boolean.valueOf(isSetIa()).compareTo(other.isSetIa()); if (lastComparison != 0) { return lastComparison; } if (isSetIa()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.ia, typedOther.ia); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.ia, other.ia); if (lastComparison != 0) { return lastComparison; } @@ -49997,6 +52809,7 @@ public class Hbase { 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 { @@ -50036,14 +52849,14 @@ public class Hbase { case 0: // SUCCESS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list550 = iprot.readListBegin(); - struct.success = new ArrayList(_list550.size); - for (int _i551 = 0; _i551 < _list550.size; ++_i551) + org.apache.thrift.protocol.TList _list558 = iprot.readListBegin(); + struct.success = new ArrayList(_list558.size); + for (int _i559 = 0; _i559 < _list558.size; ++_i559) { - TRowResult _elem552; // optional - _elem552 = new TRowResult(); - _elem552.read(iprot); - struct.success.add(_elem552); + TRowResult _elem560; + _elem560 = new TRowResult(); + _elem560.read(iprot); + struct.success.add(_elem560); } iprot.readListEnd(); } @@ -50089,9 +52902,9 @@ public class Hbase { oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size())); - for (TRowResult _iter553 : struct.success) + for (TRowResult _iter561 : struct.success) { - _iter553.write(oprot); + _iter561.write(oprot); } oprot.writeListEnd(); } @@ -50138,9 +52951,9 @@ public class Hbase { if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (TRowResult _iter554 : struct.success) + for (TRowResult _iter562 : struct.success) { - _iter554.write(oprot); + _iter562.write(oprot); } } } @@ -50158,14 +52971,14 @@ public class Hbase { BitSet incoming = iprot.readBitSet(3); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list555 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); - struct.success = new ArrayList(_list555.size); - for (int _i556 = 0; _i556 < _list555.size; ++_i556) + 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) { - TRowResult _elem557; // optional - _elem557 = new TRowResult(); - _elem557.read(iprot); - struct.success.add(_elem557); + TRowResult _elem565; + _elem565 = new TRowResult(); + _elem565.read(iprot); + struct.success.add(_elem565); } } struct.setSuccessIsSet(true); @@ -50185,7 +52998,7 @@ public class Hbase { } - public static class scannerClose_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + public static class scannerClose_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("scannerClose_args"); private static final org.apache.thrift.protocol.TField ID_FIELD_DESC = new org.apache.thrift.protocol.TField("id", org.apache.thrift.protocol.TType.I32, (short)1); @@ -50264,7 +53077,7 @@ public class Hbase { // isset id assignments private static final int __ID_ISSET_ID = 0; - private BitSet __isset_bit_vector = new BitSet(1); + 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); @@ -50289,8 +53102,7 @@ public class Hbase { * Performs a deep copy on other. */ public scannerClose_args(scannerClose_args other) { - __isset_bit_vector.clear(); - __isset_bit_vector.or(other.__isset_bit_vector); + __isset_bitfield = other.__isset_bitfield; this.id = other.id; } @@ -50321,16 +53133,16 @@ public class Hbase { } public void unsetId() { - __isset_bit_vector.clear(__ID_ISSET_ID); + __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 __isset_bit_vector.get(__ID_ISSET_ID); + return EncodingUtils.testBit(__isset_bitfield, __ID_ISSET_ID); } public void setIdIsSet(boolean value) { - __isset_bit_vector.set(__ID_ISSET_ID, value); + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __ID_ISSET_ID, value); } public void setFieldValue(_Fields field, Object value) { @@ -50398,20 +53210,20 @@ public class Hbase { return 0; } + @Override public int compareTo(scannerClose_args other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; - scannerClose_args typedOther = (scannerClose_args)other; - lastComparison = Boolean.valueOf(isSetId()).compareTo(typedOther.isSetId()); + lastComparison = Boolean.valueOf(isSetId()).compareTo(other.isSetId()); if (lastComparison != 0) { return lastComparison; } if (isSetId()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.id, typedOther.id); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.id, other.id); if (lastComparison != 0) { return lastComparison; } @@ -50445,6 +53257,7 @@ public class Hbase { 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 { @@ -50458,7 +53271,7 @@ public class Hbase { 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_bit_vector = new BitSet(1); + __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); @@ -50549,7 +53362,7 @@ public class Hbase { } - public static class scannerClose_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + public static class scannerClose_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("scannerClose_result"); private static final org.apache.thrift.protocol.TField IO_FIELD_DESC = new org.apache.thrift.protocol.TField("io", org.apache.thrift.protocol.TType.STRUCT, (short)1); @@ -50806,30 +53619,30 @@ public class Hbase { return 0; } + @Override public int compareTo(scannerClose_result other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; - scannerClose_result typedOther = (scannerClose_result)other; - lastComparison = Boolean.valueOf(isSetIo()).compareTo(typedOther.isSetIo()); + lastComparison = Boolean.valueOf(isSetIo()).compareTo(other.isSetIo()); if (lastComparison != 0) { return lastComparison; } if (isSetIo()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.io, typedOther.io); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.io, other.io); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetIa()).compareTo(typedOther.isSetIa()); + lastComparison = Boolean.valueOf(isSetIa()).compareTo(other.isSetIa()); if (lastComparison != 0) { return lastComparison; } if (isSetIa()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.ia, typedOther.ia); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.ia, other.ia); if (lastComparison != 0) { return lastComparison; } @@ -50875,6 +53688,7 @@ public class Hbase { 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 { @@ -51006,7 +53820,7 @@ public class Hbase { } - public static class getRowOrBefore_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + public static class getRowOrBefore_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getRowOrBefore_args"); private static final org.apache.thrift.protocol.TField TABLE_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("tableName", org.apache.thrift.protocol.TType.STRING, (short)1); @@ -51388,40 +54202,40 @@ public class Hbase { return 0; } + @Override public int compareTo(getRowOrBefore_args other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; - getRowOrBefore_args typedOther = (getRowOrBefore_args)other; - lastComparison = Boolean.valueOf(isSetTableName()).compareTo(typedOther.isSetTableName()); + lastComparison = Boolean.valueOf(isSetTableName()).compareTo(other.isSetTableName()); if (lastComparison != 0) { return lastComparison; } if (isSetTableName()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tableName, typedOther.tableName); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tableName, other.tableName); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetRow()).compareTo(typedOther.isSetRow()); + lastComparison = Boolean.valueOf(isSetRow()).compareTo(other.isSetRow()); if (lastComparison != 0) { return lastComparison; } if (isSetRow()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.row, typedOther.row); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.row, other.row); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetFamily()).compareTo(typedOther.isSetFamily()); + lastComparison = Boolean.valueOf(isSetFamily()).compareTo(other.isSetFamily()); if (lastComparison != 0) { return lastComparison; } if (isSetFamily()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.family, typedOther.family); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.family, other.family); if (lastComparison != 0) { return lastComparison; } @@ -51475,6 +54289,7 @@ public class Hbase { 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 { @@ -51625,7 +54440,7 @@ public class Hbase { } - public static class getRowOrBefore_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + public static class getRowOrBefore_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getRowOrBefore_result"); private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.LIST, (short)0); @@ -51731,7 +54546,7 @@ public class Hbase { */ public getRowOrBefore_result(getRowOrBefore_result other) { if (other.isSetSuccess()) { - List __this__success = new ArrayList(); + List __this__success = new ArrayList(other.success.size()); for (TCell other_element : other.success) { __this__success.add(new TCell(other_element)); } @@ -51902,30 +54717,30 @@ public class Hbase { return 0; } + @Override public int compareTo(getRowOrBefore_result other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; - getRowOrBefore_result typedOther = (getRowOrBefore_result)other; - lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(typedOther.isSetSuccess()); + lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(other.isSetSuccess()); if (lastComparison != 0) { return lastComparison; } if (isSetSuccess()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, typedOther.success); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetIo()).compareTo(typedOther.isSetIo()); + lastComparison = Boolean.valueOf(isSetIo()).compareTo(other.isSetIo()); if (lastComparison != 0) { return lastComparison; } if (isSetIo()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.io, typedOther.io); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.io, other.io); if (lastComparison != 0) { return lastComparison; } @@ -51971,6 +54786,7 @@ public class Hbase { 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 { @@ -52010,14 +54826,14 @@ public class Hbase { 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 _list566 = iprot.readListBegin(); + struct.success = new ArrayList(_list566.size); + for (int _i567 = 0; _i567 < _list566.size; ++_i567) { - TCell _elem560; // optional - _elem560 = new TCell(); - _elem560.read(iprot); - struct.success.add(_elem560); + TCell _elem568; + _elem568 = new TCell(); + _elem568.read(iprot); + struct.success.add(_elem568); } iprot.readListEnd(); } @@ -52054,9 +54870,9 @@ public class Hbase { oprot.writeFieldBegin(SUCCESS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size())); - for (TCell _iter561 : struct.success) + for (TCell _iter569 : struct.success) { - _iter561.write(oprot); + _iter569.write(oprot); } oprot.writeListEnd(); } @@ -52095,9 +54911,9 @@ public class Hbase { if (struct.isSetSuccess()) { { oprot.writeI32(struct.success.size()); - for (TCell _iter562 : struct.success) + for (TCell _iter570 : struct.success) { - _iter562.write(oprot); + _iter570.write(oprot); } } } @@ -52112,14 +54928,14 @@ public class Hbase { BitSet incoming = iprot.readBitSet(2); 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 _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) { - TCell _elem565; // optional - _elem565 = new TCell(); - _elem565.read(iprot); - struct.success.add(_elem565); + TCell _elem573; + _elem573 = new TCell(); + _elem573.read(iprot); + struct.success.add(_elem573); } } struct.setSuccessIsSet(true); @@ -52134,7 +54950,7 @@ public class Hbase { } - public static class getRegionInfo_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + public static class getRegionInfo_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getRegionInfo_args"); private static final org.apache.thrift.protocol.TField ROW_FIELD_DESC = new org.apache.thrift.protocol.TField("row", org.apache.thrift.protocol.TType.STRING, (short)1); @@ -52354,20 +55170,20 @@ public class Hbase { return 0; } + @Override public int compareTo(getRegionInfo_args other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; - getRegionInfo_args typedOther = (getRegionInfo_args)other; - lastComparison = Boolean.valueOf(isSetRow()).compareTo(typedOther.isSetRow()); + lastComparison = Boolean.valueOf(isSetRow()).compareTo(other.isSetRow()); if (lastComparison != 0) { return lastComparison; } if (isSetRow()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.row, typedOther.row); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.row, other.row); if (lastComparison != 0) { return lastComparison; } @@ -52405,6 +55221,7 @@ public class Hbase { 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 { @@ -52509,7 +55326,7 @@ public class Hbase { } - public static class getRegionInfo_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + public static class getRegionInfo_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getRegionInfo_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); @@ -52766,30 +55583,30 @@ public class Hbase { return 0; } + @Override public int compareTo(getRegionInfo_result other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; - getRegionInfo_result typedOther = (getRegionInfo_result)other; - lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(typedOther.isSetSuccess()); + lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(other.isSetSuccess()); if (lastComparison != 0) { return lastComparison; } if (isSetSuccess()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, typedOther.success); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetIo()).compareTo(typedOther.isSetIo()); + lastComparison = Boolean.valueOf(isSetIo()).compareTo(other.isSetIo()); if (lastComparison != 0) { return lastComparison; } if (isSetIo()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.io, typedOther.io); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.io, other.io); if (lastComparison != 0) { return lastComparison; } @@ -52835,6 +55652,10 @@ public class Hbase { 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 { diff --git a/src/main/java/org/apache/hadoop/hbase/thrift/generated/IOError.java b/src/main/java/org/apache/hadoop/hbase/thrift/generated/IOError.java index 11e31e3..1157c53 100644 --- a/src/main/java/org/apache/hadoop/hbase/thrift/generated/IOError.java +++ b/src/main/java/org/apache/hadoop/hbase/thrift/generated/IOError.java @@ -1,5 +1,5 @@ /** - * Autogenerated by Thrift Compiler (0.8.0) + * Autogenerated by Thrift Compiler (0.9.1) * * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING * @generated @@ -12,6 +12,11 @@ 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 org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; import java.util.List; import java.util.ArrayList; import java.util.Map; @@ -32,7 +37,7 @@ import org.slf4j.LoggerFactory; * to the Hbase master or an Hbase region server. Also used to return * more general Hbase error conditions. */ -public class IOError extends Exception implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { +public class IOError extends TException implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("IOError"); 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); @@ -230,20 +235,20 @@ public class IOError extends Exception implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { +public class IllegalArgument extends TException implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("IllegalArgument"); 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); @@ -229,20 +234,20 @@ public class IllegalArgument extends Exception implements org.apache.thrift.TBas return 0; } + @Override public int compareTo(IllegalArgument other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; - IllegalArgument typedOther = (IllegalArgument)other; - lastComparison = Boolean.valueOf(isSetMessage()).compareTo(typedOther.isSetMessage()); + lastComparison = Boolean.valueOf(isSetMessage()).compareTo(other.isSetMessage()); if (lastComparison != 0) { return lastComparison; } if (isSetMessage()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.message, typedOther.message); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.message, other.message); if (lastComparison != 0) { return lastComparison; } @@ -280,6 +285,7 @@ public class IllegalArgument extends Exception implements org.apache.thrift.TBas 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 { diff --git a/src/main/java/org/apache/hadoop/hbase/thrift/generated/Mutation.java b/src/main/java/org/apache/hadoop/hbase/thrift/generated/Mutation.java index ef1817f..c983cdd 100644 --- a/src/main/java/org/apache/hadoop/hbase/thrift/generated/Mutation.java +++ b/src/main/java/org/apache/hadoop/hbase/thrift/generated/Mutation.java @@ -1,5 +1,5 @@ /** - * Autogenerated by Thrift Compiler (0.8.0) + * Autogenerated by Thrift Compiler (0.9.1) * * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING * @generated @@ -12,6 +12,11 @@ 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 org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; import java.util.List; import java.util.ArrayList; import java.util.Map; @@ -30,7 +35,7 @@ import org.slf4j.LoggerFactory; /** * A Mutation object is used to either update or delete a column-value. */ -public class Mutation implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { +public class Mutation implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("Mutation"); private static final org.apache.thrift.protocol.TField IS_DELETE_FIELD_DESC = new org.apache.thrift.protocol.TField("isDelete", org.apache.thrift.protocol.TType.BOOL, (short)1); @@ -119,7 +124,7 @@ public class Mutation implements org.apache.thrift.TBase metaDataMap; static { Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); @@ -161,8 +166,7 @@ public class Mutation implements org.apache.thrift.TBaseother. */ public Mutation(Mutation other) { - __isset_bit_vector.clear(); - __isset_bit_vector.or(other.__isset_bit_vector); + __isset_bitfield = other.__isset_bitfield; this.isDelete = other.isDelete; if (other.isSetColumn()) { this.column = other.column; @@ -198,16 +202,16 @@ public class Mutation implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { +public class TCell implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TCell"); private static final org.apache.thrift.protocol.TField VALUE_FIELD_DESC = new org.apache.thrift.protocol.TField("value", org.apache.thrift.protocol.TType.STRING, (short)1); @@ -111,7 +116,7 @@ public class TCell implements org.apache.thrift.TBase, jav // isset id assignments private static final int __TIMESTAMP_ISSET_ID = 0; - private BitSet __isset_bit_vector = new BitSet(1); + 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); @@ -140,8 +145,7 @@ public class TCell implements org.apache.thrift.TBase, jav * Performs a deep copy on other. */ public TCell(TCell other) { - __isset_bit_vector.clear(); - __isset_bit_vector.or(other.__isset_bit_vector); + __isset_bitfield = other.__isset_bitfield; if (other.isSetValue()) { this.value = other.value; } @@ -204,16 +208,16 @@ public class TCell implements org.apache.thrift.TBase, jav } public void unsetTimestamp() { - __isset_bit_vector.clear(__TIMESTAMP_ISSET_ID); + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __TIMESTAMP_ISSET_ID); } /** Returns true if field timestamp is set (has been assigned a value) and false otherwise */ public boolean isSetTimestamp() { - return __isset_bit_vector.get(__TIMESTAMP_ISSET_ID); + return EncodingUtils.testBit(__isset_bitfield, __TIMESTAMP_ISSET_ID); } public void setTimestampIsSet(boolean value) { - __isset_bit_vector.set(__TIMESTAMP_ISSET_ID, value); + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __TIMESTAMP_ISSET_ID, value); } public void setFieldValue(_Fields field, Object value) { @@ -303,30 +307,30 @@ public class TCell implements org.apache.thrift.TBase, jav return 0; } + @Override public int compareTo(TCell other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; - TCell typedOther = (TCell)other; - lastComparison = Boolean.valueOf(isSetValue()).compareTo(typedOther.isSetValue()); + lastComparison = Boolean.valueOf(isSetValue()).compareTo(other.isSetValue()); if (lastComparison != 0) { return lastComparison; } if (isSetValue()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.value, typedOther.value); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.value, other.value); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetTimestamp()).compareTo(typedOther.isSetTimestamp()); + lastComparison = Boolean.valueOf(isSetTimestamp()).compareTo(other.isSetTimestamp()); if (lastComparison != 0) { return lastComparison; } if (isSetTimestamp()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.timestamp, typedOther.timestamp); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.timestamp, other.timestamp); if (lastComparison != 0) { return lastComparison; } @@ -368,6 +372,7 @@ public class TCell implements org.apache.thrift.TBase, jav 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 { @@ -381,7 +386,7 @@ public class TCell implements org.apache.thrift.TBase, jav 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_bit_vector = new BitSet(1); + __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); diff --git a/src/main/java/org/apache/hadoop/hbase/thrift/generated/TColumn.java b/src/main/java/org/apache/hadoop/hbase/thrift/generated/TColumn.java index cb978e8..c3b89e9 100644 --- a/src/main/java/org/apache/hadoop/hbase/thrift/generated/TColumn.java +++ b/src/main/java/org/apache/hadoop/hbase/thrift/generated/TColumn.java @@ -1,5 +1,5 @@ /** - * Autogenerated by Thrift Compiler (0.8.0) + * Autogenerated by Thrift Compiler (0.9.1) * * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING * @generated @@ -12,6 +12,11 @@ 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 org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; import java.util.List; import java.util.ArrayList; import java.util.Map; @@ -30,7 +35,7 @@ import org.slf4j.LoggerFactory; /** * Holds column name and the cell. */ -public class TColumn implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { +public class TColumn implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TColumn"); private static final org.apache.thrift.protocol.TField COLUMN_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("columnName", org.apache.thrift.protocol.TType.STRING, (short)1); @@ -297,30 +302,30 @@ public class TColumn implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { +public class TIncrement implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TIncrement"); private static final org.apache.thrift.protocol.TField TABLE_FIELD_DESC = new org.apache.thrift.protocol.TField("table", org.apache.thrift.protocol.TType.STRING, (short)1); @@ -119,7 +124,7 @@ public class TIncrement implements org.apache.thrift.TBase metaDataMap; static { Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); @@ -156,8 +161,7 @@ public class TIncrement implements org.apache.thrift.TBaseother. */ public TIncrement(TIncrement other) { - __isset_bit_vector.clear(); - __isset_bit_vector.or(other.__isset_bit_vector); + __isset_bitfield = other.__isset_bitfield; if (other.isSetTable()) { this.table = other.table; } @@ -296,16 +300,16 @@ public class TIncrement implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { +public class TRegionInfo implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TRegionInfo"); private static final org.apache.thrift.protocol.TField START_KEY_FIELD_DESC = new org.apache.thrift.protocol.TField("startKey", org.apache.thrift.protocol.TType.STRING, (short)1); @@ -135,7 +140,7 @@ public class TRegionInfo implements org.apache.thrift.TBase metaDataMap; static { Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); @@ -186,8 +191,7 @@ public class TRegionInfo implements org.apache.thrift.TBaseother. */ public TRegionInfo(TRegionInfo other) { - __isset_bit_vector.clear(); - __isset_bit_vector.or(other.__isset_bit_vector); + __isset_bitfield = other.__isset_bitfield; if (other.isSetStartKey()) { this.startKey = other.startKey; } @@ -302,16 +306,16 @@ public class TRegionInfo implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { +public class TRowResult implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TRowResult"); private static final org.apache.thrift.protocol.TField ROW_FIELD_DESC = new org.apache.thrift.protocol.TField("row", org.apache.thrift.protocol.TType.STRING, (short)1); @@ -147,7 +152,7 @@ public class TRowResult implements org.apache.thrift.TBase __this__columns = new HashMap(); + Map __this__columns = new HashMap(other.columns.size()); for (Map.Entry other_element : other.columns.entrySet()) { ByteBuffer other_element_key = other_element.getKey(); @@ -162,7 +167,7 @@ public class TRowResult implements org.apache.thrift.TBase __this__sortedColumns = new ArrayList(); + List __this__sortedColumns = new ArrayList(other.sortedColumns.size()); for (TColumn other_element : other.sortedColumns) { __this__sortedColumns.add(new TColumn(other_element)); } @@ -398,40 +403,40 @@ public class TRowResult implements org.apache.thrift.TBase(2*_map8.size); for (int _i9 = 0; _i9 < _map8.size; ++_i9) { - ByteBuffer _key10; // required - TCell _val11; // required + ByteBuffer _key10; + TCell _val11; _key10 = iprot.readBinary(); _val11 = new TCell(); _val11.read(iprot); @@ -561,7 +567,7 @@ public class TRowResult implements org.apache.thrift.TBase(_list12.size); for (int _i13 = 0; _i13 < _list12.size; ++_i13) { - TColumn _elem14; // required + TColumn _elem14; _elem14 = new TColumn(); _elem14.read(iprot); struct.sortedColumns.add(_elem14); @@ -688,8 +694,8 @@ public class TRowResult implements org.apache.thrift.TBase(2*_map19.size); for (int _i20 = 0; _i20 < _map19.size; ++_i20) { - ByteBuffer _key21; // required - TCell _val22; // required + ByteBuffer _key21; + TCell _val22; _key21 = iprot.readBinary(); _val22 = new TCell(); _val22.read(iprot); @@ -704,7 +710,7 @@ public class TRowResult implements org.apache.thrift.TBase(_list23.size); for (int _i24 = 0; _i24 < _list23.size; ++_i24) { - TColumn _elem25; // required + TColumn _elem25; _elem25 = new TColumn(); _elem25.read(iprot); struct.sortedColumns.add(_elem25); diff --git a/src/main/java/org/apache/hadoop/hbase/thrift/generated/TScan.java b/src/main/java/org/apache/hadoop/hbase/thrift/generated/TScan.java index 0dce855..9ab9e8b 100644 --- a/src/main/java/org/apache/hadoop/hbase/thrift/generated/TScan.java +++ b/src/main/java/org/apache/hadoop/hbase/thrift/generated/TScan.java @@ -1,5 +1,5 @@ /** - * Autogenerated by Thrift Compiler (0.8.0) + * Autogenerated by Thrift Compiler (0.9.1) * * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING * @generated @@ -12,6 +12,11 @@ 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 org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; import java.util.List; import java.util.ArrayList; import java.util.Map; @@ -30,7 +35,7 @@ import org.slf4j.LoggerFactory; /** * A Scan object is used to specify scanner parameters when opening a scanner. */ -public class TScan implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { +public class TScan implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TScan"); private static final org.apache.thrift.protocol.TField START_ROW_FIELD_DESC = new org.apache.thrift.protocol.TField("startRow", org.apache.thrift.protocol.TType.STRING, (short)1); @@ -135,7 +140,7 @@ public class TScan implements org.apache.thrift.TBase, jav private static final int __TIMESTAMP_ISSET_ID = 0; private static final int __CACHING_ISSET_ID = 1; private static final int __SORTCOLUMNS_ISSET_ID = 2; - private BitSet __isset_bit_vector = new BitSet(3); + private byte __isset_bitfield = 0; private _Fields optionals[] = {_Fields.START_ROW,_Fields.STOP_ROW,_Fields.TIMESTAMP,_Fields.COLUMNS,_Fields.CACHING,_Fields.FILTER_STRING,_Fields.SORT_COLUMNS}; public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; static { @@ -166,8 +171,7 @@ public class TScan implements org.apache.thrift.TBase, jav * Performs a deep copy on other. */ public TScan(TScan other) { - __isset_bit_vector.clear(); - __isset_bit_vector.or(other.__isset_bit_vector); + __isset_bitfield = other.__isset_bitfield; if (other.isSetStartRow()) { this.startRow = other.startRow; } @@ -176,7 +180,7 @@ public class TScan implements org.apache.thrift.TBase, jav } this.timestamp = other.timestamp; if (other.isSetColumns()) { - List __this__columns = new ArrayList(); + List __this__columns = new ArrayList(other.columns.size()); for (ByteBuffer other_element : other.columns) { __this__columns.add(other_element); } @@ -286,16 +290,16 @@ public class TScan implements org.apache.thrift.TBase, jav } public void unsetTimestamp() { - __isset_bit_vector.clear(__TIMESTAMP_ISSET_ID); + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __TIMESTAMP_ISSET_ID); } /** Returns true if field timestamp is set (has been assigned a value) and false otherwise */ public boolean isSetTimestamp() { - return __isset_bit_vector.get(__TIMESTAMP_ISSET_ID); + return EncodingUtils.testBit(__isset_bitfield, __TIMESTAMP_ISSET_ID); } public void setTimestampIsSet(boolean value) { - __isset_bit_vector.set(__TIMESTAMP_ISSET_ID, value); + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __TIMESTAMP_ISSET_ID, value); } public int getColumnsSize() { @@ -348,16 +352,16 @@ public class TScan implements org.apache.thrift.TBase, jav } public void unsetCaching() { - __isset_bit_vector.clear(__CACHING_ISSET_ID); + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __CACHING_ISSET_ID); } /** Returns true if field caching is set (has been assigned a value) and false otherwise */ public boolean isSetCaching() { - return __isset_bit_vector.get(__CACHING_ISSET_ID); + return EncodingUtils.testBit(__isset_bitfield, __CACHING_ISSET_ID); } public void setCachingIsSet(boolean value) { - __isset_bit_vector.set(__CACHING_ISSET_ID, value); + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __CACHING_ISSET_ID, value); } public byte[] getFilterString() { @@ -405,16 +409,16 @@ public class TScan implements org.apache.thrift.TBase, jav } public void unsetSortColumns() { - __isset_bit_vector.clear(__SORTCOLUMNS_ISSET_ID); + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __SORTCOLUMNS_ISSET_ID); } /** Returns true if field sortColumns is set (has been assigned a value) and false otherwise */ public boolean isSetSortColumns() { - return __isset_bit_vector.get(__SORTCOLUMNS_ISSET_ID); + return EncodingUtils.testBit(__isset_bitfield, __SORTCOLUMNS_ISSET_ID); } public void setSortColumnsIsSet(boolean value) { - __isset_bit_vector.set(__SORTCOLUMNS_ISSET_ID, value); + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __SORTCOLUMNS_ISSET_ID, value); } public void setFieldValue(_Fields field, Object value) { @@ -614,80 +618,80 @@ public class TScan implements org.apache.thrift.TBase, jav return 0; } + @Override public int compareTo(TScan other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; - TScan typedOther = (TScan)other; - lastComparison = Boolean.valueOf(isSetStartRow()).compareTo(typedOther.isSetStartRow()); + lastComparison = Boolean.valueOf(isSetStartRow()).compareTo(other.isSetStartRow()); if (lastComparison != 0) { return lastComparison; } if (isSetStartRow()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.startRow, typedOther.startRow); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.startRow, other.startRow); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetStopRow()).compareTo(typedOther.isSetStopRow()); + lastComparison = Boolean.valueOf(isSetStopRow()).compareTo(other.isSetStopRow()); if (lastComparison != 0) { return lastComparison; } if (isSetStopRow()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.stopRow, typedOther.stopRow); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.stopRow, other.stopRow); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetTimestamp()).compareTo(typedOther.isSetTimestamp()); + lastComparison = Boolean.valueOf(isSetTimestamp()).compareTo(other.isSetTimestamp()); if (lastComparison != 0) { return lastComparison; } if (isSetTimestamp()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.timestamp, typedOther.timestamp); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.timestamp, other.timestamp); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetColumns()).compareTo(typedOther.isSetColumns()); + lastComparison = Boolean.valueOf(isSetColumns()).compareTo(other.isSetColumns()); if (lastComparison != 0) { return lastComparison; } if (isSetColumns()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.columns, typedOther.columns); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.columns, other.columns); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetCaching()).compareTo(typedOther.isSetCaching()); + lastComparison = Boolean.valueOf(isSetCaching()).compareTo(other.isSetCaching()); if (lastComparison != 0) { return lastComparison; } if (isSetCaching()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.caching, typedOther.caching); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.caching, other.caching); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetFilterString()).compareTo(typedOther.isSetFilterString()); + lastComparison = Boolean.valueOf(isSetFilterString()).compareTo(other.isSetFilterString()); if (lastComparison != 0) { return lastComparison; } if (isSetFilterString()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.filterString, typedOther.filterString); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.filterString, other.filterString); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetSortColumns()).compareTo(typedOther.isSetSortColumns()); + lastComparison = Boolean.valueOf(isSetSortColumns()).compareTo(other.isSetSortColumns()); if (lastComparison != 0) { return lastComparison; } if (isSetSortColumns()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.sortColumns, typedOther.sortColumns); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.sortColumns, other.sortColumns); if (lastComparison != 0) { return lastComparison; } @@ -775,6 +779,7 @@ public class TScan implements org.apache.thrift.TBase, jav 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 { @@ -788,7 +793,7 @@ public class TScan implements org.apache.thrift.TBase, jav 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_bit_vector = new BitSet(1); + __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); @@ -844,7 +849,7 @@ public class TScan implements org.apache.thrift.TBase, jav struct.columns = new ArrayList(_list26.size); for (int _i27 = 0; _i27 < _list26.size; ++_i27) { - ByteBuffer _elem28; // required + ByteBuffer _elem28; _elem28 = iprot.readBinary(); struct.columns.add(_elem28); } @@ -1035,7 +1040,7 @@ public class TScan implements org.apache.thrift.TBase, jav struct.columns = new ArrayList(_list31.size); for (int _i32 = 0; _i32 < _list31.size; ++_i32) { - ByteBuffer _elem33; // required + ByteBuffer _elem33; _elem33 = iprot.readBinary(); struct.columns.add(_elem33); } diff --git a/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TColumn.java b/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TColumn.java index ba11ece..1481325 100644 --- a/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TColumn.java +++ b/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TColumn.java @@ -1,5 +1,5 @@ /** - * Autogenerated by Thrift Compiler (0.8.0) + * Autogenerated by Thrift Compiler (0.9.1) * * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING * @generated @@ -12,6 +12,11 @@ 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 org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; import java.util.List; import java.util.ArrayList; import java.util.Map; @@ -32,7 +37,7 @@ import org.slf4j.LoggerFactory; * in a HBase table by column family and optionally * a column qualifier and timestamp */ -public class TColumn implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { +public class TColumn implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TColumn"); private static final org.apache.thrift.protocol.TField FAMILY_FIELD_DESC = new org.apache.thrift.protocol.TField("family", org.apache.thrift.protocol.TType.STRING, (short)1); @@ -115,7 +120,7 @@ public class TColumn implements org.apache.thrift.TBase metaDataMap; static { @@ -144,8 +149,7 @@ public class TColumn implements org.apache.thrift.TBaseother. */ public TColumn(TColumn other) { - __isset_bit_vector.clear(); - __isset_bit_vector.or(other.__isset_bit_vector); + __isset_bitfield = other.__isset_bitfield; if (other.isSetFamily()) { this.family = org.apache.thrift.TBaseHelper.copyBinary(other.family); ; @@ -248,16 +252,16 @@ public class TColumn implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { +public class TColumnIncrement implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TColumnIncrement"); private static final org.apache.thrift.protocol.TField FAMILY_FIELD_DESC = new org.apache.thrift.protocol.TField("family", org.apache.thrift.protocol.TType.STRING, (short)1); @@ -113,7 +118,7 @@ public class TColumnIncrement implements org.apache.thrift.TBase metaDataMap; static { @@ -146,8 +151,7 @@ public class TColumnIncrement implements org.apache.thrift.TBaseother. */ public TColumnIncrement(TColumnIncrement other) { - __isset_bit_vector.clear(); - __isset_bit_vector.or(other.__isset_bit_vector); + __isset_bitfield = other.__isset_bitfield; if (other.isSetFamily()) { this.family = org.apache.thrift.TBaseHelper.copyBinary(other.family); ; @@ -250,16 +254,16 @@ public class TColumnIncrement implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { +public class TColumnValue implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TColumnValue"); private static final org.apache.thrift.protocol.TField FAMILY_FIELD_DESC = new org.apache.thrift.protocol.TField("family", org.apache.thrift.protocol.TType.STRING, (short)1); @@ -118,7 +123,7 @@ public class TColumnValue implements org.apache.thrift.TBase metaDataMap; static { @@ -153,8 +158,7 @@ public class TColumnValue implements org.apache.thrift.TBaseother. */ public TColumnValue(TColumnValue other) { - __isset_bit_vector.clear(); - __isset_bit_vector.or(other.__isset_bit_vector); + __isset_bitfield = other.__isset_bitfield; if (other.isSetFamily()) { this.family = org.apache.thrift.TBaseHelper.copyBinary(other.family); ; @@ -296,16 +300,16 @@ public class TColumnValue implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { +public class TDelete implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TDelete"); private static final org.apache.thrift.protocol.TField ROW_FIELD_DESC = new org.apache.thrift.protocol.TField("row", org.apache.thrift.protocol.TType.STRING, (short)1); @@ -170,7 +175,7 @@ public class TDelete implements org.apache.thrift.TBase metaDataMap; static { @@ -212,14 +217,13 @@ public class TDelete implements org.apache.thrift.TBaseother. */ public TDelete(TDelete other) { - __isset_bit_vector.clear(); - __isset_bit_vector.or(other.__isset_bit_vector); + __isset_bitfield = other.__isset_bitfield; if (other.isSetRow()) { this.row = org.apache.thrift.TBaseHelper.copyBinary(other.row); ; } if (other.isSetColumns()) { - List __this__columns = new ArrayList(); + List __this__columns = new ArrayList(other.columns.size()); for (TColumn other_element : other.columns) { __this__columns.add(new TColumn(other_element)); } @@ -231,20 +235,7 @@ public class TDelete implements org.apache.thrift.TBase __this__attributes = new HashMap(); - for (Map.Entry other_element : other.attributes.entrySet()) { - - ByteBuffer other_element_key = other_element.getKey(); - ByteBuffer other_element_value = other_element.getValue(); - - ByteBuffer __this__attributes_copy_key = org.apache.thrift.TBaseHelper.copyBinary(other_element_key); -; - - ByteBuffer __this__attributes_copy_value = org.apache.thrift.TBaseHelper.copyBinary(other_element_value); -; - - __this__attributes.put(__this__attributes_copy_key, __this__attributes_copy_value); - } + Map __this__attributes = new HashMap(other.attributes); this.attributes = __this__attributes; } if (other.isSetDurability()) { @@ -354,16 +345,16 @@ public class TDelete implements org.apache.thrift.TBase(_list44.size); for (int _i45 = 0; _i45 < _list44.size; ++_i45) { - TColumn _elem46; // required + TColumn _elem46; _elem46 = new TColumn(); _elem46.read(iprot); struct.columns.add(_elem46); @@ -947,8 +939,8 @@ public class TDelete implements org.apache.thrift.TBase(2*_map47.size); for (int _i48 = 0; _i48 < _map47.size; ++_i48) { - ByteBuffer _key49; // required - ByteBuffer _val50; // optional + ByteBuffer _key49; + ByteBuffer _val50; _key49 = iprot.readBinary(); _val50 = iprot.readBinary(); struct.attributes.put(_key49, _val50); @@ -1124,7 +1116,7 @@ public class TDelete implements org.apache.thrift.TBase(_list55.size); for (int _i56 = 0; _i56 < _list55.size; ++_i56) { - TColumn _elem57; // required + TColumn _elem57; _elem57 = new TColumn(); _elem57.read(iprot); struct.columns.add(_elem57); @@ -1150,8 +1142,8 @@ public class TDelete implements org.apache.thrift.TBase(2*_map58.size); for (int _i59 = 0; _i59 < _map58.size; ++_i59) { - ByteBuffer _key60; // required - ByteBuffer _val61; // optional + ByteBuffer _key60; + ByteBuffer _val61; _key60 = iprot.readBinary(); _val61 = iprot.readBinary(); struct.attributes.put(_key60, _val61); diff --git a/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TDeleteType.java b/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TDeleteType.java index e5d77a6..4e9d226 100644 --- a/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TDeleteType.java +++ b/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TDeleteType.java @@ -1,5 +1,5 @@ /** - * Autogenerated by Thrift Compiler (0.8.0) + * Autogenerated by Thrift Compiler (0.9.1) * * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING * @generated diff --git a/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TDurability.java b/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TDurability.java index cffe84c..7607841 100644 --- a/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TDurability.java +++ b/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TDurability.java @@ -1,5 +1,5 @@ /** - * Autogenerated by Thrift Compiler (0.8.0) + * Autogenerated by Thrift Compiler (0.9.1) * * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING * @generated diff --git a/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TGet.java b/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TGet.java index fca79c3..11b393e 100644 --- a/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TGet.java +++ b/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TGet.java @@ -1,5 +1,5 @@ /** - * Autogenerated by Thrift Compiler (0.8.0) + * Autogenerated by Thrift Compiler (0.9.1) * * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING * @generated @@ -12,6 +12,11 @@ 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 org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; import java.util.List; import java.util.ArrayList; import java.util.Map; @@ -40,7 +45,7 @@ import org.slf4j.LoggerFactory; * If you specify a time range and a timestamp the range is ignored. * Timestamps on TColumns are ignored. */ -public class TGet implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { +public class TGet implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TGet"); private static final org.apache.thrift.protocol.TField ROW_FIELD_DESC = new org.apache.thrift.protocol.TField("row", org.apache.thrift.protocol.TType.STRING, (short)1); @@ -144,7 +149,7 @@ public class TGet implements org.apache.thrift.TBase, java.i // isset id assignments private static final int __TIMESTAMP_ISSET_ID = 0; private static final int __MAXVERSIONS_ISSET_ID = 1; - private BitSet __isset_bit_vector = new BitSet(2); + private byte __isset_bitfield = 0; private _Fields optionals[] = {_Fields.COLUMNS,_Fields.TIMESTAMP,_Fields.TIME_RANGE,_Fields.MAX_VERSIONS,_Fields.FILTER_STRING,_Fields.ATTRIBUTES}; public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; static { @@ -184,14 +189,13 @@ public class TGet implements org.apache.thrift.TBase, java.i * Performs a deep copy on other. */ public TGet(TGet other) { - __isset_bit_vector.clear(); - __isset_bit_vector.or(other.__isset_bit_vector); + __isset_bitfield = other.__isset_bitfield; if (other.isSetRow()) { this.row = org.apache.thrift.TBaseHelper.copyBinary(other.row); ; } if (other.isSetColumns()) { - List __this__columns = new ArrayList(); + List __this__columns = new ArrayList(other.columns.size()); for (TColumn other_element : other.columns) { __this__columns.add(new TColumn(other_element)); } @@ -207,20 +211,7 @@ public class TGet implements org.apache.thrift.TBase, java.i ; } if (other.isSetAttributes()) { - Map __this__attributes = new HashMap(); - for (Map.Entry other_element : other.attributes.entrySet()) { - - ByteBuffer other_element_key = other_element.getKey(); - ByteBuffer other_element_value = other_element.getValue(); - - ByteBuffer __this__attributes_copy_key = org.apache.thrift.TBaseHelper.copyBinary(other_element_key); -; - - ByteBuffer __this__attributes_copy_value = org.apache.thrift.TBaseHelper.copyBinary(other_element_value); -; - - __this__attributes.put(__this__attributes_copy_key, __this__attributes_copy_value); - } + Map __this__attributes = new HashMap(other.attributes); this.attributes = __this__attributes; } } @@ -326,16 +317,16 @@ public class TGet implements org.apache.thrift.TBase, java.i } public void unsetTimestamp() { - __isset_bit_vector.clear(__TIMESTAMP_ISSET_ID); + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __TIMESTAMP_ISSET_ID); } /** Returns true if field timestamp is set (has been assigned a value) and false otherwise */ public boolean isSetTimestamp() { - return __isset_bit_vector.get(__TIMESTAMP_ISSET_ID); + return EncodingUtils.testBit(__isset_bitfield, __TIMESTAMP_ISSET_ID); } public void setTimestampIsSet(boolean value) { - __isset_bit_vector.set(__TIMESTAMP_ISSET_ID, value); + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __TIMESTAMP_ISSET_ID, value); } public TTimeRange getTimeRange() { @@ -373,16 +364,16 @@ public class TGet implements org.apache.thrift.TBase, java.i } public void unsetMaxVersions() { - __isset_bit_vector.clear(__MAXVERSIONS_ISSET_ID); + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __MAXVERSIONS_ISSET_ID); } /** Returns true if field maxVersions is set (has been assigned a value) and false otherwise */ public boolean isSetMaxVersions() { - return __isset_bit_vector.get(__MAXVERSIONS_ISSET_ID); + return EncodingUtils.testBit(__isset_bitfield, __MAXVERSIONS_ISSET_ID); } public void setMaxVersionsIsSet(boolean value) { - __isset_bit_vector.set(__MAXVERSIONS_ISSET_ID, value); + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __MAXVERSIONS_ISSET_ID, value); } public byte[] getFilterString() { @@ -651,80 +642,80 @@ public class TGet implements org.apache.thrift.TBase, java.i return 0; } + @Override public int compareTo(TGet other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; - TGet typedOther = (TGet)other; - lastComparison = Boolean.valueOf(isSetRow()).compareTo(typedOther.isSetRow()); + lastComparison = Boolean.valueOf(isSetRow()).compareTo(other.isSetRow()); if (lastComparison != 0) { return lastComparison; } if (isSetRow()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.row, typedOther.row); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.row, other.row); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetColumns()).compareTo(typedOther.isSetColumns()); + lastComparison = Boolean.valueOf(isSetColumns()).compareTo(other.isSetColumns()); if (lastComparison != 0) { return lastComparison; } if (isSetColumns()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.columns, typedOther.columns); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.columns, other.columns); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetTimestamp()).compareTo(typedOther.isSetTimestamp()); + lastComparison = Boolean.valueOf(isSetTimestamp()).compareTo(other.isSetTimestamp()); if (lastComparison != 0) { return lastComparison; } if (isSetTimestamp()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.timestamp, typedOther.timestamp); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.timestamp, other.timestamp); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetTimeRange()).compareTo(typedOther.isSetTimeRange()); + lastComparison = Boolean.valueOf(isSetTimeRange()).compareTo(other.isSetTimeRange()); if (lastComparison != 0) { return lastComparison; } if (isSetTimeRange()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.timeRange, typedOther.timeRange); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.timeRange, other.timeRange); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetMaxVersions()).compareTo(typedOther.isSetMaxVersions()); + lastComparison = Boolean.valueOf(isSetMaxVersions()).compareTo(other.isSetMaxVersions()); if (lastComparison != 0) { return lastComparison; } if (isSetMaxVersions()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.maxVersions, typedOther.maxVersions); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.maxVersions, other.maxVersions); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetFilterString()).compareTo(typedOther.isSetFilterString()); + lastComparison = Boolean.valueOf(isSetFilterString()).compareTo(other.isSetFilterString()); if (lastComparison != 0) { return lastComparison; } if (isSetFilterString()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.filterString, typedOther.filterString); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.filterString, other.filterString); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetAttributes()).compareTo(typedOther.isSetAttributes()); + lastComparison = Boolean.valueOf(isSetAttributes()).compareTo(other.isSetAttributes()); if (lastComparison != 0) { return lastComparison; } if (isSetAttributes()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.attributes, typedOther.attributes); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.attributes, other.attributes); if (lastComparison != 0) { return lastComparison; } @@ -817,6 +808,10 @@ public class TGet implements org.apache.thrift.TBase, java.i if (row == null) { throw new org.apache.thrift.protocol.TProtocolException("Required field 'row' was not present! Struct: " + toString()); } + // check for sub-struct validity + if (timeRange != null) { + timeRange.validate(); + } } private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { @@ -830,7 +825,7 @@ public class TGet implements org.apache.thrift.TBase, java.i 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_bit_vector = new BitSet(1); + __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); @@ -870,7 +865,7 @@ public class TGet implements org.apache.thrift.TBase, java.i struct.columns = new ArrayList(_list8.size); for (int _i9 = 0; _i9 < _list8.size; ++_i9) { - TColumn _elem10; // required + TColumn _elem10; _elem10 = new TColumn(); _elem10.read(iprot); struct.columns.add(_elem10); @@ -922,8 +917,8 @@ public class TGet implements org.apache.thrift.TBase, java.i struct.attributes = new HashMap(2*_map11.size); for (int _i12 = 0; _i12 < _map11.size; ++_i12) { - ByteBuffer _key13; // required - ByteBuffer _val14; // optional + ByteBuffer _key13; + ByteBuffer _val14; _key13 = iprot.readBinary(); _val14 = iprot.readBinary(); struct.attributes.put(_key13, _val14); @@ -1091,7 +1086,7 @@ public class TGet implements org.apache.thrift.TBase, java.i struct.columns = new ArrayList(_list19.size); for (int _i20 = 0; _i20 < _list19.size; ++_i20) { - TColumn _elem21; // required + TColumn _elem21; _elem21 = new TColumn(); _elem21.read(iprot); struct.columns.add(_elem21); @@ -1122,8 +1117,8 @@ public class TGet implements org.apache.thrift.TBase, java.i struct.attributes = new HashMap(2*_map22.size); for (int _i23 = 0; _i23 < _map22.size; ++_i23) { - ByteBuffer _key24; // required - ByteBuffer _val25; // optional + ByteBuffer _key24; + ByteBuffer _val25; _key24 = iprot.readBinary(); _val25 = iprot.readBinary(); struct.attributes.put(_key24, _val25); diff --git a/src/main/java/org/apache/hadoop/hbase/thrift2/generated/THBaseService.java b/src/main/java/org/apache/hadoop/hbase/thrift2/generated/THBaseService.java index 4a80126..c4b8033 100644 --- a/src/main/java/org/apache/hadoop/hbase/thrift2/generated/THBaseService.java +++ b/src/main/java/org/apache/hadoop/hbase/thrift2/generated/THBaseService.java @@ -1,5 +1,5 @@ /** - * Autogenerated by Thrift Compiler (0.8.0) + * Autogenerated by Thrift Compiler (0.9.1) * * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING * @generated @@ -12,6 +12,11 @@ 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 org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; import java.util.List; import java.util.ArrayList; import java.util.Map; @@ -220,35 +225,35 @@ public class THBaseService { public interface AsyncIface { - public void exists(ByteBuffer table, TGet get, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + public void exists(ByteBuffer table, TGet get, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; - public void get(ByteBuffer table, TGet get, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + public void get(ByteBuffer table, TGet get, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; - public void getMultiple(ByteBuffer table, List gets, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + public void getMultiple(ByteBuffer table, List gets, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; - public void put(ByteBuffer table, TPut put, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + public void put(ByteBuffer table, TPut put, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; - public void checkAndPut(ByteBuffer table, ByteBuffer row, ByteBuffer family, ByteBuffer qualifier, ByteBuffer value, TPut put, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + public void checkAndPut(ByteBuffer table, ByteBuffer row, ByteBuffer family, ByteBuffer qualifier, ByteBuffer value, TPut put, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; - public void putMultiple(ByteBuffer table, List puts, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + public void putMultiple(ByteBuffer table, List puts, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; - public void deleteSingle(ByteBuffer table, TDelete deleteSingle, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + public void deleteSingle(ByteBuffer table, TDelete deleteSingle, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; - public void deleteMultiple(ByteBuffer table, List deletes, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + public void deleteMultiple(ByteBuffer table, List deletes, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; - public void checkAndDelete(ByteBuffer table, ByteBuffer row, ByteBuffer family, ByteBuffer qualifier, ByteBuffer value, TDelete deleteSingle, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + public void checkAndDelete(ByteBuffer table, ByteBuffer row, ByteBuffer family, ByteBuffer qualifier, ByteBuffer value, TDelete deleteSingle, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; - public void increment(ByteBuffer table, TIncrement increment, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + public void increment(ByteBuffer table, TIncrement increment, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; - public void openScanner(ByteBuffer table, TScan scan, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + public void openScanner(ByteBuffer table, TScan scan, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; - public void getScannerRows(int scannerId, int numRows, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + public void getScannerRows(int scannerId, int numRows, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; - public void closeScanner(int scannerId, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + public void closeScanner(int scannerId, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; - public void mutateRow(ByteBuffer table, TRowMutations rowMutations, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + public void mutateRow(ByteBuffer table, TRowMutations rowMutations, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; - public void getScannerResults(ByteBuffer table, TScan scan, int numRows, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + public void getScannerResults(ByteBuffer table, TScan scan, int numRows, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; } @@ -694,7 +699,7 @@ public class THBaseService { super(protocolFactory, clientManager, transport); } - public void exists(ByteBuffer table, TGet get, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + public void exists(ByteBuffer table, TGet get, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { checkReady(); exists_call method_call = new exists_call(table, get, resultHandler, this, ___protocolFactory, ___transport); this.___currentMethod = method_call; @@ -704,7 +709,7 @@ public class THBaseService { public static class exists_call extends org.apache.thrift.async.TAsyncMethodCall { private ByteBuffer table; private TGet get; - public exists_call(ByteBuffer table, TGet get, 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 { + public exists_call(ByteBuffer table, TGet get, 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.table = table; this.get = get; @@ -729,7 +734,7 @@ public class THBaseService { } } - public void get(ByteBuffer table, TGet get, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + public void get(ByteBuffer table, TGet get, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { checkReady(); get_call method_call = new get_call(table, get, resultHandler, this, ___protocolFactory, ___transport); this.___currentMethod = method_call; @@ -739,7 +744,7 @@ public class THBaseService { public static class get_call extends org.apache.thrift.async.TAsyncMethodCall { private ByteBuffer table; private TGet get; - public get_call(ByteBuffer table, TGet get, 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 { + public get_call(ByteBuffer table, TGet get, 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.table = table; this.get = get; @@ -764,7 +769,7 @@ public class THBaseService { } } - public void getMultiple(ByteBuffer table, List gets, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + public void getMultiple(ByteBuffer table, List gets, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { checkReady(); getMultiple_call method_call = new getMultiple_call(table, gets, resultHandler, this, ___protocolFactory, ___transport); this.___currentMethod = method_call; @@ -774,7 +779,7 @@ public class THBaseService { public static class getMultiple_call extends org.apache.thrift.async.TAsyncMethodCall { private ByteBuffer table; private List gets; - public getMultiple_call(ByteBuffer table, List gets, 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 { + public getMultiple_call(ByteBuffer table, List gets, 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.table = table; this.gets = gets; @@ -799,7 +804,7 @@ public class THBaseService { } } - public void put(ByteBuffer table, TPut put, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + public void put(ByteBuffer table, TPut put, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { checkReady(); put_call method_call = new put_call(table, put, resultHandler, this, ___protocolFactory, ___transport); this.___currentMethod = method_call; @@ -809,7 +814,7 @@ public class THBaseService { public static class put_call extends org.apache.thrift.async.TAsyncMethodCall { private ByteBuffer table; private TPut put; - public put_call(ByteBuffer table, TPut put, 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 { + public put_call(ByteBuffer table, TPut put, 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.table = table; this.put = put; @@ -834,7 +839,7 @@ public class THBaseService { } } - public void checkAndPut(ByteBuffer table, ByteBuffer row, ByteBuffer family, ByteBuffer qualifier, ByteBuffer value, TPut put, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + public void checkAndPut(ByteBuffer table, ByteBuffer row, ByteBuffer family, ByteBuffer qualifier, ByteBuffer value, TPut put, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { checkReady(); checkAndPut_call method_call = new checkAndPut_call(table, row, family, qualifier, value, put, resultHandler, this, ___protocolFactory, ___transport); this.___currentMethod = method_call; @@ -848,7 +853,7 @@ public class THBaseService { private ByteBuffer qualifier; private ByteBuffer value; private TPut put; - public checkAndPut_call(ByteBuffer table, ByteBuffer row, ByteBuffer family, ByteBuffer qualifier, ByteBuffer value, TPut put, 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 { + public checkAndPut_call(ByteBuffer table, ByteBuffer row, ByteBuffer family, ByteBuffer qualifier, ByteBuffer value, TPut put, 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.table = table; this.row = row; @@ -881,7 +886,7 @@ public class THBaseService { } } - public void putMultiple(ByteBuffer table, List puts, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + public void putMultiple(ByteBuffer table, List puts, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { checkReady(); putMultiple_call method_call = new putMultiple_call(table, puts, resultHandler, this, ___protocolFactory, ___transport); this.___currentMethod = method_call; @@ -891,7 +896,7 @@ public class THBaseService { public static class putMultiple_call extends org.apache.thrift.async.TAsyncMethodCall { private ByteBuffer table; private List puts; - public putMultiple_call(ByteBuffer table, List puts, 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 { + public putMultiple_call(ByteBuffer table, List puts, 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.table = table; this.puts = puts; @@ -916,7 +921,7 @@ public class THBaseService { } } - public void deleteSingle(ByteBuffer table, TDelete deleteSingle, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + public void deleteSingle(ByteBuffer table, TDelete deleteSingle, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { checkReady(); deleteSingle_call method_call = new deleteSingle_call(table, deleteSingle, resultHandler, this, ___protocolFactory, ___transport); this.___currentMethod = method_call; @@ -926,7 +931,7 @@ public class THBaseService { public static class deleteSingle_call extends org.apache.thrift.async.TAsyncMethodCall { private ByteBuffer table; private TDelete deleteSingle; - public deleteSingle_call(ByteBuffer table, TDelete deleteSingle, 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 { + public deleteSingle_call(ByteBuffer table, TDelete deleteSingle, 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.table = table; this.deleteSingle = deleteSingle; @@ -951,7 +956,7 @@ public class THBaseService { } } - public void deleteMultiple(ByteBuffer table, List deletes, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + public void deleteMultiple(ByteBuffer table, List deletes, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { checkReady(); deleteMultiple_call method_call = new deleteMultiple_call(table, deletes, resultHandler, this, ___protocolFactory, ___transport); this.___currentMethod = method_call; @@ -961,7 +966,7 @@ public class THBaseService { public static class deleteMultiple_call extends org.apache.thrift.async.TAsyncMethodCall { private ByteBuffer table; private List deletes; - public deleteMultiple_call(ByteBuffer table, List deletes, 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 { + public deleteMultiple_call(ByteBuffer table, List deletes, 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.table = table; this.deletes = deletes; @@ -986,7 +991,7 @@ public class THBaseService { } } - public void checkAndDelete(ByteBuffer table, ByteBuffer row, ByteBuffer family, ByteBuffer qualifier, ByteBuffer value, TDelete deleteSingle, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + public void checkAndDelete(ByteBuffer table, ByteBuffer row, ByteBuffer family, ByteBuffer qualifier, ByteBuffer value, TDelete deleteSingle, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { checkReady(); checkAndDelete_call method_call = new checkAndDelete_call(table, row, family, qualifier, value, deleteSingle, resultHandler, this, ___protocolFactory, ___transport); this.___currentMethod = method_call; @@ -1000,7 +1005,7 @@ public class THBaseService { private ByteBuffer qualifier; private ByteBuffer value; private TDelete deleteSingle; - public checkAndDelete_call(ByteBuffer table, ByteBuffer row, ByteBuffer family, ByteBuffer qualifier, ByteBuffer value, TDelete deleteSingle, 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 { + public checkAndDelete_call(ByteBuffer table, ByteBuffer row, ByteBuffer family, ByteBuffer qualifier, ByteBuffer value, TDelete deleteSingle, 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.table = table; this.row = row; @@ -1033,7 +1038,7 @@ public class THBaseService { } } - public void increment(ByteBuffer table, TIncrement increment, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + public void increment(ByteBuffer table, TIncrement increment, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { checkReady(); increment_call method_call = new increment_call(table, increment, resultHandler, this, ___protocolFactory, ___transport); this.___currentMethod = method_call; @@ -1043,7 +1048,7 @@ public class THBaseService { public static class increment_call extends org.apache.thrift.async.TAsyncMethodCall { private ByteBuffer table; private TIncrement increment; - public increment_call(ByteBuffer table, TIncrement increment, 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 { + public increment_call(ByteBuffer table, TIncrement increment, 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.table = table; this.increment = increment; @@ -1068,7 +1073,7 @@ public class THBaseService { } } - public void openScanner(ByteBuffer table, TScan scan, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + public void openScanner(ByteBuffer table, TScan scan, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { checkReady(); openScanner_call method_call = new openScanner_call(table, scan, resultHandler, this, ___protocolFactory, ___transport); this.___currentMethod = method_call; @@ -1078,7 +1083,7 @@ public class THBaseService { public static class openScanner_call extends org.apache.thrift.async.TAsyncMethodCall { private ByteBuffer table; private TScan scan; - public openScanner_call(ByteBuffer table, TScan scan, 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 { + public openScanner_call(ByteBuffer table, TScan scan, 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.table = table; this.scan = scan; @@ -1103,7 +1108,7 @@ public class THBaseService { } } - public void getScannerRows(int scannerId, int numRows, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + public void getScannerRows(int scannerId, int numRows, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { checkReady(); getScannerRows_call method_call = new getScannerRows_call(scannerId, numRows, resultHandler, this, ___protocolFactory, ___transport); this.___currentMethod = method_call; @@ -1113,7 +1118,7 @@ public class THBaseService { public static class getScannerRows_call extends org.apache.thrift.async.TAsyncMethodCall { private int scannerId; private int numRows; - public getScannerRows_call(int scannerId, int numRows, 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 { + public getScannerRows_call(int scannerId, int numRows, 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.scannerId = scannerId; this.numRows = numRows; @@ -1138,7 +1143,7 @@ public class THBaseService { } } - public void closeScanner(int scannerId, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + public void closeScanner(int scannerId, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { checkReady(); closeScanner_call method_call = new closeScanner_call(scannerId, resultHandler, this, ___protocolFactory, ___transport); this.___currentMethod = method_call; @@ -1147,7 +1152,7 @@ public class THBaseService { public static class closeScanner_call extends org.apache.thrift.async.TAsyncMethodCall { private int scannerId; - public closeScanner_call(int scannerId, 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 { + public closeScanner_call(int scannerId, 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.scannerId = scannerId; } @@ -1170,7 +1175,7 @@ public class THBaseService { } } - public void mutateRow(ByteBuffer table, TRowMutations rowMutations, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + public void mutateRow(ByteBuffer table, TRowMutations rowMutations, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { checkReady(); mutateRow_call method_call = new mutateRow_call(table, rowMutations, resultHandler, this, ___protocolFactory, ___transport); this.___currentMethod = method_call; @@ -1180,7 +1185,7 @@ public class THBaseService { public static class mutateRow_call extends org.apache.thrift.async.TAsyncMethodCall { private ByteBuffer table; private TRowMutations rowMutations; - public mutateRow_call(ByteBuffer table, TRowMutations rowMutations, 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 { + public mutateRow_call(ByteBuffer table, TRowMutations rowMutations, 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.table = table; this.rowMutations = rowMutations; @@ -1205,7 +1210,7 @@ public class THBaseService { } } - public void getScannerResults(ByteBuffer table, TScan scan, int numRows, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + public void getScannerResults(ByteBuffer table, TScan scan, int numRows, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { checkReady(); getScannerResults_call method_call = new getScannerResults_call(table, scan, numRows, resultHandler, this, ___protocolFactory, ___transport); this.___currentMethod = method_call; @@ -1216,7 +1221,7 @@ public class THBaseService { private ByteBuffer table; private TScan scan; private int numRows; - public getScannerResults_call(ByteBuffer table, TScan scan, int numRows, 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 { + public getScannerResults_call(ByteBuffer table, TScan scan, int numRows, 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.table = table; this.scan = scan; @@ -1274,16 +1279,20 @@ public class THBaseService { return processMap; } - private static class exists extends org.apache.thrift.ProcessFunction { + public static class exists extends org.apache.thrift.ProcessFunction { public exists() { super("exists"); } - protected exists_args getEmptyArgsInstance() { + public exists_args getEmptyArgsInstance() { return new exists_args(); } - protected exists_result getResult(I iface, exists_args args) throws org.apache.thrift.TException { + protected boolean isOneway() { + return false; + } + + public exists_result getResult(I iface, exists_args args) throws org.apache.thrift.TException { exists_result result = new exists_result(); try { result.success = iface.exists(args.table, args.get); @@ -1295,16 +1304,20 @@ public class THBaseService { } } - private static class get extends org.apache.thrift.ProcessFunction { + public static class get extends org.apache.thrift.ProcessFunction { public get() { super("get"); } - protected get_args getEmptyArgsInstance() { + public get_args getEmptyArgsInstance() { return new get_args(); } - protected get_result getResult(I iface, get_args args) throws org.apache.thrift.TException { + protected boolean isOneway() { + return false; + } + + public get_result getResult(I iface, get_args args) throws org.apache.thrift.TException { get_result result = new get_result(); try { result.success = iface.get(args.table, args.get); @@ -1315,16 +1328,20 @@ public class THBaseService { } } - private static class getMultiple extends org.apache.thrift.ProcessFunction { + public static class getMultiple extends org.apache.thrift.ProcessFunction { public getMultiple() { super("getMultiple"); } - protected getMultiple_args getEmptyArgsInstance() { + public getMultiple_args getEmptyArgsInstance() { return new getMultiple_args(); } - protected getMultiple_result getResult(I iface, getMultiple_args args) throws org.apache.thrift.TException { + protected boolean isOneway() { + return false; + } + + public getMultiple_result getResult(I iface, getMultiple_args args) throws org.apache.thrift.TException { getMultiple_result result = new getMultiple_result(); try { result.success = iface.getMultiple(args.table, args.gets); @@ -1335,16 +1352,20 @@ public class THBaseService { } } - private static class put extends org.apache.thrift.ProcessFunction { + public static class put extends org.apache.thrift.ProcessFunction { public put() { super("put"); } - protected put_args getEmptyArgsInstance() { + public put_args getEmptyArgsInstance() { return new put_args(); } - protected put_result getResult(I iface, put_args args) throws org.apache.thrift.TException { + protected boolean isOneway() { + return false; + } + + public put_result getResult(I iface, put_args args) throws org.apache.thrift.TException { put_result result = new put_result(); try { iface.put(args.table, args.put); @@ -1355,16 +1376,20 @@ public class THBaseService { } } - private static class checkAndPut extends org.apache.thrift.ProcessFunction { + public static class checkAndPut extends org.apache.thrift.ProcessFunction { public checkAndPut() { super("checkAndPut"); } - protected checkAndPut_args getEmptyArgsInstance() { + public checkAndPut_args getEmptyArgsInstance() { return new checkAndPut_args(); } - protected checkAndPut_result getResult(I iface, checkAndPut_args args) throws org.apache.thrift.TException { + protected boolean isOneway() { + return false; + } + + public checkAndPut_result getResult(I iface, checkAndPut_args args) throws org.apache.thrift.TException { checkAndPut_result result = new checkAndPut_result(); try { result.success = iface.checkAndPut(args.table, args.row, args.family, args.qualifier, args.value, args.put); @@ -1376,16 +1401,20 @@ public class THBaseService { } } - private static class putMultiple extends org.apache.thrift.ProcessFunction { + public static class putMultiple extends org.apache.thrift.ProcessFunction { public putMultiple() { super("putMultiple"); } - protected putMultiple_args getEmptyArgsInstance() { + public putMultiple_args getEmptyArgsInstance() { return new putMultiple_args(); } - protected putMultiple_result getResult(I iface, putMultiple_args args) throws org.apache.thrift.TException { + protected boolean isOneway() { + return false; + } + + public putMultiple_result getResult(I iface, putMultiple_args args) throws org.apache.thrift.TException { putMultiple_result result = new putMultiple_result(); try { iface.putMultiple(args.table, args.puts); @@ -1396,16 +1425,20 @@ public class THBaseService { } } - private static class deleteSingle extends org.apache.thrift.ProcessFunction { + public static class deleteSingle extends org.apache.thrift.ProcessFunction { public deleteSingle() { super("deleteSingle"); } - protected deleteSingle_args getEmptyArgsInstance() { + public deleteSingle_args getEmptyArgsInstance() { return new deleteSingle_args(); } - protected deleteSingle_result getResult(I iface, deleteSingle_args args) throws org.apache.thrift.TException { + protected boolean isOneway() { + return false; + } + + public deleteSingle_result getResult(I iface, deleteSingle_args args) throws org.apache.thrift.TException { deleteSingle_result result = new deleteSingle_result(); try { iface.deleteSingle(args.table, args.deleteSingle); @@ -1416,16 +1449,20 @@ public class THBaseService { } } - private static class deleteMultiple extends org.apache.thrift.ProcessFunction { + public static class deleteMultiple extends org.apache.thrift.ProcessFunction { public deleteMultiple() { super("deleteMultiple"); } - protected deleteMultiple_args getEmptyArgsInstance() { + public deleteMultiple_args getEmptyArgsInstance() { return new deleteMultiple_args(); } - protected deleteMultiple_result getResult(I iface, deleteMultiple_args args) throws org.apache.thrift.TException { + protected boolean isOneway() { + return false; + } + + public deleteMultiple_result getResult(I iface, deleteMultiple_args args) throws org.apache.thrift.TException { deleteMultiple_result result = new deleteMultiple_result(); try { result.success = iface.deleteMultiple(args.table, args.deletes); @@ -1436,16 +1473,20 @@ public class THBaseService { } } - private static class checkAndDelete extends org.apache.thrift.ProcessFunction { + public static class checkAndDelete extends org.apache.thrift.ProcessFunction { public checkAndDelete() { super("checkAndDelete"); } - protected checkAndDelete_args getEmptyArgsInstance() { + public checkAndDelete_args getEmptyArgsInstance() { return new checkAndDelete_args(); } - protected checkAndDelete_result getResult(I iface, checkAndDelete_args args) throws org.apache.thrift.TException { + protected boolean isOneway() { + return false; + } + + public checkAndDelete_result getResult(I iface, checkAndDelete_args args) throws org.apache.thrift.TException { checkAndDelete_result result = new checkAndDelete_result(); try { result.success = iface.checkAndDelete(args.table, args.row, args.family, args.qualifier, args.value, args.deleteSingle); @@ -1457,16 +1498,20 @@ public class THBaseService { } } - private static class increment extends org.apache.thrift.ProcessFunction { + public static class increment extends org.apache.thrift.ProcessFunction { public increment() { super("increment"); } - protected increment_args getEmptyArgsInstance() { + public increment_args getEmptyArgsInstance() { return new increment_args(); } - protected increment_result getResult(I iface, increment_args args) throws org.apache.thrift.TException { + protected boolean isOneway() { + return false; + } + + public increment_result getResult(I iface, increment_args args) throws org.apache.thrift.TException { increment_result result = new increment_result(); try { result.success = iface.increment(args.table, args.increment); @@ -1477,16 +1522,20 @@ public class THBaseService { } } - private static class openScanner extends org.apache.thrift.ProcessFunction { + public static class openScanner extends org.apache.thrift.ProcessFunction { public openScanner() { super("openScanner"); } - protected openScanner_args getEmptyArgsInstance() { + public openScanner_args getEmptyArgsInstance() { return new openScanner_args(); } - protected openScanner_result getResult(I iface, openScanner_args args) throws org.apache.thrift.TException { + protected boolean isOneway() { + return false; + } + + public openScanner_result getResult(I iface, openScanner_args args) throws org.apache.thrift.TException { openScanner_result result = new openScanner_result(); try { result.success = iface.openScanner(args.table, args.scan); @@ -1498,16 +1547,20 @@ public class THBaseService { } } - private static class getScannerRows extends org.apache.thrift.ProcessFunction { + public static class getScannerRows extends org.apache.thrift.ProcessFunction { public getScannerRows() { super("getScannerRows"); } - protected getScannerRows_args getEmptyArgsInstance() { + public getScannerRows_args getEmptyArgsInstance() { return new getScannerRows_args(); } - protected getScannerRows_result getResult(I iface, getScannerRows_args args) throws org.apache.thrift.TException { + protected boolean isOneway() { + return false; + } + + public getScannerRows_result getResult(I iface, getScannerRows_args args) throws org.apache.thrift.TException { getScannerRows_result result = new getScannerRows_result(); try { result.success = iface.getScannerRows(args.scannerId, args.numRows); @@ -1520,16 +1573,20 @@ public class THBaseService { } } - private static class closeScanner extends org.apache.thrift.ProcessFunction { + public static class closeScanner extends org.apache.thrift.ProcessFunction { public closeScanner() { super("closeScanner"); } - protected closeScanner_args getEmptyArgsInstance() { + public closeScanner_args getEmptyArgsInstance() { return new closeScanner_args(); } - protected closeScanner_result getResult(I iface, closeScanner_args args) throws org.apache.thrift.TException { + protected boolean isOneway() { + return false; + } + + public closeScanner_result getResult(I iface, closeScanner_args args) throws org.apache.thrift.TException { closeScanner_result result = new closeScanner_result(); try { iface.closeScanner(args.scannerId); @@ -1542,16 +1599,20 @@ public class THBaseService { } } - private static class mutateRow extends org.apache.thrift.ProcessFunction { + public static class mutateRow extends org.apache.thrift.ProcessFunction { public mutateRow() { super("mutateRow"); } - protected mutateRow_args getEmptyArgsInstance() { + public mutateRow_args getEmptyArgsInstance() { return new mutateRow_args(); } - protected mutateRow_result getResult(I iface, mutateRow_args args) throws org.apache.thrift.TException { + protected boolean isOneway() { + return false; + } + + public mutateRow_result getResult(I iface, mutateRow_args args) throws org.apache.thrift.TException { mutateRow_result result = new mutateRow_result(); try { iface.mutateRow(args.table, args.rowMutations); @@ -1562,16 +1623,20 @@ public class THBaseService { } } - private static class getScannerResults extends org.apache.thrift.ProcessFunction { + public static class getScannerResults extends org.apache.thrift.ProcessFunction { public getScannerResults() { super("getScannerResults"); } - protected getScannerResults_args getEmptyArgsInstance() { + public getScannerResults_args getEmptyArgsInstance() { return new getScannerResults_args(); } - protected getScannerResults_result getResult(I iface, getScannerResults_args args) throws org.apache.thrift.TException { + protected boolean isOneway() { + return false; + } + + public getScannerResults_result getResult(I iface, getScannerResults_args args) throws org.apache.thrift.TException { getScannerResults_result result = new getScannerResults_result(); try { result.success = iface.getScannerResults(args.table, args.scan, args.numRows); @@ -1584,7 +1649,902 @@ public class THBaseService { } - public static class exists_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + public static class AsyncProcessor extends org.apache.thrift.TBaseAsyncProcessor { + private static final Logger LOGGER = LoggerFactory.getLogger(AsyncProcessor.class.getName()); + public AsyncProcessor(I iface) { + super(iface, getProcessMap(new HashMap>())); + } + + protected AsyncProcessor(I iface, Map> processMap) { + super(iface, getProcessMap(processMap)); + } + + private static Map> getProcessMap(Map> processMap) { + processMap.put("exists", new exists()); + processMap.put("get", new get()); + processMap.put("getMultiple", new getMultiple()); + processMap.put("put", new put()); + processMap.put("checkAndPut", new checkAndPut()); + processMap.put("putMultiple", new putMultiple()); + processMap.put("deleteSingle", new deleteSingle()); + processMap.put("deleteMultiple", new deleteMultiple()); + processMap.put("checkAndDelete", new checkAndDelete()); + processMap.put("increment", new increment()); + processMap.put("openScanner", new openScanner()); + processMap.put("getScannerRows", new getScannerRows()); + processMap.put("closeScanner", new closeScanner()); + processMap.put("mutateRow", new mutateRow()); + processMap.put("getScannerResults", new getScannerResults()); + return processMap; + } + + public static class exists extends org.apache.thrift.AsyncProcessFunction { + public exists() { + super("exists"); + } + + public exists_args getEmptyArgsInstance() { + return new exists_args(); + } + + public AsyncMethodCallback getResultHandler(final AsyncFrameBuffer fb, final int seqid) { + final org.apache.thrift.AsyncProcessFunction fcall = this; + return new AsyncMethodCallback() { + public void onComplete(Boolean o) { + exists_result result = new exists_result(); + result.success = o; + result.setSuccessIsSet(true); + try { + fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); + return; + } catch (Exception e) { + LOGGER.error("Exception writing to internal frame buffer", e); + } + fb.close(); + } + public void onError(Exception e) { + byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; + org.apache.thrift.TBase msg; + exists_result result = new exists_result(); + if (e instanceof TIOError) { + result.io = (TIOError) e; + result.setIoIsSet(true); + msg = result; + } + else + { + msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION; + msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage()); + } + try { + fcall.sendResponse(fb,msg,msgType,seqid); + return; + } catch (Exception ex) { + LOGGER.error("Exception writing to internal frame buffer", ex); + } + fb.close(); + } + }; + } + + protected boolean isOneway() { + return false; + } + + public void start(I iface, exists_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws TException { + iface.exists(args.table, args.get,resultHandler); + } + } + + public static class get extends org.apache.thrift.AsyncProcessFunction { + public get() { + super("get"); + } + + public get_args getEmptyArgsInstance() { + return new get_args(); + } + + public AsyncMethodCallback getResultHandler(final AsyncFrameBuffer fb, final int seqid) { + final org.apache.thrift.AsyncProcessFunction fcall = this; + return new AsyncMethodCallback() { + public void onComplete(TResult o) { + get_result result = new get_result(); + result.success = o; + try { + fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); + return; + } catch (Exception e) { + LOGGER.error("Exception writing to internal frame buffer", e); + } + fb.close(); + } + public void onError(Exception e) { + byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; + org.apache.thrift.TBase msg; + get_result result = new get_result(); + if (e instanceof TIOError) { + result.io = (TIOError) e; + result.setIoIsSet(true); + msg = result; + } + else + { + msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION; + msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage()); + } + try { + fcall.sendResponse(fb,msg,msgType,seqid); + return; + } catch (Exception ex) { + LOGGER.error("Exception writing to internal frame buffer", ex); + } + fb.close(); + } + }; + } + + protected boolean isOneway() { + return false; + } + + public void start(I iface, get_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws TException { + iface.get(args.table, args.get,resultHandler); + } + } + + public static class getMultiple extends org.apache.thrift.AsyncProcessFunction> { + public getMultiple() { + super("getMultiple"); + } + + public getMultiple_args getEmptyArgsInstance() { + return new getMultiple_args(); + } + + public AsyncMethodCallback> getResultHandler(final AsyncFrameBuffer fb, final int seqid) { + final org.apache.thrift.AsyncProcessFunction fcall = this; + return new AsyncMethodCallback>() { + public void onComplete(List o) { + getMultiple_result result = new getMultiple_result(); + result.success = o; + try { + fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); + return; + } catch (Exception e) { + LOGGER.error("Exception writing to internal frame buffer", e); + } + fb.close(); + } + public void onError(Exception e) { + byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; + org.apache.thrift.TBase msg; + getMultiple_result result = new getMultiple_result(); + if (e instanceof TIOError) { + result.io = (TIOError) e; + result.setIoIsSet(true); + msg = result; + } + else + { + msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION; + msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage()); + } + try { + fcall.sendResponse(fb,msg,msgType,seqid); + return; + } catch (Exception ex) { + LOGGER.error("Exception writing to internal frame buffer", ex); + } + fb.close(); + } + }; + } + + protected boolean isOneway() { + return false; + } + + public void start(I iface, getMultiple_args args, org.apache.thrift.async.AsyncMethodCallback> resultHandler) throws TException { + iface.getMultiple(args.table, args.gets,resultHandler); + } + } + + public static class put extends org.apache.thrift.AsyncProcessFunction { + public put() { + super("put"); + } + + public put_args getEmptyArgsInstance() { + return new put_args(); + } + + public AsyncMethodCallback getResultHandler(final AsyncFrameBuffer fb, final int seqid) { + final org.apache.thrift.AsyncProcessFunction fcall = this; + return new AsyncMethodCallback() { + public void onComplete(Void o) { + put_result result = new put_result(); + try { + fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); + return; + } catch (Exception e) { + LOGGER.error("Exception writing to internal frame buffer", e); + } + fb.close(); + } + public void onError(Exception e) { + byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; + org.apache.thrift.TBase msg; + put_result result = new put_result(); + if (e instanceof TIOError) { + result.io = (TIOError) e; + result.setIoIsSet(true); + msg = result; + } + else + { + msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION; + msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage()); + } + try { + fcall.sendResponse(fb,msg,msgType,seqid); + return; + } catch (Exception ex) { + LOGGER.error("Exception writing to internal frame buffer", ex); + } + fb.close(); + } + }; + } + + protected boolean isOneway() { + return false; + } + + public void start(I iface, put_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws TException { + iface.put(args.table, args.put,resultHandler); + } + } + + public static class checkAndPut extends org.apache.thrift.AsyncProcessFunction { + public checkAndPut() { + super("checkAndPut"); + } + + public checkAndPut_args getEmptyArgsInstance() { + return new checkAndPut_args(); + } + + public AsyncMethodCallback getResultHandler(final AsyncFrameBuffer fb, final int seqid) { + final org.apache.thrift.AsyncProcessFunction fcall = this; + return new AsyncMethodCallback() { + public void onComplete(Boolean o) { + checkAndPut_result result = new checkAndPut_result(); + result.success = o; + result.setSuccessIsSet(true); + try { + fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); + return; + } catch (Exception e) { + LOGGER.error("Exception writing to internal frame buffer", e); + } + fb.close(); + } + public void onError(Exception e) { + byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; + org.apache.thrift.TBase msg; + checkAndPut_result result = new checkAndPut_result(); + if (e instanceof TIOError) { + result.io = (TIOError) e; + result.setIoIsSet(true); + msg = result; + } + else + { + msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION; + msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage()); + } + try { + fcall.sendResponse(fb,msg,msgType,seqid); + return; + } catch (Exception ex) { + LOGGER.error("Exception writing to internal frame buffer", ex); + } + fb.close(); + } + }; + } + + protected boolean isOneway() { + return false; + } + + public void start(I iface, checkAndPut_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws TException { + iface.checkAndPut(args.table, args.row, args.family, args.qualifier, args.value, args.put,resultHandler); + } + } + + public static class putMultiple extends org.apache.thrift.AsyncProcessFunction { + public putMultiple() { + super("putMultiple"); + } + + public putMultiple_args getEmptyArgsInstance() { + return new putMultiple_args(); + } + + public AsyncMethodCallback getResultHandler(final AsyncFrameBuffer fb, final int seqid) { + final org.apache.thrift.AsyncProcessFunction fcall = this; + return new AsyncMethodCallback() { + public void onComplete(Void o) { + putMultiple_result result = new putMultiple_result(); + try { + fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); + return; + } catch (Exception e) { + LOGGER.error("Exception writing to internal frame buffer", e); + } + fb.close(); + } + public void onError(Exception e) { + byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; + org.apache.thrift.TBase msg; + putMultiple_result result = new putMultiple_result(); + if (e instanceof TIOError) { + result.io = (TIOError) e; + result.setIoIsSet(true); + msg = result; + } + else + { + msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION; + msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage()); + } + try { + fcall.sendResponse(fb,msg,msgType,seqid); + return; + } catch (Exception ex) { + LOGGER.error("Exception writing to internal frame buffer", ex); + } + fb.close(); + } + }; + } + + protected boolean isOneway() { + return false; + } + + public void start(I iface, putMultiple_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws TException { + iface.putMultiple(args.table, args.puts,resultHandler); + } + } + + public static class deleteSingle extends org.apache.thrift.AsyncProcessFunction { + public deleteSingle() { + super("deleteSingle"); + } + + public deleteSingle_args getEmptyArgsInstance() { + return new deleteSingle_args(); + } + + public AsyncMethodCallback getResultHandler(final AsyncFrameBuffer fb, final int seqid) { + final org.apache.thrift.AsyncProcessFunction fcall = this; + return new AsyncMethodCallback() { + public void onComplete(Void o) { + deleteSingle_result result = new deleteSingle_result(); + try { + fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); + return; + } catch (Exception e) { + LOGGER.error("Exception writing to internal frame buffer", e); + } + fb.close(); + } + public void onError(Exception e) { + byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; + org.apache.thrift.TBase msg; + deleteSingle_result result = new deleteSingle_result(); + if (e instanceof TIOError) { + result.io = (TIOError) e; + result.setIoIsSet(true); + msg = result; + } + else + { + msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION; + msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage()); + } + try { + fcall.sendResponse(fb,msg,msgType,seqid); + return; + } catch (Exception ex) { + LOGGER.error("Exception writing to internal frame buffer", ex); + } + fb.close(); + } + }; + } + + protected boolean isOneway() { + return false; + } + + public void start(I iface, deleteSingle_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws TException { + iface.deleteSingle(args.table, args.deleteSingle,resultHandler); + } + } + + public static class deleteMultiple extends org.apache.thrift.AsyncProcessFunction> { + public deleteMultiple() { + super("deleteMultiple"); + } + + public deleteMultiple_args getEmptyArgsInstance() { + return new deleteMultiple_args(); + } + + public AsyncMethodCallback> getResultHandler(final AsyncFrameBuffer fb, final int seqid) { + final org.apache.thrift.AsyncProcessFunction fcall = this; + return new AsyncMethodCallback>() { + public void onComplete(List o) { + deleteMultiple_result result = new deleteMultiple_result(); + result.success = o; + try { + fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); + return; + } catch (Exception e) { + LOGGER.error("Exception writing to internal frame buffer", e); + } + fb.close(); + } + public void onError(Exception e) { + byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; + org.apache.thrift.TBase msg; + deleteMultiple_result result = new deleteMultiple_result(); + if (e instanceof TIOError) { + result.io = (TIOError) e; + result.setIoIsSet(true); + msg = result; + } + else + { + msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION; + msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage()); + } + try { + fcall.sendResponse(fb,msg,msgType,seqid); + return; + } catch (Exception ex) { + LOGGER.error("Exception writing to internal frame buffer", ex); + } + fb.close(); + } + }; + } + + protected boolean isOneway() { + return false; + } + + public void start(I iface, deleteMultiple_args args, org.apache.thrift.async.AsyncMethodCallback> resultHandler) throws TException { + iface.deleteMultiple(args.table, args.deletes,resultHandler); + } + } + + public static class checkAndDelete extends org.apache.thrift.AsyncProcessFunction { + public checkAndDelete() { + super("checkAndDelete"); + } + + public checkAndDelete_args getEmptyArgsInstance() { + return new checkAndDelete_args(); + } + + public AsyncMethodCallback getResultHandler(final AsyncFrameBuffer fb, final int seqid) { + final org.apache.thrift.AsyncProcessFunction fcall = this; + return new AsyncMethodCallback() { + public void onComplete(Boolean o) { + checkAndDelete_result result = new checkAndDelete_result(); + result.success = o; + result.setSuccessIsSet(true); + try { + fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); + return; + } catch (Exception e) { + LOGGER.error("Exception writing to internal frame buffer", e); + } + fb.close(); + } + public void onError(Exception e) { + byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; + org.apache.thrift.TBase msg; + checkAndDelete_result result = new checkAndDelete_result(); + if (e instanceof TIOError) { + result.io = (TIOError) e; + result.setIoIsSet(true); + msg = result; + } + else + { + msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION; + msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage()); + } + try { + fcall.sendResponse(fb,msg,msgType,seqid); + return; + } catch (Exception ex) { + LOGGER.error("Exception writing to internal frame buffer", ex); + } + fb.close(); + } + }; + } + + protected boolean isOneway() { + return false; + } + + public void start(I iface, checkAndDelete_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws TException { + iface.checkAndDelete(args.table, args.row, args.family, args.qualifier, args.value, args.deleteSingle,resultHandler); + } + } + + public static class increment extends org.apache.thrift.AsyncProcessFunction { + public increment() { + super("increment"); + } + + public increment_args getEmptyArgsInstance() { + return new increment_args(); + } + + public AsyncMethodCallback getResultHandler(final AsyncFrameBuffer fb, final int seqid) { + final org.apache.thrift.AsyncProcessFunction fcall = this; + return new AsyncMethodCallback() { + public void onComplete(TResult o) { + increment_result result = new increment_result(); + result.success = o; + try { + fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); + return; + } catch (Exception e) { + LOGGER.error("Exception writing to internal frame buffer", e); + } + fb.close(); + } + public void onError(Exception e) { + byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; + org.apache.thrift.TBase msg; + increment_result result = new increment_result(); + if (e instanceof TIOError) { + result.io = (TIOError) e; + result.setIoIsSet(true); + msg = result; + } + else + { + msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION; + msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage()); + } + try { + fcall.sendResponse(fb,msg,msgType,seqid); + return; + } catch (Exception ex) { + LOGGER.error("Exception writing to internal frame buffer", ex); + } + fb.close(); + } + }; + } + + protected boolean isOneway() { + return false; + } + + public void start(I iface, increment_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws TException { + iface.increment(args.table, args.increment,resultHandler); + } + } + + public static class openScanner extends org.apache.thrift.AsyncProcessFunction { + public openScanner() { + super("openScanner"); + } + + public openScanner_args getEmptyArgsInstance() { + return new openScanner_args(); + } + + public AsyncMethodCallback getResultHandler(final AsyncFrameBuffer fb, final int seqid) { + final org.apache.thrift.AsyncProcessFunction fcall = this; + return new AsyncMethodCallback() { + public void onComplete(Integer o) { + openScanner_result result = new openScanner_result(); + result.success = o; + result.setSuccessIsSet(true); + try { + fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); + return; + } catch (Exception e) { + LOGGER.error("Exception writing to internal frame buffer", e); + } + fb.close(); + } + public void onError(Exception e) { + byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; + org.apache.thrift.TBase msg; + openScanner_result result = new openScanner_result(); + if (e instanceof TIOError) { + result.io = (TIOError) e; + result.setIoIsSet(true); + msg = result; + } + else + { + msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION; + msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage()); + } + try { + fcall.sendResponse(fb,msg,msgType,seqid); + return; + } catch (Exception ex) { + LOGGER.error("Exception writing to internal frame buffer", ex); + } + fb.close(); + } + }; + } + + protected boolean isOneway() { + return false; + } + + public void start(I iface, openScanner_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws TException { + iface.openScanner(args.table, args.scan,resultHandler); + } + } + + public static class getScannerRows extends org.apache.thrift.AsyncProcessFunction> { + public getScannerRows() { + super("getScannerRows"); + } + + public getScannerRows_args getEmptyArgsInstance() { + return new getScannerRows_args(); + } + + public AsyncMethodCallback> getResultHandler(final AsyncFrameBuffer fb, final int seqid) { + final org.apache.thrift.AsyncProcessFunction fcall = this; + return new AsyncMethodCallback>() { + public void onComplete(List o) { + getScannerRows_result result = new getScannerRows_result(); + result.success = o; + try { + fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); + return; + } catch (Exception e) { + LOGGER.error("Exception writing to internal frame buffer", e); + } + fb.close(); + } + public void onError(Exception e) { + byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; + org.apache.thrift.TBase msg; + getScannerRows_result result = new getScannerRows_result(); + if (e instanceof TIOError) { + result.io = (TIOError) e; + result.setIoIsSet(true); + msg = result; + } + else if (e instanceof TIllegalArgument) { + result.ia = (TIllegalArgument) e; + result.setIaIsSet(true); + msg = result; + } + else + { + msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION; + msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage()); + } + try { + fcall.sendResponse(fb,msg,msgType,seqid); + return; + } catch (Exception ex) { + LOGGER.error("Exception writing to internal frame buffer", ex); + } + fb.close(); + } + }; + } + + protected boolean isOneway() { + return false; + } + + public void start(I iface, getScannerRows_args args, org.apache.thrift.async.AsyncMethodCallback> resultHandler) throws TException { + iface.getScannerRows(args.scannerId, args.numRows,resultHandler); + } + } + + public static class closeScanner extends org.apache.thrift.AsyncProcessFunction { + public closeScanner() { + super("closeScanner"); + } + + public closeScanner_args getEmptyArgsInstance() { + return new closeScanner_args(); + } + + public AsyncMethodCallback getResultHandler(final AsyncFrameBuffer fb, final int seqid) { + final org.apache.thrift.AsyncProcessFunction fcall = this; + return new AsyncMethodCallback() { + public void onComplete(Void o) { + closeScanner_result result = new closeScanner_result(); + try { + fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); + return; + } catch (Exception e) { + LOGGER.error("Exception writing to internal frame buffer", e); + } + fb.close(); + } + public void onError(Exception e) { + byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; + org.apache.thrift.TBase msg; + closeScanner_result result = new closeScanner_result(); + if (e instanceof TIOError) { + result.io = (TIOError) e; + result.setIoIsSet(true); + msg = result; + } + else if (e instanceof TIllegalArgument) { + result.ia = (TIllegalArgument) e; + result.setIaIsSet(true); + msg = result; + } + else + { + msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION; + msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage()); + } + try { + fcall.sendResponse(fb,msg,msgType,seqid); + return; + } catch (Exception ex) { + LOGGER.error("Exception writing to internal frame buffer", ex); + } + fb.close(); + } + }; + } + + protected boolean isOneway() { + return false; + } + + public void start(I iface, closeScanner_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws TException { + iface.closeScanner(args.scannerId,resultHandler); + } + } + + public static class mutateRow extends org.apache.thrift.AsyncProcessFunction { + public mutateRow() { + super("mutateRow"); + } + + public mutateRow_args getEmptyArgsInstance() { + return new mutateRow_args(); + } + + public AsyncMethodCallback getResultHandler(final AsyncFrameBuffer fb, final int seqid) { + final org.apache.thrift.AsyncProcessFunction fcall = this; + return new AsyncMethodCallback() { + public void onComplete(Void o) { + mutateRow_result result = new mutateRow_result(); + try { + fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); + return; + } catch (Exception e) { + LOGGER.error("Exception writing to internal frame buffer", e); + } + fb.close(); + } + public void onError(Exception e) { + byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; + org.apache.thrift.TBase msg; + mutateRow_result result = new mutateRow_result(); + if (e instanceof TIOError) { + result.io = (TIOError) e; + result.setIoIsSet(true); + msg = result; + } + else + { + msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION; + msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage()); + } + try { + fcall.sendResponse(fb,msg,msgType,seqid); + return; + } catch (Exception ex) { + LOGGER.error("Exception writing to internal frame buffer", ex); + } + fb.close(); + } + }; + } + + protected boolean isOneway() { + return false; + } + + public void start(I iface, mutateRow_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws TException { + iface.mutateRow(args.table, args.rowMutations,resultHandler); + } + } + + public static class getScannerResults extends org.apache.thrift.AsyncProcessFunction> { + public getScannerResults() { + super("getScannerResults"); + } + + public getScannerResults_args getEmptyArgsInstance() { + return new getScannerResults_args(); + } + + public AsyncMethodCallback> getResultHandler(final AsyncFrameBuffer fb, final int seqid) { + final org.apache.thrift.AsyncProcessFunction fcall = this; + return new AsyncMethodCallback>() { + public void onComplete(List o) { + getScannerResults_result result = new getScannerResults_result(); + result.success = o; + try { + fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); + return; + } catch (Exception e) { + LOGGER.error("Exception writing to internal frame buffer", e); + } + fb.close(); + } + public void onError(Exception e) { + byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; + org.apache.thrift.TBase msg; + getScannerResults_result result = new getScannerResults_result(); + if (e instanceof TIOError) { + result.io = (TIOError) e; + result.setIoIsSet(true); + msg = result; + } + else + { + msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION; + msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage()); + } + try { + fcall.sendResponse(fb,msg,msgType,seqid); + return; + } catch (Exception ex) { + LOGGER.error("Exception writing to internal frame buffer", ex); + } + fb.close(); + } + }; + } + + protected boolean isOneway() { + return false; + } + + public void start(I iface, getScannerResults_args args, org.apache.thrift.async.AsyncMethodCallback> resultHandler) throws TException { + iface.getScannerResults(args.table, args.scan, args.numRows,resultHandler); + } + } + + } + + public static class exists_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("exists_args"); private static final org.apache.thrift.protocol.TField TABLE_FIELD_DESC = new org.apache.thrift.protocol.TField("table", org.apache.thrift.protocol.TType.STRING, (short)1); @@ -1876,30 +2836,30 @@ public class THBaseService { return 0; } + @Override public int compareTo(exists_args other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; - exists_args typedOther = (exists_args)other; - lastComparison = Boolean.valueOf(isSetTable()).compareTo(typedOther.isSetTable()); + lastComparison = Boolean.valueOf(isSetTable()).compareTo(other.isSetTable()); if (lastComparison != 0) { return lastComparison; } if (isSetTable()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.table, typedOther.table); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.table, other.table); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetGet()).compareTo(typedOther.isSetGet()); + lastComparison = Boolean.valueOf(isSetGet()).compareTo(other.isSetGet()); if (lastComparison != 0) { return lastComparison; } if (isSetGet()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.get, typedOther.get); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.get, other.get); if (lastComparison != 0) { return lastComparison; } @@ -1951,6 +2911,10 @@ public class THBaseService { if (get == null) { throw new org.apache.thrift.protocol.TProtocolException("Required field 'get' was not present! Struct: " + toString()); } + // check for sub-struct validity + if (get != null) { + get.validate(); + } } private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { @@ -2063,7 +3027,7 @@ public class THBaseService { } - public static class exists_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + public static class exists_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("exists_result"); private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.BOOL, (short)0); @@ -2141,7 +3105,7 @@ public class THBaseService { // isset id assignments private static final int __SUCCESS_ISSET_ID = 0; - private BitSet __isset_bit_vector = new BitSet(1); + 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); @@ -2170,8 +3134,7 @@ public class THBaseService { * Performs a deep copy on other. */ public exists_result(exists_result other) { - __isset_bit_vector.clear(); - __isset_bit_vector.or(other.__isset_bit_vector); + __isset_bitfield = other.__isset_bitfield; this.success = other.success; if (other.isSetIo()) { this.io = new TIOError(other.io); @@ -2200,16 +3163,16 @@ public class THBaseService { } public void unsetSuccess() { - __isset_bit_vector.clear(__SUCCESS_ISSET_ID); + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __SUCCESS_ISSET_ID); } /** Returns true if field success is set (has been assigned a value) and false otherwise */ public boolean isSetSuccess() { - return __isset_bit_vector.get(__SUCCESS_ISSET_ID); + return EncodingUtils.testBit(__isset_bitfield, __SUCCESS_ISSET_ID); } public void setSuccessIsSet(boolean value) { - __isset_bit_vector.set(__SUCCESS_ISSET_ID, value); + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __SUCCESS_ISSET_ID, value); } public TIOError getIo() { @@ -2323,30 +3286,30 @@ public class THBaseService { return 0; } + @Override public int compareTo(exists_result other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; - exists_result typedOther = (exists_result)other; - lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(typedOther.isSetSuccess()); + lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(other.isSetSuccess()); if (lastComparison != 0) { return lastComparison; } if (isSetSuccess()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, typedOther.success); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetIo()).compareTo(typedOther.isSetIo()); + lastComparison = Boolean.valueOf(isSetIo()).compareTo(other.isSetIo()); if (lastComparison != 0) { return lastComparison; } if (isSetIo()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.io, typedOther.io); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.io, other.io); if (lastComparison != 0) { return lastComparison; } @@ -2388,6 +3351,7 @@ public class THBaseService { 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 { @@ -2401,7 +3365,7 @@ public class THBaseService { 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_bit_vector = new BitSet(1); + __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); @@ -2458,9 +3422,11 @@ public class THBaseService { struct.validate(); oprot.writeStructBegin(STRUCT_DESC); - oprot.writeFieldBegin(SUCCESS_FIELD_DESC); - oprot.writeBool(struct.success); - oprot.writeFieldEnd(); + if (struct.isSetSuccess()) { + oprot.writeFieldBegin(SUCCESS_FIELD_DESC); + oprot.writeBool(struct.success); + oprot.writeFieldEnd(); + } if (struct.io != null) { oprot.writeFieldBegin(IO_FIELD_DESC); struct.io.write(oprot); @@ -2517,7 +3483,7 @@ public class THBaseService { } - public static class get_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + public static class get_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("get_args"); private static final org.apache.thrift.protocol.TField TABLE_FIELD_DESC = new org.apache.thrift.protocol.TField("table", org.apache.thrift.protocol.TType.STRING, (short)1); @@ -2809,30 +3775,30 @@ public class THBaseService { return 0; } + @Override public int compareTo(get_args other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; - get_args typedOther = (get_args)other; - lastComparison = Boolean.valueOf(isSetTable()).compareTo(typedOther.isSetTable()); + lastComparison = Boolean.valueOf(isSetTable()).compareTo(other.isSetTable()); if (lastComparison != 0) { return lastComparison; } if (isSetTable()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.table, typedOther.table); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.table, other.table); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetGet()).compareTo(typedOther.isSetGet()); + lastComparison = Boolean.valueOf(isSetGet()).compareTo(other.isSetGet()); if (lastComparison != 0) { return lastComparison; } if (isSetGet()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.get, typedOther.get); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.get, other.get); if (lastComparison != 0) { return lastComparison; } @@ -2884,6 +3850,10 @@ public class THBaseService { if (get == null) { throw new org.apache.thrift.protocol.TProtocolException("Required field 'get' was not present! Struct: " + toString()); } + // check for sub-struct validity + if (get != null) { + get.validate(); + } } private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { @@ -2996,7 +3966,7 @@ public class THBaseService { } - public static class get_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + public static class get_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("get_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); @@ -3253,30 +4223,30 @@ public class THBaseService { return 0; } + @Override public int compareTo(get_result other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; - get_result typedOther = (get_result)other; - lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(typedOther.isSetSuccess()); + lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(other.isSetSuccess()); if (lastComparison != 0) { return lastComparison; } if (isSetSuccess()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, typedOther.success); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetIo()).compareTo(typedOther.isSetIo()); + lastComparison = Boolean.valueOf(isSetIo()).compareTo(other.isSetIo()); if (lastComparison != 0) { return lastComparison; } if (isSetIo()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.io, typedOther.io); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.io, other.io); if (lastComparison != 0) { return lastComparison; } @@ -3322,6 +4292,10 @@ public class THBaseService { 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 { @@ -3453,7 +4427,7 @@ public class THBaseService { } - public static class getMultiple_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + public static class getMultiple_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getMultiple_args"); private static final org.apache.thrift.protocol.TField TABLE_FIELD_DESC = new org.apache.thrift.protocol.TField("table", org.apache.thrift.protocol.TType.STRING, (short)1); @@ -3579,7 +4553,7 @@ public class THBaseService { ; } if (other.isSetGets()) { - List __this__gets = new ArrayList(); + List __this__gets = new ArrayList(other.gets.size()); for (TGet other_element : other.gets) { __this__gets.add(new TGet(other_element)); } @@ -3773,30 +4747,30 @@ public class THBaseService { return 0; } + @Override public int compareTo(getMultiple_args other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; - getMultiple_args typedOther = (getMultiple_args)other; - lastComparison = Boolean.valueOf(isSetTable()).compareTo(typedOther.isSetTable()); + lastComparison = Boolean.valueOf(isSetTable()).compareTo(other.isSetTable()); if (lastComparison != 0) { return lastComparison; } if (isSetTable()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.table, typedOther.table); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.table, other.table); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetGets()).compareTo(typedOther.isSetGets()); + lastComparison = Boolean.valueOf(isSetGets()).compareTo(other.isSetGets()); if (lastComparison != 0) { return lastComparison; } if (isSetGets()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.gets, typedOther.gets); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.gets, other.gets); if (lastComparison != 0) { return lastComparison; } @@ -3848,6 +4822,7 @@ public class THBaseService { if (gets == null) { throw new org.apache.thrift.protocol.TProtocolException("Required field 'gets' was not present! Struct: " + toString()); } + // check for sub-struct validity } private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { @@ -3899,7 +4874,7 @@ public class THBaseService { struct.gets = new ArrayList(_list96.size); for (int _i97 = 0; _i97 < _list96.size; ++_i97) { - TGet _elem98; // required + TGet _elem98; _elem98 = new TGet(); _elem98.read(iprot); struct.gets.add(_elem98); @@ -3980,7 +4955,7 @@ public class THBaseService { struct.gets = new ArrayList(_list101.size); for (int _i102 = 0; _i102 < _list101.size; ++_i102) { - TGet _elem103; // required + TGet _elem103; _elem103 = new TGet(); _elem103.read(iprot); struct.gets.add(_elem103); @@ -3992,7 +4967,7 @@ public class THBaseService { } - public static class getMultiple_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + public static class getMultiple_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getMultiple_result"); private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.LIST, (short)0); @@ -4098,7 +5073,7 @@ public class THBaseService { */ public getMultiple_result(getMultiple_result other) { if (other.isSetSuccess()) { - List __this__success = new ArrayList(); + List __this__success = new ArrayList(other.success.size()); for (TResult other_element : other.success) { __this__success.add(new TResult(other_element)); } @@ -4269,30 +5244,30 @@ public class THBaseService { return 0; } + @Override public int compareTo(getMultiple_result other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; - getMultiple_result typedOther = (getMultiple_result)other; - lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(typedOther.isSetSuccess()); + lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(other.isSetSuccess()); if (lastComparison != 0) { return lastComparison; } if (isSetSuccess()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, typedOther.success); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetIo()).compareTo(typedOther.isSetIo()); + lastComparison = Boolean.valueOf(isSetIo()).compareTo(other.isSetIo()); if (lastComparison != 0) { return lastComparison; } if (isSetIo()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.io, typedOther.io); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.io, other.io); if (lastComparison != 0) { return lastComparison; } @@ -4338,6 +5313,7 @@ public class THBaseService { 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 { @@ -4381,7 +5357,7 @@ public class THBaseService { struct.success = new ArrayList(_list104.size); for (int _i105 = 0; _i105 < _list104.size; ++_i105) { - TResult _elem106; // required + TResult _elem106; _elem106 = new TResult(); _elem106.read(iprot); struct.success.add(_elem106); @@ -4483,7 +5459,7 @@ public class THBaseService { struct.success = new ArrayList(_list109.size); for (int _i110 = 0; _i110 < _list109.size; ++_i110) { - TResult _elem111; // required + TResult _elem111; _elem111 = new TResult(); _elem111.read(iprot); struct.success.add(_elem111); @@ -4501,7 +5477,7 @@ public class THBaseService { } - public static class put_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + public static class put_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("put_args"); private static final org.apache.thrift.protocol.TField TABLE_FIELD_DESC = new org.apache.thrift.protocol.TField("table", org.apache.thrift.protocol.TType.STRING, (short)1); @@ -4793,30 +5769,30 @@ public class THBaseService { return 0; } + @Override public int compareTo(put_args other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; - put_args typedOther = (put_args)other; - lastComparison = Boolean.valueOf(isSetTable()).compareTo(typedOther.isSetTable()); + lastComparison = Boolean.valueOf(isSetTable()).compareTo(other.isSetTable()); if (lastComparison != 0) { return lastComparison; } if (isSetTable()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.table, typedOther.table); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.table, other.table); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetPut()).compareTo(typedOther.isSetPut()); + lastComparison = Boolean.valueOf(isSetPut()).compareTo(other.isSetPut()); if (lastComparison != 0) { return lastComparison; } if (isSetPut()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.put, typedOther.put); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.put, other.put); if (lastComparison != 0) { return lastComparison; } @@ -4868,6 +5844,10 @@ public class THBaseService { if (put == null) { throw new org.apache.thrift.protocol.TProtocolException("Required field 'put' was not present! Struct: " + toString()); } + // check for sub-struct validity + if (put != null) { + put.validate(); + } } private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { @@ -4980,7 +5960,7 @@ public class THBaseService { } - public static class put_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + public static class put_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("put_result"); private static final org.apache.thrift.protocol.TField IO_FIELD_DESC = new org.apache.thrift.protocol.TField("io", org.apache.thrift.protocol.TType.STRUCT, (short)1); @@ -5178,20 +6158,20 @@ public class THBaseService { return 0; } + @Override public int compareTo(put_result other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; - put_result typedOther = (put_result)other; - lastComparison = Boolean.valueOf(isSetIo()).compareTo(typedOther.isSetIo()); + lastComparison = Boolean.valueOf(isSetIo()).compareTo(other.isSetIo()); if (lastComparison != 0) { return lastComparison; } if (isSetIo()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.io, typedOther.io); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.io, other.io); if (lastComparison != 0) { return lastComparison; } @@ -5229,6 +6209,7 @@ public class THBaseService { 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 { @@ -5335,7 +6316,7 @@ public class THBaseService { } - public static class checkAndPut_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + public static class checkAndPut_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("checkAndPut_args"); private static final org.apache.thrift.protocol.TField TABLE_FIELD_DESC = new org.apache.thrift.protocol.TField("table", org.apache.thrift.protocol.TType.STRING, (short)1); @@ -5963,70 +6944,70 @@ public class THBaseService { return 0; } + @Override public int compareTo(checkAndPut_args other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; - checkAndPut_args typedOther = (checkAndPut_args)other; - lastComparison = Boolean.valueOf(isSetTable()).compareTo(typedOther.isSetTable()); + lastComparison = Boolean.valueOf(isSetTable()).compareTo(other.isSetTable()); if (lastComparison != 0) { return lastComparison; } if (isSetTable()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.table, typedOther.table); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.table, other.table); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetRow()).compareTo(typedOther.isSetRow()); + lastComparison = Boolean.valueOf(isSetRow()).compareTo(other.isSetRow()); if (lastComparison != 0) { return lastComparison; } if (isSetRow()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.row, typedOther.row); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.row, other.row); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetFamily()).compareTo(typedOther.isSetFamily()); + lastComparison = Boolean.valueOf(isSetFamily()).compareTo(other.isSetFamily()); if (lastComparison != 0) { return lastComparison; } if (isSetFamily()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.family, typedOther.family); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.family, other.family); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetQualifier()).compareTo(typedOther.isSetQualifier()); + lastComparison = Boolean.valueOf(isSetQualifier()).compareTo(other.isSetQualifier()); if (lastComparison != 0) { return lastComparison; } if (isSetQualifier()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.qualifier, typedOther.qualifier); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.qualifier, other.qualifier); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetValue()).compareTo(typedOther.isSetValue()); + lastComparison = Boolean.valueOf(isSetValue()).compareTo(other.isSetValue()); if (lastComparison != 0) { return lastComparison; } if (isSetValue()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.value, typedOther.value); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.value, other.value); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetPut()).compareTo(typedOther.isSetPut()); + lastComparison = Boolean.valueOf(isSetPut()).compareTo(other.isSetPut()); if (lastComparison != 0) { return lastComparison; } if (isSetPut()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.put, typedOther.put); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.put, other.put); if (lastComparison != 0) { return lastComparison; } @@ -6119,6 +7100,10 @@ public class THBaseService { if (put == null) { throw new org.apache.thrift.protocol.TProtocolException("Required field 'put' was not present! Struct: " + toString()); } + // check for sub-struct validity + if (put != null) { + put.validate(); + } } private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { @@ -6305,7 +7290,7 @@ public class THBaseService { } - public static class checkAndPut_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + public static class checkAndPut_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("checkAndPut_result"); private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.BOOL, (short)0); @@ -6383,7 +7368,7 @@ public class THBaseService { // isset id assignments private static final int __SUCCESS_ISSET_ID = 0; - private BitSet __isset_bit_vector = new BitSet(1); + 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); @@ -6412,8 +7397,7 @@ public class THBaseService { * Performs a deep copy on other. */ public checkAndPut_result(checkAndPut_result other) { - __isset_bit_vector.clear(); - __isset_bit_vector.or(other.__isset_bit_vector); + __isset_bitfield = other.__isset_bitfield; this.success = other.success; if (other.isSetIo()) { this.io = new TIOError(other.io); @@ -6442,16 +7426,16 @@ public class THBaseService { } public void unsetSuccess() { - __isset_bit_vector.clear(__SUCCESS_ISSET_ID); + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __SUCCESS_ISSET_ID); } /** Returns true if field success is set (has been assigned a value) and false otherwise */ public boolean isSetSuccess() { - return __isset_bit_vector.get(__SUCCESS_ISSET_ID); + return EncodingUtils.testBit(__isset_bitfield, __SUCCESS_ISSET_ID); } public void setSuccessIsSet(boolean value) { - __isset_bit_vector.set(__SUCCESS_ISSET_ID, value); + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __SUCCESS_ISSET_ID, value); } public TIOError getIo() { @@ -6565,30 +7549,30 @@ public class THBaseService { return 0; } + @Override public int compareTo(checkAndPut_result other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; - checkAndPut_result typedOther = (checkAndPut_result)other; - lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(typedOther.isSetSuccess()); + lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(other.isSetSuccess()); if (lastComparison != 0) { return lastComparison; } if (isSetSuccess()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, typedOther.success); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetIo()).compareTo(typedOther.isSetIo()); + lastComparison = Boolean.valueOf(isSetIo()).compareTo(other.isSetIo()); if (lastComparison != 0) { return lastComparison; } if (isSetIo()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.io, typedOther.io); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.io, other.io); if (lastComparison != 0) { return lastComparison; } @@ -6630,6 +7614,7 @@ public class THBaseService { 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 { @@ -6643,7 +7628,7 @@ public class THBaseService { 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_bit_vector = new BitSet(1); + __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); @@ -6700,9 +7685,11 @@ public class THBaseService { struct.validate(); oprot.writeStructBegin(STRUCT_DESC); - oprot.writeFieldBegin(SUCCESS_FIELD_DESC); - oprot.writeBool(struct.success); - oprot.writeFieldEnd(); + if (struct.isSetSuccess()) { + oprot.writeFieldBegin(SUCCESS_FIELD_DESC); + oprot.writeBool(struct.success); + oprot.writeFieldEnd(); + } if (struct.io != null) { oprot.writeFieldBegin(IO_FIELD_DESC); struct.io.write(oprot); @@ -6759,7 +7746,7 @@ public class THBaseService { } - public static class putMultiple_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + public static class putMultiple_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("putMultiple_args"); private static final org.apache.thrift.protocol.TField TABLE_FIELD_DESC = new org.apache.thrift.protocol.TField("table", org.apache.thrift.protocol.TType.STRING, (short)1); @@ -6881,7 +7868,7 @@ public class THBaseService { ; } if (other.isSetPuts()) { - List __this__puts = new ArrayList(); + List __this__puts = new ArrayList(other.puts.size()); for (TPut other_element : other.puts) { __this__puts.add(new TPut(other_element)); } @@ -7071,30 +8058,30 @@ public class THBaseService { return 0; } + @Override public int compareTo(putMultiple_args other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; - putMultiple_args typedOther = (putMultiple_args)other; - lastComparison = Boolean.valueOf(isSetTable()).compareTo(typedOther.isSetTable()); + lastComparison = Boolean.valueOf(isSetTable()).compareTo(other.isSetTable()); if (lastComparison != 0) { return lastComparison; } if (isSetTable()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.table, typedOther.table); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.table, other.table); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetPuts()).compareTo(typedOther.isSetPuts()); + lastComparison = Boolean.valueOf(isSetPuts()).compareTo(other.isSetPuts()); if (lastComparison != 0) { return lastComparison; } if (isSetPuts()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.puts, typedOther.puts); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.puts, other.puts); if (lastComparison != 0) { return lastComparison; } @@ -7146,6 +8133,7 @@ public class THBaseService { if (puts == null) { throw new org.apache.thrift.protocol.TProtocolException("Required field 'puts' was not present! Struct: " + toString()); } + // check for sub-struct validity } private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { @@ -7197,7 +8185,7 @@ public class THBaseService { struct.puts = new ArrayList(_list112.size); for (int _i113 = 0; _i113 < _list112.size; ++_i113) { - TPut _elem114; // required + TPut _elem114; _elem114 = new TPut(); _elem114.read(iprot); struct.puts.add(_elem114); @@ -7278,7 +8266,7 @@ public class THBaseService { struct.puts = new ArrayList(_list117.size); for (int _i118 = 0; _i118 < _list117.size; ++_i118) { - TPut _elem119; // required + TPut _elem119; _elem119 = new TPut(); _elem119.read(iprot); struct.puts.add(_elem119); @@ -7290,7 +8278,7 @@ public class THBaseService { } - public static class putMultiple_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + public static class putMultiple_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("putMultiple_result"); private static final org.apache.thrift.protocol.TField IO_FIELD_DESC = new org.apache.thrift.protocol.TField("io", org.apache.thrift.protocol.TType.STRUCT, (short)1); @@ -7488,20 +8476,20 @@ public class THBaseService { return 0; } + @Override public int compareTo(putMultiple_result other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; - putMultiple_result typedOther = (putMultiple_result)other; - lastComparison = Boolean.valueOf(isSetIo()).compareTo(typedOther.isSetIo()); + lastComparison = Boolean.valueOf(isSetIo()).compareTo(other.isSetIo()); if (lastComparison != 0) { return lastComparison; } if (isSetIo()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.io, typedOther.io); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.io, other.io); if (lastComparison != 0) { return lastComparison; } @@ -7539,6 +8527,7 @@ public class THBaseService { 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 { @@ -7645,7 +8634,7 @@ public class THBaseService { } - public static class deleteSingle_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + public static class deleteSingle_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("deleteSingle_args"); private static final org.apache.thrift.protocol.TField TABLE_FIELD_DESC = new org.apache.thrift.protocol.TField("table", org.apache.thrift.protocol.TType.STRING, (short)1); @@ -7937,30 +8926,30 @@ public class THBaseService { return 0; } + @Override public int compareTo(deleteSingle_args other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; - deleteSingle_args typedOther = (deleteSingle_args)other; - lastComparison = Boolean.valueOf(isSetTable()).compareTo(typedOther.isSetTable()); + lastComparison = Boolean.valueOf(isSetTable()).compareTo(other.isSetTable()); if (lastComparison != 0) { return lastComparison; } if (isSetTable()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.table, typedOther.table); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.table, other.table); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetDeleteSingle()).compareTo(typedOther.isSetDeleteSingle()); + lastComparison = Boolean.valueOf(isSetDeleteSingle()).compareTo(other.isSetDeleteSingle()); if (lastComparison != 0) { return lastComparison; } if (isSetDeleteSingle()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.deleteSingle, typedOther.deleteSingle); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.deleteSingle, other.deleteSingle); if (lastComparison != 0) { return lastComparison; } @@ -8012,6 +9001,10 @@ public class THBaseService { if (deleteSingle == null) { throw new org.apache.thrift.protocol.TProtocolException("Required field 'deleteSingle' was not present! Struct: " + toString()); } + // check for sub-struct validity + if (deleteSingle != null) { + deleteSingle.validate(); + } } private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { @@ -8124,7 +9117,7 @@ public class THBaseService { } - public static class deleteSingle_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + public static class deleteSingle_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("deleteSingle_result"); private static final org.apache.thrift.protocol.TField IO_FIELD_DESC = new org.apache.thrift.protocol.TField("io", org.apache.thrift.protocol.TType.STRUCT, (short)1); @@ -8322,20 +9315,20 @@ public class THBaseService { return 0; } + @Override public int compareTo(deleteSingle_result other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; - deleteSingle_result typedOther = (deleteSingle_result)other; - lastComparison = Boolean.valueOf(isSetIo()).compareTo(typedOther.isSetIo()); + lastComparison = Boolean.valueOf(isSetIo()).compareTo(other.isSetIo()); if (lastComparison != 0) { return lastComparison; } if (isSetIo()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.io, typedOther.io); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.io, other.io); if (lastComparison != 0) { return lastComparison; } @@ -8373,6 +9366,7 @@ public class THBaseService { 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 { @@ -8479,7 +9473,7 @@ public class THBaseService { } - public static class deleteMultiple_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + public static class deleteMultiple_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("deleteMultiple_args"); private static final org.apache.thrift.protocol.TField TABLE_FIELD_DESC = new org.apache.thrift.protocol.TField("table", org.apache.thrift.protocol.TType.STRING, (short)1); @@ -8601,7 +9595,7 @@ public class THBaseService { ; } if (other.isSetDeletes()) { - List __this__deletes = new ArrayList(); + List __this__deletes = new ArrayList(other.deletes.size()); for (TDelete other_element : other.deletes) { __this__deletes.add(new TDelete(other_element)); } @@ -8791,30 +9785,30 @@ public class THBaseService { return 0; } + @Override public int compareTo(deleteMultiple_args other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; - deleteMultiple_args typedOther = (deleteMultiple_args)other; - lastComparison = Boolean.valueOf(isSetTable()).compareTo(typedOther.isSetTable()); + lastComparison = Boolean.valueOf(isSetTable()).compareTo(other.isSetTable()); if (lastComparison != 0) { return lastComparison; } if (isSetTable()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.table, typedOther.table); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.table, other.table); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetDeletes()).compareTo(typedOther.isSetDeletes()); + lastComparison = Boolean.valueOf(isSetDeletes()).compareTo(other.isSetDeletes()); if (lastComparison != 0) { return lastComparison; } if (isSetDeletes()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.deletes, typedOther.deletes); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.deletes, other.deletes); if (lastComparison != 0) { return lastComparison; } @@ -8866,6 +9860,7 @@ public class THBaseService { if (deletes == null) { throw new org.apache.thrift.protocol.TProtocolException("Required field 'deletes' was not present! Struct: " + toString()); } + // check for sub-struct validity } private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { @@ -8917,7 +9912,7 @@ public class THBaseService { struct.deletes = new ArrayList(_list120.size); for (int _i121 = 0; _i121 < _list120.size; ++_i121) { - TDelete _elem122; // required + TDelete _elem122; _elem122 = new TDelete(); _elem122.read(iprot); struct.deletes.add(_elem122); @@ -8998,7 +9993,7 @@ public class THBaseService { struct.deletes = new ArrayList(_list125.size); for (int _i126 = 0; _i126 < _list125.size; ++_i126) { - TDelete _elem127; // required + TDelete _elem127; _elem127 = new TDelete(); _elem127.read(iprot); struct.deletes.add(_elem127); @@ -9010,7 +10005,7 @@ public class THBaseService { } - public static class deleteMultiple_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + public static class deleteMultiple_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("deleteMultiple_result"); private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.LIST, (short)0); @@ -9116,7 +10111,7 @@ public class THBaseService { */ public deleteMultiple_result(deleteMultiple_result other) { if (other.isSetSuccess()) { - List __this__success = new ArrayList(); + List __this__success = new ArrayList(other.success.size()); for (TDelete other_element : other.success) { __this__success.add(new TDelete(other_element)); } @@ -9287,30 +10282,30 @@ public class THBaseService { return 0; } + @Override public int compareTo(deleteMultiple_result other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; - deleteMultiple_result typedOther = (deleteMultiple_result)other; - lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(typedOther.isSetSuccess()); + lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(other.isSetSuccess()); if (lastComparison != 0) { return lastComparison; } if (isSetSuccess()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, typedOther.success); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetIo()).compareTo(typedOther.isSetIo()); + lastComparison = Boolean.valueOf(isSetIo()).compareTo(other.isSetIo()); if (lastComparison != 0) { return lastComparison; } if (isSetIo()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.io, typedOther.io); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.io, other.io); if (lastComparison != 0) { return lastComparison; } @@ -9356,6 +10351,7 @@ public class THBaseService { 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 { @@ -9399,7 +10395,7 @@ public class THBaseService { struct.success = new ArrayList(_list128.size); for (int _i129 = 0; _i129 < _list128.size; ++_i129) { - TDelete _elem130; // required + TDelete _elem130; _elem130 = new TDelete(); _elem130.read(iprot); struct.success.add(_elem130); @@ -9501,7 +10497,7 @@ public class THBaseService { struct.success = new ArrayList(_list133.size); for (int _i134 = 0; _i134 < _list133.size; ++_i134) { - TDelete _elem135; // required + TDelete _elem135; _elem135 = new TDelete(); _elem135.read(iprot); struct.success.add(_elem135); @@ -9519,7 +10515,7 @@ public class THBaseService { } - public static class checkAndDelete_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + public static class checkAndDelete_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("checkAndDelete_args"); private static final org.apache.thrift.protocol.TField TABLE_FIELD_DESC = new org.apache.thrift.protocol.TField("table", org.apache.thrift.protocol.TType.STRING, (short)1); @@ -10147,70 +11143,70 @@ public class THBaseService { return 0; } + @Override public int compareTo(checkAndDelete_args other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; - checkAndDelete_args typedOther = (checkAndDelete_args)other; - lastComparison = Boolean.valueOf(isSetTable()).compareTo(typedOther.isSetTable()); + lastComparison = Boolean.valueOf(isSetTable()).compareTo(other.isSetTable()); if (lastComparison != 0) { return lastComparison; } if (isSetTable()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.table, typedOther.table); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.table, other.table); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetRow()).compareTo(typedOther.isSetRow()); + lastComparison = Boolean.valueOf(isSetRow()).compareTo(other.isSetRow()); if (lastComparison != 0) { return lastComparison; } if (isSetRow()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.row, typedOther.row); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.row, other.row); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetFamily()).compareTo(typedOther.isSetFamily()); + lastComparison = Boolean.valueOf(isSetFamily()).compareTo(other.isSetFamily()); if (lastComparison != 0) { return lastComparison; } if (isSetFamily()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.family, typedOther.family); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.family, other.family); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetQualifier()).compareTo(typedOther.isSetQualifier()); + lastComparison = Boolean.valueOf(isSetQualifier()).compareTo(other.isSetQualifier()); if (lastComparison != 0) { return lastComparison; } if (isSetQualifier()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.qualifier, typedOther.qualifier); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.qualifier, other.qualifier); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetValue()).compareTo(typedOther.isSetValue()); + lastComparison = Boolean.valueOf(isSetValue()).compareTo(other.isSetValue()); if (lastComparison != 0) { return lastComparison; } if (isSetValue()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.value, typedOther.value); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.value, other.value); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetDeleteSingle()).compareTo(typedOther.isSetDeleteSingle()); + lastComparison = Boolean.valueOf(isSetDeleteSingle()).compareTo(other.isSetDeleteSingle()); if (lastComparison != 0) { return lastComparison; } if (isSetDeleteSingle()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.deleteSingle, typedOther.deleteSingle); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.deleteSingle, other.deleteSingle); if (lastComparison != 0) { return lastComparison; } @@ -10303,6 +11299,10 @@ public class THBaseService { if (deleteSingle == null) { throw new org.apache.thrift.protocol.TProtocolException("Required field 'deleteSingle' was not present! Struct: " + toString()); } + // check for sub-struct validity + if (deleteSingle != null) { + deleteSingle.validate(); + } } private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { @@ -10489,7 +11489,7 @@ public class THBaseService { } - public static class checkAndDelete_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + public static class checkAndDelete_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("checkAndDelete_result"); private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.BOOL, (short)0); @@ -10567,7 +11567,7 @@ public class THBaseService { // isset id assignments private static final int __SUCCESS_ISSET_ID = 0; - private BitSet __isset_bit_vector = new BitSet(1); + 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); @@ -10596,8 +11596,7 @@ public class THBaseService { * Performs a deep copy on other. */ public checkAndDelete_result(checkAndDelete_result other) { - __isset_bit_vector.clear(); - __isset_bit_vector.or(other.__isset_bit_vector); + __isset_bitfield = other.__isset_bitfield; this.success = other.success; if (other.isSetIo()) { this.io = new TIOError(other.io); @@ -10626,16 +11625,16 @@ public class THBaseService { } public void unsetSuccess() { - __isset_bit_vector.clear(__SUCCESS_ISSET_ID); + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __SUCCESS_ISSET_ID); } /** Returns true if field success is set (has been assigned a value) and false otherwise */ public boolean isSetSuccess() { - return __isset_bit_vector.get(__SUCCESS_ISSET_ID); + return EncodingUtils.testBit(__isset_bitfield, __SUCCESS_ISSET_ID); } public void setSuccessIsSet(boolean value) { - __isset_bit_vector.set(__SUCCESS_ISSET_ID, value); + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __SUCCESS_ISSET_ID, value); } public TIOError getIo() { @@ -10749,30 +11748,30 @@ public class THBaseService { return 0; } + @Override public int compareTo(checkAndDelete_result other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; - checkAndDelete_result typedOther = (checkAndDelete_result)other; - lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(typedOther.isSetSuccess()); + lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(other.isSetSuccess()); if (lastComparison != 0) { return lastComparison; } if (isSetSuccess()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, typedOther.success); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetIo()).compareTo(typedOther.isSetIo()); + lastComparison = Boolean.valueOf(isSetIo()).compareTo(other.isSetIo()); if (lastComparison != 0) { return lastComparison; } if (isSetIo()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.io, typedOther.io); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.io, other.io); if (lastComparison != 0) { return lastComparison; } @@ -10814,6 +11813,7 @@ public class THBaseService { 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 { @@ -10827,7 +11827,7 @@ public class THBaseService { 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_bit_vector = new BitSet(1); + __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); @@ -10884,9 +11884,11 @@ public class THBaseService { struct.validate(); oprot.writeStructBegin(STRUCT_DESC); - oprot.writeFieldBegin(SUCCESS_FIELD_DESC); - oprot.writeBool(struct.success); - oprot.writeFieldEnd(); + if (struct.isSetSuccess()) { + oprot.writeFieldBegin(SUCCESS_FIELD_DESC); + oprot.writeBool(struct.success); + oprot.writeFieldEnd(); + } if (struct.io != null) { oprot.writeFieldBegin(IO_FIELD_DESC); struct.io.write(oprot); @@ -10943,7 +11945,7 @@ public class THBaseService { } - public static class increment_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + public static class increment_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("increment_args"); private static final org.apache.thrift.protocol.TField TABLE_FIELD_DESC = new org.apache.thrift.protocol.TField("table", org.apache.thrift.protocol.TType.STRING, (short)1); @@ -11235,30 +12237,30 @@ public class THBaseService { return 0; } + @Override public int compareTo(increment_args other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; - increment_args typedOther = (increment_args)other; - lastComparison = Boolean.valueOf(isSetTable()).compareTo(typedOther.isSetTable()); + lastComparison = Boolean.valueOf(isSetTable()).compareTo(other.isSetTable()); if (lastComparison != 0) { return lastComparison; } if (isSetTable()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.table, typedOther.table); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.table, other.table); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetIncrement()).compareTo(typedOther.isSetIncrement()); + lastComparison = Boolean.valueOf(isSetIncrement()).compareTo(other.isSetIncrement()); if (lastComparison != 0) { return lastComparison; } if (isSetIncrement()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.increment, typedOther.increment); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.increment, other.increment); if (lastComparison != 0) { return lastComparison; } @@ -11310,6 +12312,10 @@ public class THBaseService { if (increment == null) { throw new org.apache.thrift.protocol.TProtocolException("Required field 'increment' was not present! Struct: " + toString()); } + // check for sub-struct validity + if (increment != null) { + increment.validate(); + } } private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { @@ -11422,7 +12428,7 @@ public class THBaseService { } - public static class increment_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + public static class increment_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("increment_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); @@ -11679,30 +12685,30 @@ public class THBaseService { return 0; } + @Override public int compareTo(increment_result other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; - increment_result typedOther = (increment_result)other; - lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(typedOther.isSetSuccess()); + lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(other.isSetSuccess()); if (lastComparison != 0) { return lastComparison; } if (isSetSuccess()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, typedOther.success); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetIo()).compareTo(typedOther.isSetIo()); + lastComparison = Boolean.valueOf(isSetIo()).compareTo(other.isSetIo()); if (lastComparison != 0) { return lastComparison; } if (isSetIo()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.io, typedOther.io); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.io, other.io); if (lastComparison != 0) { return lastComparison; } @@ -11748,6 +12754,10 @@ public class THBaseService { 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 { @@ -11879,7 +12889,7 @@ public class THBaseService { } - public static class openScanner_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + public static class openScanner_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("openScanner_args"); private static final org.apache.thrift.protocol.TField TABLE_FIELD_DESC = new org.apache.thrift.protocol.TField("table", org.apache.thrift.protocol.TType.STRING, (short)1); @@ -12171,30 +13181,30 @@ public class THBaseService { return 0; } + @Override public int compareTo(openScanner_args other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; - openScanner_args typedOther = (openScanner_args)other; - lastComparison = Boolean.valueOf(isSetTable()).compareTo(typedOther.isSetTable()); + lastComparison = Boolean.valueOf(isSetTable()).compareTo(other.isSetTable()); if (lastComparison != 0) { return lastComparison; } if (isSetTable()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.table, typedOther.table); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.table, other.table); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetScan()).compareTo(typedOther.isSetScan()); + lastComparison = Boolean.valueOf(isSetScan()).compareTo(other.isSetScan()); if (lastComparison != 0) { return lastComparison; } if (isSetScan()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.scan, typedOther.scan); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.scan, other.scan); if (lastComparison != 0) { return lastComparison; } @@ -12246,6 +13256,10 @@ public class THBaseService { if (scan == null) { throw new org.apache.thrift.protocol.TProtocolException("Required field 'scan' was not present! Struct: " + toString()); } + // check for sub-struct validity + if (scan != null) { + scan.validate(); + } } private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { @@ -12358,7 +13372,7 @@ public class THBaseService { } - public static class openScanner_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + public static class openScanner_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("openScanner_result"); private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.I32, (short)0); @@ -12436,7 +13450,7 @@ public class THBaseService { // isset id assignments private static final int __SUCCESS_ISSET_ID = 0; - private BitSet __isset_bit_vector = new BitSet(1); + 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); @@ -12465,8 +13479,7 @@ public class THBaseService { * Performs a deep copy on other. */ public openScanner_result(openScanner_result other) { - __isset_bit_vector.clear(); - __isset_bit_vector.or(other.__isset_bit_vector); + __isset_bitfield = other.__isset_bitfield; this.success = other.success; if (other.isSetIo()) { this.io = new TIOError(other.io); @@ -12495,16 +13508,16 @@ public class THBaseService { } public void unsetSuccess() { - __isset_bit_vector.clear(__SUCCESS_ISSET_ID); + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __SUCCESS_ISSET_ID); } /** Returns true if field success is set (has been assigned a value) and false otherwise */ public boolean isSetSuccess() { - return __isset_bit_vector.get(__SUCCESS_ISSET_ID); + return EncodingUtils.testBit(__isset_bitfield, __SUCCESS_ISSET_ID); } public void setSuccessIsSet(boolean value) { - __isset_bit_vector.set(__SUCCESS_ISSET_ID, value); + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __SUCCESS_ISSET_ID, value); } public TIOError getIo() { @@ -12618,30 +13631,30 @@ public class THBaseService { return 0; } + @Override public int compareTo(openScanner_result other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; - openScanner_result typedOther = (openScanner_result)other; - lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(typedOther.isSetSuccess()); + lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(other.isSetSuccess()); if (lastComparison != 0) { return lastComparison; } if (isSetSuccess()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, typedOther.success); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetIo()).compareTo(typedOther.isSetIo()); + lastComparison = Boolean.valueOf(isSetIo()).compareTo(other.isSetIo()); if (lastComparison != 0) { return lastComparison; } if (isSetIo()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.io, typedOther.io); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.io, other.io); if (lastComparison != 0) { return lastComparison; } @@ -12683,6 +13696,7 @@ public class THBaseService { 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 { @@ -12696,7 +13710,7 @@ public class THBaseService { 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_bit_vector = new BitSet(1); + __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); @@ -12753,9 +13767,11 @@ public class THBaseService { struct.validate(); oprot.writeStructBegin(STRUCT_DESC); - oprot.writeFieldBegin(SUCCESS_FIELD_DESC); - oprot.writeI32(struct.success); - oprot.writeFieldEnd(); + if (struct.isSetSuccess()) { + oprot.writeFieldBegin(SUCCESS_FIELD_DESC); + oprot.writeI32(struct.success); + oprot.writeFieldEnd(); + } if (struct.io != null) { oprot.writeFieldBegin(IO_FIELD_DESC); struct.io.write(oprot); @@ -12812,7 +13828,7 @@ public class THBaseService { } - public static class getScannerRows_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + public static class getScannerRows_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getScannerRows_args"); private static final org.apache.thrift.protocol.TField SCANNER_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("scannerId", org.apache.thrift.protocol.TType.I32, (short)1); @@ -12903,7 +13919,7 @@ public class THBaseService { // isset id assignments private static final int __SCANNERID_ISSET_ID = 0; private static final int __NUMROWS_ISSET_ID = 1; - private BitSet __isset_bit_vector = new BitSet(2); + 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); @@ -12935,8 +13951,7 @@ public class THBaseService { * Performs a deep copy on other. */ public getScannerRows_args(getScannerRows_args other) { - __isset_bit_vector.clear(); - __isset_bit_vector.or(other.__isset_bit_vector); + __isset_bitfield = other.__isset_bitfield; this.scannerId = other.scannerId; this.numRows = other.numRows; } @@ -12970,16 +13985,16 @@ public class THBaseService { } public void unsetScannerId() { - __isset_bit_vector.clear(__SCANNERID_ISSET_ID); + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __SCANNERID_ISSET_ID); } /** Returns true if field scannerId is set (has been assigned a value) and false otherwise */ public boolean isSetScannerId() { - return __isset_bit_vector.get(__SCANNERID_ISSET_ID); + return EncodingUtils.testBit(__isset_bitfield, __SCANNERID_ISSET_ID); } public void setScannerIdIsSet(boolean value) { - __isset_bit_vector.set(__SCANNERID_ISSET_ID, value); + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __SCANNERID_ISSET_ID, value); } /** @@ -12999,16 +14014,16 @@ public class THBaseService { } public void unsetNumRows() { - __isset_bit_vector.clear(__NUMROWS_ISSET_ID); + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __NUMROWS_ISSET_ID); } /** Returns true if field numRows is set (has been assigned a value) and false otherwise */ public boolean isSetNumRows() { - return __isset_bit_vector.get(__NUMROWS_ISSET_ID); + return EncodingUtils.testBit(__isset_bitfield, __NUMROWS_ISSET_ID); } public void setNumRowsIsSet(boolean value) { - __isset_bit_vector.set(__NUMROWS_ISSET_ID, value); + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __NUMROWS_ISSET_ID, value); } public void setFieldValue(_Fields field, Object value) { @@ -13098,30 +14113,30 @@ public class THBaseService { return 0; } + @Override public int compareTo(getScannerRows_args other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; - getScannerRows_args typedOther = (getScannerRows_args)other; - lastComparison = Boolean.valueOf(isSetScannerId()).compareTo(typedOther.isSetScannerId()); + lastComparison = Boolean.valueOf(isSetScannerId()).compareTo(other.isSetScannerId()); if (lastComparison != 0) { return lastComparison; } if (isSetScannerId()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.scannerId, typedOther.scannerId); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.scannerId, other.scannerId); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetNumRows()).compareTo(typedOther.isSetNumRows()); + lastComparison = Boolean.valueOf(isSetNumRows()).compareTo(other.isSetNumRows()); if (lastComparison != 0) { return lastComparison; } if (isSetNumRows()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.numRows, typedOther.numRows); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.numRows, other.numRows); if (lastComparison != 0) { return lastComparison; } @@ -13160,6 +14175,7 @@ public class THBaseService { public void validate() throws org.apache.thrift.TException { // check for required fields // alas, we cannot check 'scannerId' because it's a primitive and you chose the non-beans generator. + // check for sub-struct validity } private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { @@ -13173,7 +14189,7 @@ public class THBaseService { 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_bit_vector = new BitSet(1); + __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); @@ -13281,7 +14297,7 @@ public class THBaseService { } - public static class getScannerRows_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + public static class getScannerRows_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getScannerRows_result"); private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.LIST, (short)0); @@ -13402,7 +14418,7 @@ public class THBaseService { */ public getScannerRows_result(getScannerRows_result other) { if (other.isSetSuccess()) { - List __this__success = new ArrayList(); + List __this__success = new ArrayList(other.success.size()); for (TResult other_element : other.success) { __this__success.add(new TResult(other_element)); } @@ -13629,40 +14645,40 @@ public class THBaseService { return 0; } + @Override public int compareTo(getScannerRows_result other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; - getScannerRows_result typedOther = (getScannerRows_result)other; - lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(typedOther.isSetSuccess()); + lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(other.isSetSuccess()); if (lastComparison != 0) { return lastComparison; } if (isSetSuccess()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, typedOther.success); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetIo()).compareTo(typedOther.isSetIo()); + lastComparison = Boolean.valueOf(isSetIo()).compareTo(other.isSetIo()); if (lastComparison != 0) { return lastComparison; } if (isSetIo()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.io, typedOther.io); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.io, other.io); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetIa()).compareTo(typedOther.isSetIa()); + lastComparison = Boolean.valueOf(isSetIa()).compareTo(other.isSetIa()); if (lastComparison != 0) { return lastComparison; } if (isSetIa()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.ia, typedOther.ia); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.ia, other.ia); if (lastComparison != 0) { return lastComparison; } @@ -13716,6 +14732,7 @@ public class THBaseService { 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 { @@ -13759,7 +14776,7 @@ public class THBaseService { struct.success = new ArrayList(_list136.size); for (int _i137 = 0; _i137 < _list136.size; ++_i137) { - TResult _elem138; // required + TResult _elem138; _elem138 = new TResult(); _elem138.read(iprot); struct.success.add(_elem138); @@ -13881,7 +14898,7 @@ public class THBaseService { struct.success = new ArrayList(_list141.size); for (int _i142 = 0; _i142 < _list141.size; ++_i142) { - TResult _elem143; // required + TResult _elem143; _elem143 = new TResult(); _elem143.read(iprot); struct.success.add(_elem143); @@ -13904,7 +14921,7 @@ public class THBaseService { } - public static class closeScanner_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + public static class closeScanner_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("closeScanner_args"); private static final org.apache.thrift.protocol.TField SCANNER_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("scannerId", org.apache.thrift.protocol.TType.I32, (short)1); @@ -13983,7 +15000,7 @@ public class THBaseService { // isset id assignments private static final int __SCANNERID_ISSET_ID = 0; - private BitSet __isset_bit_vector = new BitSet(1); + 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); @@ -14008,8 +15025,7 @@ public class THBaseService { * Performs a deep copy on other. */ public closeScanner_args(closeScanner_args other) { - __isset_bit_vector.clear(); - __isset_bit_vector.or(other.__isset_bit_vector); + __isset_bitfield = other.__isset_bitfield; this.scannerId = other.scannerId; } @@ -14040,16 +15056,16 @@ public class THBaseService { } public void unsetScannerId() { - __isset_bit_vector.clear(__SCANNERID_ISSET_ID); + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __SCANNERID_ISSET_ID); } /** Returns true if field scannerId is set (has been assigned a value) and false otherwise */ public boolean isSetScannerId() { - return __isset_bit_vector.get(__SCANNERID_ISSET_ID); + return EncodingUtils.testBit(__isset_bitfield, __SCANNERID_ISSET_ID); } public void setScannerIdIsSet(boolean value) { - __isset_bit_vector.set(__SCANNERID_ISSET_ID, value); + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __SCANNERID_ISSET_ID, value); } public void setFieldValue(_Fields field, Object value) { @@ -14117,20 +15133,20 @@ public class THBaseService { return 0; } + @Override public int compareTo(closeScanner_args other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; - closeScanner_args typedOther = (closeScanner_args)other; - lastComparison = Boolean.valueOf(isSetScannerId()).compareTo(typedOther.isSetScannerId()); + lastComparison = Boolean.valueOf(isSetScannerId()).compareTo(other.isSetScannerId()); if (lastComparison != 0) { return lastComparison; } if (isSetScannerId()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.scannerId, typedOther.scannerId); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.scannerId, other.scannerId); if (lastComparison != 0) { return lastComparison; } @@ -14165,6 +15181,7 @@ public class THBaseService { public void validate() throws org.apache.thrift.TException { // check for required fields // alas, we cannot check 'scannerId' because it's a primitive and you chose the non-beans generator. + // check for sub-struct validity } private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { @@ -14178,7 +15195,7 @@ public class THBaseService { 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_bit_vector = new BitSet(1); + __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); @@ -14262,7 +15279,7 @@ public class THBaseService { } - public static class closeScanner_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + public static class closeScanner_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("closeScanner_result"); private static final org.apache.thrift.protocol.TField IO_FIELD_DESC = new org.apache.thrift.protocol.TField("io", org.apache.thrift.protocol.TType.STRUCT, (short)1); @@ -14531,30 +15548,30 @@ public class THBaseService { return 0; } + @Override public int compareTo(closeScanner_result other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; - closeScanner_result typedOther = (closeScanner_result)other; - lastComparison = Boolean.valueOf(isSetIo()).compareTo(typedOther.isSetIo()); + lastComparison = Boolean.valueOf(isSetIo()).compareTo(other.isSetIo()); if (lastComparison != 0) { return lastComparison; } if (isSetIo()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.io, typedOther.io); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.io, other.io); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetIa()).compareTo(typedOther.isSetIa()); + lastComparison = Boolean.valueOf(isSetIa()).compareTo(other.isSetIa()); if (lastComparison != 0) { return lastComparison; } if (isSetIa()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.ia, typedOther.ia); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.ia, other.ia); if (lastComparison != 0) { return lastComparison; } @@ -14600,6 +15617,7 @@ public class THBaseService { 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 { @@ -14731,7 +15749,7 @@ public class THBaseService { } - public static class mutateRow_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + public static class mutateRow_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("mutateRow_args"); private static final org.apache.thrift.protocol.TField TABLE_FIELD_DESC = new org.apache.thrift.protocol.TField("table", org.apache.thrift.protocol.TType.STRING, (short)1); @@ -15023,30 +16041,30 @@ public class THBaseService { return 0; } + @Override public int compareTo(mutateRow_args other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; - mutateRow_args typedOther = (mutateRow_args)other; - lastComparison = Boolean.valueOf(isSetTable()).compareTo(typedOther.isSetTable()); + lastComparison = Boolean.valueOf(isSetTable()).compareTo(other.isSetTable()); if (lastComparison != 0) { return lastComparison; } if (isSetTable()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.table, typedOther.table); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.table, other.table); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetRowMutations()).compareTo(typedOther.isSetRowMutations()); + lastComparison = Boolean.valueOf(isSetRowMutations()).compareTo(other.isSetRowMutations()); if (lastComparison != 0) { return lastComparison; } if (isSetRowMutations()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.rowMutations, typedOther.rowMutations); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.rowMutations, other.rowMutations); if (lastComparison != 0) { return lastComparison; } @@ -15098,6 +16116,10 @@ public class THBaseService { if (rowMutations == null) { throw new org.apache.thrift.protocol.TProtocolException("Required field 'rowMutations' was not present! Struct: " + toString()); } + // check for sub-struct validity + if (rowMutations != null) { + rowMutations.validate(); + } } private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { @@ -15210,7 +16232,7 @@ public class THBaseService { } - public static class mutateRow_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + public static class mutateRow_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("mutateRow_result"); private static final org.apache.thrift.protocol.TField IO_FIELD_DESC = new org.apache.thrift.protocol.TField("io", org.apache.thrift.protocol.TType.STRUCT, (short)1); @@ -15408,20 +16430,20 @@ public class THBaseService { return 0; } + @Override public int compareTo(mutateRow_result other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; - mutateRow_result typedOther = (mutateRow_result)other; - lastComparison = Boolean.valueOf(isSetIo()).compareTo(typedOther.isSetIo()); + lastComparison = Boolean.valueOf(isSetIo()).compareTo(other.isSetIo()); if (lastComparison != 0) { return lastComparison; } if (isSetIo()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.io, typedOther.io); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.io, other.io); if (lastComparison != 0) { return lastComparison; } @@ -15459,6 +16481,7 @@ public class THBaseService { 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 { @@ -15565,7 +16588,7 @@ public class THBaseService { } - public static class getScannerResults_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + public static class getScannerResults_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getScannerResults_args"); private static final org.apache.thrift.protocol.TField TABLE_FIELD_DESC = new org.apache.thrift.protocol.TField("table", org.apache.thrift.protocol.TType.STRING, (short)1); @@ -15666,7 +16689,7 @@ public class THBaseService { // isset id assignments private static final int __NUMROWS_ISSET_ID = 0; - private BitSet __isset_bit_vector = new BitSet(1); + 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); @@ -15701,8 +16724,7 @@ public class THBaseService { * Performs a deep copy on other. */ public getScannerResults_args(getScannerResults_args other) { - __isset_bit_vector.clear(); - __isset_bit_vector.or(other.__isset_bit_vector); + __isset_bitfield = other.__isset_bitfield; if (other.isSetTable()) { this.table = org.apache.thrift.TBaseHelper.copyBinary(other.table); ; @@ -15812,16 +16834,16 @@ public class THBaseService { } public void unsetNumRows() { - __isset_bit_vector.clear(__NUMROWS_ISSET_ID); + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __NUMROWS_ISSET_ID); } /** Returns true if field numRows is set (has been assigned a value) and false otherwise */ public boolean isSetNumRows() { - return __isset_bit_vector.get(__NUMROWS_ISSET_ID); + return EncodingUtils.testBit(__isset_bitfield, __NUMROWS_ISSET_ID); } public void setNumRowsIsSet(boolean value) { - __isset_bit_vector.set(__NUMROWS_ISSET_ID, value); + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __NUMROWS_ISSET_ID, value); } public void setFieldValue(_Fields field, Object value) { @@ -15933,40 +16955,40 @@ public class THBaseService { return 0; } + @Override public int compareTo(getScannerResults_args other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; - getScannerResults_args typedOther = (getScannerResults_args)other; - lastComparison = Boolean.valueOf(isSetTable()).compareTo(typedOther.isSetTable()); + lastComparison = Boolean.valueOf(isSetTable()).compareTo(other.isSetTable()); if (lastComparison != 0) { return lastComparison; } if (isSetTable()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.table, typedOther.table); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.table, other.table); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetScan()).compareTo(typedOther.isSetScan()); + lastComparison = Boolean.valueOf(isSetScan()).compareTo(other.isSetScan()); if (lastComparison != 0) { return lastComparison; } if (isSetScan()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.scan, typedOther.scan); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.scan, other.scan); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetNumRows()).compareTo(typedOther.isSetNumRows()); + lastComparison = Boolean.valueOf(isSetNumRows()).compareTo(other.isSetNumRows()); if (lastComparison != 0) { return lastComparison; } if (isSetNumRows()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.numRows, typedOther.numRows); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.numRows, other.numRows); if (lastComparison != 0) { return lastComparison; } @@ -16022,6 +17044,10 @@ public class THBaseService { if (scan == null) { throw new org.apache.thrift.protocol.TProtocolException("Required field 'scan' was not present! Struct: " + toString()); } + // check for sub-struct validity + if (scan != null) { + scan.validate(); + } } private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { @@ -16035,7 +17061,7 @@ public class THBaseService { 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_bit_vector = new BitSet(1); + __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); @@ -16160,7 +17186,7 @@ public class THBaseService { } - public static class getScannerResults_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + public static class getScannerResults_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getScannerResults_result"); private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.LIST, (short)0); @@ -16266,7 +17292,7 @@ public class THBaseService { */ public getScannerResults_result(getScannerResults_result other) { if (other.isSetSuccess()) { - List __this__success = new ArrayList(); + List __this__success = new ArrayList(other.success.size()); for (TResult other_element : other.success) { __this__success.add(new TResult(other_element)); } @@ -16437,30 +17463,30 @@ public class THBaseService { return 0; } + @Override public int compareTo(getScannerResults_result other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; - getScannerResults_result typedOther = (getScannerResults_result)other; - lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(typedOther.isSetSuccess()); + lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(other.isSetSuccess()); if (lastComparison != 0) { return lastComparison; } if (isSetSuccess()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, typedOther.success); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetIo()).compareTo(typedOther.isSetIo()); + lastComparison = Boolean.valueOf(isSetIo()).compareTo(other.isSetIo()); if (lastComparison != 0) { return lastComparison; } if (isSetIo()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.io, typedOther.io); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.io, other.io); if (lastComparison != 0) { return lastComparison; } @@ -16506,6 +17532,7 @@ public class THBaseService { 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 { @@ -16549,7 +17576,7 @@ public class THBaseService { struct.success = new ArrayList(_list144.size); for (int _i145 = 0; _i145 < _list144.size; ++_i145) { - TResult _elem146; // required + TResult _elem146; _elem146 = new TResult(); _elem146.read(iprot); struct.success.add(_elem146); @@ -16651,7 +17678,7 @@ public class THBaseService { struct.success = new ArrayList(_list149.size); for (int _i150 = 0; _i150 < _list149.size; ++_i150) { - TResult _elem151; // required + TResult _elem151; _elem151 = new TResult(); _elem151.read(iprot); struct.success.add(_elem151); diff --git a/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TIOError.java b/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TIOError.java index 19f7546..4b8fa86 100644 --- a/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TIOError.java +++ b/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TIOError.java @@ -1,5 +1,5 @@ /** - * Autogenerated by Thrift Compiler (0.8.0) + * Autogenerated by Thrift Compiler (0.9.1) * * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING * @generated @@ -12,6 +12,11 @@ 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 org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; import java.util.List; import java.util.ArrayList; import java.util.Map; @@ -32,7 +37,7 @@ import org.slf4j.LoggerFactory; * to the HBase master or a HBase region server. Also used to return * more general HBase error conditions. */ -public class TIOError extends Exception implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { +public class TIOError extends TException implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TIOError"); 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); @@ -224,20 +229,20 @@ public class TIOError extends Exception implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { +public class TIllegalArgument extends TException implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TIllegalArgument"); 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); @@ -223,20 +228,20 @@ public class TIllegalArgument extends Exception implements org.apache.thrift.TBa return 0; } + @Override public int compareTo(TIllegalArgument other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; - TIllegalArgument typedOther = (TIllegalArgument)other; - lastComparison = Boolean.valueOf(isSetMessage()).compareTo(typedOther.isSetMessage()); + lastComparison = Boolean.valueOf(isSetMessage()).compareTo(other.isSetMessage()); if (lastComparison != 0) { return lastComparison; } if (isSetMessage()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.message, typedOther.message); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.message, other.message); if (lastComparison != 0) { return lastComparison; } @@ -276,6 +281,7 @@ public class TIllegalArgument extends Exception implements org.apache.thrift.TBa 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 { diff --git a/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TIncrement.java b/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TIncrement.java index c618f42..1f194f0 100644 --- a/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TIncrement.java +++ b/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TIncrement.java @@ -1,5 +1,5 @@ /** - * Autogenerated by Thrift Compiler (0.8.0) + * Autogenerated by Thrift Compiler (0.9.1) * * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING * @generated @@ -12,6 +12,11 @@ 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 org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; import java.util.List; import java.util.ArrayList; import java.util.Map; @@ -33,7 +38,7 @@ import org.slf4j.LoggerFactory; * You can specify if this Increment should be written * to the write-ahead Log (WAL) or not. It defaults to true. */ -public class TIncrement implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { +public class TIncrement implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TIncrement"); private static final org.apache.thrift.protocol.TField ROW_FIELD_DESC = new org.apache.thrift.protocol.TField("row", org.apache.thrift.protocol.TType.STRING, (short)1); @@ -116,7 +121,7 @@ public class TIncrement implements org.apache.thrift.TBase metaDataMap; static { @@ -150,14 +155,13 @@ public class TIncrement implements org.apache.thrift.TBaseother. */ public TIncrement(TIncrement other) { - __isset_bit_vector.clear(); - __isset_bit_vector.or(other.__isset_bit_vector); + __isset_bitfield = other.__isset_bitfield; if (other.isSetRow()) { this.row = org.apache.thrift.TBaseHelper.copyBinary(other.row); ; } if (other.isSetColumns()) { - List __this__columns = new ArrayList(); + List __this__columns = new ArrayList(other.columns.size()); for (TColumnIncrement other_element : other.columns) { __this__columns.add(new TColumnIncrement(other_element)); } @@ -262,16 +266,16 @@ public class TIncrement implements org.apache.thrift.TBase(_list32.size); - for (int _i33 = 0; _i33 < _list32.size; ++_i33) + org.apache.thrift.protocol.TList _list62 = iprot.readListBegin(); + struct.columns = new ArrayList(_list62.size); + for (int _i63 = 0; _i63 < _list62.size; ++_i63) { - TColumnIncrement _elem34; // required - _elem34 = new TColumnIncrement(); - _elem34.read(iprot); - struct.columns.add(_elem34); + TColumnIncrement _elem64; + _elem64 = new TColumnIncrement(); + _elem64.read(iprot); + struct.columns.add(_elem64); } iprot.readListEnd(); } @@ -571,9 +576,9 @@ public class TIncrement implements org.apache.thrift.TBase(_list37.size); - for (int _i38 = 0; _i38 < _list37.size; ++_i38) + org.apache.thrift.protocol.TList _list67 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); + struct.columns = new ArrayList(_list67.size); + for (int _i68 = 0; _i68 < _list67.size; ++_i68) { - TColumnIncrement _elem39; // required - _elem39 = new TColumnIncrement(); - _elem39.read(iprot); - struct.columns.add(_elem39); + TColumnIncrement _elem69; + _elem69 = new TColumnIncrement(); + _elem69.read(iprot); + struct.columns.add(_elem69); } } struct.setColumnsIsSet(true); diff --git a/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TMutation.java b/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TMutation.java index 5c40952..4da3814 100644 --- a/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TMutation.java +++ b/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TMutation.java @@ -1,5 +1,5 @@ /** - * Autogenerated by Thrift Compiler (0.8.0) + * Autogenerated by Thrift Compiler (0.9.1) * * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING * @generated @@ -12,6 +12,11 @@ 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 org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; import java.util.List; import java.util.ArrayList; import java.util.Map; @@ -182,6 +187,7 @@ public class TMutation extends org.apache.thrift.TUnion, java.io.Serializable, Cloneable { +public class TPut implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TPut"); private static final org.apache.thrift.protocol.TField ROW_FIELD_DESC = new org.apache.thrift.protocol.TField("row", org.apache.thrift.protocol.TType.STRING, (short)1); @@ -145,7 +150,7 @@ public class TPut implements org.apache.thrift.TBase, java.i // isset id assignments private static final int __TIMESTAMP_ISSET_ID = 0; private static final int __WRITETOWAL_ISSET_ID = 1; - private BitSet __isset_bit_vector = new BitSet(2); + private byte __isset_bitfield = 0; private _Fields optionals[] = {_Fields.TIMESTAMP,_Fields.WRITE_TO_WAL,_Fields.ATTRIBUTES,_Fields.DURABILITY}; public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; static { @@ -185,14 +190,13 @@ public class TPut implements org.apache.thrift.TBase, java.i * Performs a deep copy on other. */ public TPut(TPut other) { - __isset_bit_vector.clear(); - __isset_bit_vector.or(other.__isset_bit_vector); + __isset_bitfield = other.__isset_bitfield; if (other.isSetRow()) { this.row = org.apache.thrift.TBaseHelper.copyBinary(other.row); ; } if (other.isSetColumnValues()) { - List __this__columnValues = new ArrayList(); + List __this__columnValues = new ArrayList(other.columnValues.size()); for (TColumnValue other_element : other.columnValues) { __this__columnValues.add(new TColumnValue(other_element)); } @@ -201,20 +205,7 @@ public class TPut implements org.apache.thrift.TBase, java.i this.timestamp = other.timestamp; this.writeToWal = other.writeToWal; if (other.isSetAttributes()) { - Map __this__attributes = new HashMap(); - for (Map.Entry other_element : other.attributes.entrySet()) { - - ByteBuffer other_element_key = other_element.getKey(); - ByteBuffer other_element_value = other_element.getValue(); - - ByteBuffer __this__attributes_copy_key = org.apache.thrift.TBaseHelper.copyBinary(other_element_key); -; - - ByteBuffer __this__attributes_copy_value = org.apache.thrift.TBaseHelper.copyBinary(other_element_value); -; - - __this__attributes.put(__this__attributes_copy_key, __this__attributes_copy_value); - } + Map __this__attributes = new HashMap(other.attributes); this.attributes = __this__attributes; } if (other.isSetDurability()) { @@ -322,16 +313,16 @@ public class TPut implements org.apache.thrift.TBase, java.i } public void unsetTimestamp() { - __isset_bit_vector.clear(__TIMESTAMP_ISSET_ID); + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __TIMESTAMP_ISSET_ID); } /** Returns true if field timestamp is set (has been assigned a value) and false otherwise */ public boolean isSetTimestamp() { - return __isset_bit_vector.get(__TIMESTAMP_ISSET_ID); + return EncodingUtils.testBit(__isset_bitfield, __TIMESTAMP_ISSET_ID); } public void setTimestampIsSet(boolean value) { - __isset_bit_vector.set(__TIMESTAMP_ISSET_ID, value); + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __TIMESTAMP_ISSET_ID, value); } public boolean isWriteToWal() { @@ -345,16 +336,16 @@ public class TPut implements org.apache.thrift.TBase, java.i } public void unsetWriteToWal() { - __isset_bit_vector.clear(__WRITETOWAL_ISSET_ID); + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __WRITETOWAL_ISSET_ID); } /** Returns true if field writeToWal is set (has been assigned a value) and false otherwise */ public boolean isSetWriteToWal() { - return __isset_bit_vector.get(__WRITETOWAL_ISSET_ID); + return EncodingUtils.testBit(__isset_bitfield, __WRITETOWAL_ISSET_ID); } public void setWriteToWalIsSet(boolean value) { - __isset_bit_vector.set(__WRITETOWAL_ISSET_ID, value); + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __WRITETOWAL_ISSET_ID, value); } public int getAttributesSize() { @@ -599,70 +590,70 @@ public class TPut implements org.apache.thrift.TBase, java.i return 0; } + @Override public int compareTo(TPut other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; - TPut typedOther = (TPut)other; - lastComparison = Boolean.valueOf(isSetRow()).compareTo(typedOther.isSetRow()); + lastComparison = Boolean.valueOf(isSetRow()).compareTo(other.isSetRow()); if (lastComparison != 0) { return lastComparison; } if (isSetRow()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.row, typedOther.row); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.row, other.row); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetColumnValues()).compareTo(typedOther.isSetColumnValues()); + lastComparison = Boolean.valueOf(isSetColumnValues()).compareTo(other.isSetColumnValues()); if (lastComparison != 0) { return lastComparison; } if (isSetColumnValues()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.columnValues, typedOther.columnValues); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.columnValues, other.columnValues); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetTimestamp()).compareTo(typedOther.isSetTimestamp()); + lastComparison = Boolean.valueOf(isSetTimestamp()).compareTo(other.isSetTimestamp()); if (lastComparison != 0) { return lastComparison; } if (isSetTimestamp()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.timestamp, typedOther.timestamp); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.timestamp, other.timestamp); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetWriteToWal()).compareTo(typedOther.isSetWriteToWal()); + lastComparison = Boolean.valueOf(isSetWriteToWal()).compareTo(other.isSetWriteToWal()); if (lastComparison != 0) { return lastComparison; } if (isSetWriteToWal()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.writeToWal, typedOther.writeToWal); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.writeToWal, other.writeToWal); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetAttributes()).compareTo(typedOther.isSetAttributes()); + lastComparison = Boolean.valueOf(isSetAttributes()).compareTo(other.isSetAttributes()); if (lastComparison != 0) { return lastComparison; } if (isSetAttributes()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.attributes, typedOther.attributes); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.attributes, other.attributes); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetDurability()).compareTo(typedOther.isSetDurability()); + lastComparison = Boolean.valueOf(isSetDurability()).compareTo(other.isSetDurability()); if (lastComparison != 0) { return lastComparison; } if (isSetDurability()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.durability, typedOther.durability); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.durability, other.durability); if (lastComparison != 0) { return lastComparison; } @@ -746,6 +737,7 @@ public class TPut implements org.apache.thrift.TBase, java.i if (columnValues == null) { throw new org.apache.thrift.protocol.TProtocolException("Required field 'columnValues' was not present! Struct: " + toString()); } + // check for sub-struct validity } private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { @@ -759,7 +751,7 @@ public class TPut implements org.apache.thrift.TBase, java.i 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_bit_vector = new BitSet(1); + __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); @@ -799,7 +791,7 @@ public class TPut implements org.apache.thrift.TBase, java.i struct.columnValues = new ArrayList(_list26.size); for (int _i27 = 0; _i27 < _list26.size; ++_i27) { - TColumnValue _elem28; // required + TColumnValue _elem28; _elem28 = new TColumnValue(); _elem28.read(iprot); struct.columnValues.add(_elem28); @@ -834,8 +826,8 @@ public class TPut implements org.apache.thrift.TBase, java.i struct.attributes = new HashMap(2*_map29.size); for (int _i30 = 0; _i30 < _map29.size; ++_i30) { - ByteBuffer _key31; // required - ByteBuffer _val32; // optional + ByteBuffer _key31; + ByteBuffer _val32; _key31 = iprot.readBinary(); _val32 = iprot.readBinary(); struct.attributes.put(_key31, _val32); @@ -989,7 +981,7 @@ public class TPut implements org.apache.thrift.TBase, java.i struct.columnValues = new ArrayList(_list37.size); for (int _i38 = 0; _i38 < _list37.size; ++_i38) { - TColumnValue _elem39; // required + TColumnValue _elem39; _elem39 = new TColumnValue(); _elem39.read(iprot); struct.columnValues.add(_elem39); @@ -1011,8 +1003,8 @@ public class TPut implements org.apache.thrift.TBase, java.i struct.attributes = new HashMap(2*_map40.size); for (int _i41 = 0; _i41 < _map40.size; ++_i41) { - ByteBuffer _key42; // required - ByteBuffer _val43; // optional + ByteBuffer _key42; + ByteBuffer _val43; _key42 = iprot.readBinary(); _val43 = iprot.readBinary(); struct.attributes.put(_key42, _val43); diff --git a/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TResult.java b/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TResult.java index e91f137..101a36f 100644 --- a/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TResult.java +++ b/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TResult.java @@ -1,5 +1,5 @@ /** - * Autogenerated by Thrift Compiler (0.8.0) + * Autogenerated by Thrift Compiler (0.9.1) * * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING * @generated @@ -12,6 +12,11 @@ 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 org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; import java.util.List; import java.util.ArrayList; import java.util.Map; @@ -30,7 +35,7 @@ import org.slf4j.LoggerFactory; /** * if no Result is found, row and columnValues will not be set. */ -public class TResult implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { +public class TResult implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TResult"); private static final org.apache.thrift.protocol.TField ROW_FIELD_DESC = new org.apache.thrift.protocol.TField("row", org.apache.thrift.protocol.TType.STRING, (short)1); @@ -139,7 +144,7 @@ public class TResult implements org.apache.thrift.TBase __this__columnValues = new ArrayList(); + List __this__columnValues = new ArrayList(other.columnValues.size()); for (TColumnValue other_element : other.columnValues) { __this__columnValues.add(new TColumnValue(other_element)); } @@ -317,30 +322,30 @@ public class TResult implements org.apache.thrift.TBase(_list0.size); for (int _i1 = 0; _i1 < _list0.size; ++_i1) { - TColumnValue _elem2; // required + TColumnValue _elem2; _elem2 = new TColumnValue(); _elem2.read(iprot); struct.columnValues.add(_elem2); @@ -530,7 +536,7 @@ public class TResult implements org.apache.thrift.TBase(_list5.size); for (int _i6 = 0; _i6 < _list5.size; ++_i6) { - TColumnValue _elem7; // required + TColumnValue _elem7; _elem7 = new TColumnValue(); _elem7.read(iprot); struct.columnValues.add(_elem7); diff --git a/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TRowMutations.java b/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TRowMutations.java index 44e6678..7de60f7 100644 --- a/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TRowMutations.java +++ b/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TRowMutations.java @@ -1,5 +1,5 @@ /** - * Autogenerated by Thrift Compiler (0.8.0) + * Autogenerated by Thrift Compiler (0.9.1) * * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING * @generated @@ -12,6 +12,11 @@ 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 org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; import java.util.List; import java.util.ArrayList; import java.util.Map; @@ -30,7 +35,7 @@ import org.slf4j.LoggerFactory; /** * A TRowMutations object is used to apply a number of Mutations to a single row. */ -public class TRowMutations implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { +public class TRowMutations implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TRowMutations"); private static final org.apache.thrift.protocol.TField ROW_FIELD_DESC = new org.apache.thrift.protocol.TField("row", org.apache.thrift.protocol.TType.STRING, (short)1); @@ -140,7 +145,7 @@ public class TRowMutations implements org.apache.thrift.TBase __this__mutations = new ArrayList(); + List __this__mutations = new ArrayList(other.mutations.size()); for (TMutation other_element : other.mutations) { __this__mutations.add(new TMutation(other_element)); } @@ -318,30 +323,30 @@ public class TRowMutations implements org.apache.thrift.TBase(_list98.size); - for (int _i99 = 0; _i99 < _list98.size; ++_i99) + org.apache.thrift.protocol.TList _list88 = iprot.readListBegin(); + struct.mutations = new ArrayList(_list88.size); + for (int _i89 = 0; _i89 < _list88.size; ++_i89) { - TMutation _elem100; // required - _elem100 = new TMutation(); - _elem100.read(iprot); - struct.mutations.add(_elem100); + TMutation _elem90; + _elem90 = new TMutation(); + _elem90.read(iprot); + struct.mutations.add(_elem90); } iprot.readListEnd(); } @@ -480,9 +486,9 @@ public class TRowMutations implements org.apache.thrift.TBase(_list103.size); - for (int _i104 = 0; _i104 < _list103.size; ++_i104) + org.apache.thrift.protocol.TList _list93 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); + struct.mutations = new ArrayList(_list93.size); + for (int _i94 = 0; _i94 < _list93.size; ++_i94) { - TMutation _elem105; // required - _elem105 = new TMutation(); - _elem105.read(iprot); - struct.mutations.add(_elem105); + TMutation _elem95; + _elem95 = new TMutation(); + _elem95.read(iprot); + struct.mutations.add(_elem95); } } struct.setMutationsIsSet(true); diff --git a/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TScan.java b/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TScan.java index 4c58a8c..2fa4a85 100644 --- a/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TScan.java +++ b/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TScan.java @@ -1,5 +1,5 @@ /** - * Autogenerated by Thrift Compiler (0.8.0) + * Autogenerated by Thrift Compiler (0.9.1) * * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING * @generated @@ -12,6 +12,11 @@ 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 org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; import java.util.List; import java.util.ArrayList; import java.util.Map; @@ -31,7 +36,7 @@ import org.slf4j.LoggerFactory; * Any timestamps in the columns are ignored, use timeRange to select by timestamp. * Max versions defaults to 1. */ -public class TScan implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { +public class TScan implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TScan"); private static final org.apache.thrift.protocol.TField START_ROW_FIELD_DESC = new org.apache.thrift.protocol.TField("startRow", org.apache.thrift.protocol.TType.STRING, (short)1); @@ -146,7 +151,7 @@ public class TScan implements org.apache.thrift.TBase, jav private static final int __CACHING_ISSET_ID = 0; private static final int __MAXVERSIONS_ISSET_ID = 1; private static final int __BATCHSIZE_ISSET_ID = 2; - private BitSet __isset_bit_vector = new BitSet(3); + private byte __isset_bitfield = 0; private _Fields optionals[] = {_Fields.START_ROW,_Fields.STOP_ROW,_Fields.COLUMNS,_Fields.CACHING,_Fields.MAX_VERSIONS,_Fields.TIME_RANGE,_Fields.FILTER_STRING,_Fields.BATCH_SIZE,_Fields.ATTRIBUTES}; public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; static { @@ -185,8 +190,7 @@ public class TScan implements org.apache.thrift.TBase, jav * Performs a deep copy on other. */ public TScan(TScan other) { - __isset_bit_vector.clear(); - __isset_bit_vector.or(other.__isset_bit_vector); + __isset_bitfield = other.__isset_bitfield; if (other.isSetStartRow()) { this.startRow = org.apache.thrift.TBaseHelper.copyBinary(other.startRow); ; @@ -196,7 +200,7 @@ public class TScan implements org.apache.thrift.TBase, jav ; } if (other.isSetColumns()) { - List __this__columns = new ArrayList(); + List __this__columns = new ArrayList(other.columns.size()); for (TColumn other_element : other.columns) { __this__columns.add(new TColumn(other_element)); } @@ -213,20 +217,7 @@ public class TScan implements org.apache.thrift.TBase, jav } this.batchSize = other.batchSize; if (other.isSetAttributes()) { - Map __this__attributes = new HashMap(); - for (Map.Entry other_element : other.attributes.entrySet()) { - - ByteBuffer other_element_key = other_element.getKey(); - ByteBuffer other_element_value = other_element.getValue(); - - ByteBuffer __this__attributes_copy_key = org.apache.thrift.TBaseHelper.copyBinary(other_element_key); -; - - ByteBuffer __this__attributes_copy_value = org.apache.thrift.TBaseHelper.copyBinary(other_element_value); -; - - __this__attributes.put(__this__attributes_copy_key, __this__attributes_copy_value); - } + Map __this__attributes = new HashMap(other.attributes); this.attributes = __this__attributes; } } @@ -369,16 +360,16 @@ public class TScan implements org.apache.thrift.TBase, jav } public void unsetCaching() { - __isset_bit_vector.clear(__CACHING_ISSET_ID); + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __CACHING_ISSET_ID); } /** Returns true if field caching is set (has been assigned a value) and false otherwise */ public boolean isSetCaching() { - return __isset_bit_vector.get(__CACHING_ISSET_ID); + return EncodingUtils.testBit(__isset_bitfield, __CACHING_ISSET_ID); } public void setCachingIsSet(boolean value) { - __isset_bit_vector.set(__CACHING_ISSET_ID, value); + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __CACHING_ISSET_ID, value); } public int getMaxVersions() { @@ -392,16 +383,16 @@ public class TScan implements org.apache.thrift.TBase, jav } public void unsetMaxVersions() { - __isset_bit_vector.clear(__MAXVERSIONS_ISSET_ID); + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __MAXVERSIONS_ISSET_ID); } /** Returns true if field maxVersions is set (has been assigned a value) and false otherwise */ public boolean isSetMaxVersions() { - return __isset_bit_vector.get(__MAXVERSIONS_ISSET_ID); + return EncodingUtils.testBit(__isset_bitfield, __MAXVERSIONS_ISSET_ID); } public void setMaxVersionsIsSet(boolean value) { - __isset_bit_vector.set(__MAXVERSIONS_ISSET_ID, value); + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __MAXVERSIONS_ISSET_ID, value); } public TTimeRange getTimeRange() { @@ -473,16 +464,16 @@ public class TScan implements org.apache.thrift.TBase, jav } public void unsetBatchSize() { - __isset_bit_vector.clear(__BATCHSIZE_ISSET_ID); + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __BATCHSIZE_ISSET_ID); } /** Returns true if field batchSize is set (has been assigned a value) and false otherwise */ public boolean isSetBatchSize() { - return __isset_bit_vector.get(__BATCHSIZE_ISSET_ID); + return EncodingUtils.testBit(__isset_bitfield, __BATCHSIZE_ISSET_ID); } public void setBatchSizeIsSet(boolean value) { - __isset_bit_vector.set(__BATCHSIZE_ISSET_ID, value); + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __BATCHSIZE_ISSET_ID, value); } public int getAttributesSize() { @@ -761,100 +752,100 @@ public class TScan implements org.apache.thrift.TBase, jav return 0; } + @Override public int compareTo(TScan other) { if (!getClass().equals(other.getClass())) { return getClass().getName().compareTo(other.getClass().getName()); } int lastComparison = 0; - TScan typedOther = (TScan)other; - lastComparison = Boolean.valueOf(isSetStartRow()).compareTo(typedOther.isSetStartRow()); + lastComparison = Boolean.valueOf(isSetStartRow()).compareTo(other.isSetStartRow()); if (lastComparison != 0) { return lastComparison; } if (isSetStartRow()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.startRow, typedOther.startRow); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.startRow, other.startRow); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetStopRow()).compareTo(typedOther.isSetStopRow()); + lastComparison = Boolean.valueOf(isSetStopRow()).compareTo(other.isSetStopRow()); if (lastComparison != 0) { return lastComparison; } if (isSetStopRow()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.stopRow, typedOther.stopRow); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.stopRow, other.stopRow); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetColumns()).compareTo(typedOther.isSetColumns()); + lastComparison = Boolean.valueOf(isSetColumns()).compareTo(other.isSetColumns()); if (lastComparison != 0) { return lastComparison; } if (isSetColumns()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.columns, typedOther.columns); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.columns, other.columns); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetCaching()).compareTo(typedOther.isSetCaching()); + lastComparison = Boolean.valueOf(isSetCaching()).compareTo(other.isSetCaching()); if (lastComparison != 0) { return lastComparison; } if (isSetCaching()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.caching, typedOther.caching); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.caching, other.caching); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetMaxVersions()).compareTo(typedOther.isSetMaxVersions()); + lastComparison = Boolean.valueOf(isSetMaxVersions()).compareTo(other.isSetMaxVersions()); if (lastComparison != 0) { return lastComparison; } if (isSetMaxVersions()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.maxVersions, typedOther.maxVersions); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.maxVersions, other.maxVersions); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetTimeRange()).compareTo(typedOther.isSetTimeRange()); + lastComparison = Boolean.valueOf(isSetTimeRange()).compareTo(other.isSetTimeRange()); if (lastComparison != 0) { return lastComparison; } if (isSetTimeRange()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.timeRange, typedOther.timeRange); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.timeRange, other.timeRange); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetFilterString()).compareTo(typedOther.isSetFilterString()); + lastComparison = Boolean.valueOf(isSetFilterString()).compareTo(other.isSetFilterString()); if (lastComparison != 0) { return lastComparison; } if (isSetFilterString()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.filterString, typedOther.filterString); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.filterString, other.filterString); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetBatchSize()).compareTo(typedOther.isSetBatchSize()); + lastComparison = Boolean.valueOf(isSetBatchSize()).compareTo(other.isSetBatchSize()); if (lastComparison != 0) { return lastComparison; } if (isSetBatchSize()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.batchSize, typedOther.batchSize); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.batchSize, other.batchSize); if (lastComparison != 0) { return lastComparison; } } - lastComparison = Boolean.valueOf(isSetAttributes()).compareTo(typedOther.isSetAttributes()); + lastComparison = Boolean.valueOf(isSetAttributes()).compareTo(other.isSetAttributes()); if (lastComparison != 0) { return lastComparison; } if (isSetAttributes()) { - lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.attributes, typedOther.attributes); + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.attributes, other.attributes); if (lastComparison != 0) { return lastComparison; } @@ -962,6 +953,10 @@ public class TScan implements org.apache.thrift.TBase, jav public void validate() throws org.apache.thrift.TException { // check for required fields + // check for sub-struct validity + if (timeRange != null) { + timeRange.validate(); + } } private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { @@ -975,7 +970,7 @@ public class TScan implements org.apache.thrift.TBase, jav 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_bit_vector = new BitSet(1); + __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); @@ -1023,7 +1018,7 @@ public class TScan implements org.apache.thrift.TBase, jav struct.columns = new ArrayList(_list70.size); for (int _i71 = 0; _i71 < _list70.size; ++_i71) { - TColumn _elem72; // required + TColumn _elem72; _elem72 = new TColumn(); _elem72.read(iprot); struct.columns.add(_elem72); @@ -1083,8 +1078,8 @@ public class TScan implements org.apache.thrift.TBase, jav struct.attributes = new HashMap(2*_map73.size); for (int _i74 = 0; _i74 < _map73.size; ++_i74) { - ByteBuffer _key75; // required - ByteBuffer _val76; // optional + ByteBuffer _key75; + ByteBuffer _val76; _key75 = iprot.readBinary(); _val76 = iprot.readBinary(); struct.attributes.put(_key75, _val76); @@ -1289,7 +1284,7 @@ public class TScan implements org.apache.thrift.TBase, jav struct.columns = new ArrayList(_list81.size); for (int _i82 = 0; _i82 < _list81.size; ++_i82) { - TColumn _elem83; // required + TColumn _elem83; _elem83 = new TColumn(); _elem83.read(iprot); struct.columns.add(_elem83); @@ -1324,8 +1319,8 @@ public class TScan implements org.apache.thrift.TBase, jav struct.attributes = new HashMap(2*_map84.size); for (int _i85 = 0; _i85 < _map84.size; ++_i85) { - ByteBuffer _key86; // required - ByteBuffer _val87; // optional + ByteBuffer _key86; + ByteBuffer _val87; _key86 = iprot.readBinary(); _val87 = iprot.readBinary(); struct.attributes.put(_key86, _val87); diff --git a/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TTimeRange.java b/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TTimeRange.java index 98058ea..3fbd30f 100644 --- a/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TTimeRange.java +++ b/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TTimeRange.java @@ -1,5 +1,5 @@ /** - * Autogenerated by Thrift Compiler (0.8.0) + * Autogenerated by Thrift Compiler (0.9.1) * * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING * @generated @@ -12,6 +12,11 @@ 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 org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; import java.util.List; import java.util.ArrayList; import java.util.Map; @@ -27,7 +32,7 @@ import java.util.Arrays; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -public class TTimeRange implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { +public class TTimeRange implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TTimeRange"); private static final org.apache.thrift.protocol.TField MIN_STAMP_FIELD_DESC = new org.apache.thrift.protocol.TField("minStamp", org.apache.thrift.protocol.TType.I64, (short)1); @@ -106,7 +111,7 @@ public class TTimeRange implements org.apache.thrift.TBase metaDataMap; static { Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); @@ -136,8 +141,7 @@ public class TTimeRange implements org.apache.thrift.TBaseother. */ public TTimeRange(TTimeRange other) { - __isset_bit_vector.clear(); - __isset_bit_vector.or(other.__isset_bit_vector); + __isset_bitfield = other.__isset_bitfield; this.minStamp = other.minStamp; this.maxStamp = other.maxStamp; } @@ -165,16 +169,16 @@ public class TTimeRange implements org.apache.thrift.TBase mutations, +} + +struct TEdit { + 1: required THLogKey hLogKey, + 2: required TWalLEdit walEdit + 3: required list clusterIds +} + +struct TBatchEdit { + 1: required list edits +} + +// +// Exceptions +// + +/** + * A TIOError exception signals that an error occurred communicating + * to the HBase master or a HBase region server. Also used to return + * more general HBase error conditions. + */ +exception TIOError { + 1: optional string message +} + +/** + * A TIllegalArgument exception indicates an illegal or invalid + * argument was passed into a procedure. + */ +exception TIllegalArgument { + 1: optional string message +} + +service THBaseService { + + void replicate( + 1: required TBatchEdit edits ) throws (1: TIOError io) + + void ping() + + string getClusterUUID() +} diff --git a/src/main/ruby/hbase/replication_admin.rb b/src/main/ruby/hbase/replication_admin.rb index 99f0479..260f4c2 100644 --- a/src/main/ruby/hbase/replication_admin.rb +++ b/src/main/ruby/hbase/replication_admin.rb @@ -33,8 +33,8 @@ module Hbase #---------------------------------------------------------------------------------------------- # Add a new peer cluster to replicate to - def add_peer(id, cluster_key) - @replication_admin.addPeer(id, cluster_key) + def add_peer(id, cluster_key, protocol='NATIVE') + @replication_admin.addPeer(id, cluster_key, protocol) end #---------------------------------------------------------------------------------------------- @@ -62,6 +62,12 @@ module Hbase end #---------------------------------------------------------------------------------------------- + # Get peer cluster protocol + def get_peer_protocol(id) + @replication_admin.getPeerProtocol(id) + end + + #---------------------------------------------------------------------------------------------- # Restart the replication stream to the specified peer def enable_peer(id) @replication_admin.enablePeer(id) diff --git a/src/main/ruby/shell/commands/add_peer.rb b/src/main/ruby/shell/commands/add_peer.rb index 7669fb7..caf7125 100644 --- a/src/main/ruby/shell/commands/add_peer.rb +++ b/src/main/ruby/shell/commands/add_peer.rb @@ -27,16 +27,18 @@ Add a peer cluster to replicate to, the id must be a short and the cluster key is composed like this: hbase.zookeeper.quorum:hbase.zookeeper.property.clientPort:zookeeper.znode.parent This gives a full path for HBase to connect to another cluster. +You can optionally pass a parameter for the replication protocol {NATIVE, THRIFT} default: NATIVE Examples: hbase> add_peer '1', "server1.cie.com:2181:/hbase" hbase> add_peer '2', "zk1,zk2,zk3:2182:/hbase-prod" + hbase> add_peer '3', "zk1,zk2,zk3:2182:/hbase-prod", "THRIFT" EOF end - def command(id, cluster_key) + def command(id, cluster_key, protocol='NATIVE') format_simple_command do - replication_admin.add_peer(id, cluster_key) + replication_admin.add_peer(id, cluster_key, protocol) end end end diff --git a/src/main/ruby/shell/commands/list_peers.rb b/src/main/ruby/shell/commands/list_peers.rb index 2f57592..46c184c 100644 --- a/src/main/ruby/shell/commands/list_peers.rb +++ b/src/main/ruby/shell/commands/list_peers.rb @@ -33,11 +33,12 @@ EOF now = Time.now peers = replication_admin.list_peers - formatter.header(["PEER_ID", "CLUSTER_KEY", "STATE"]) + formatter.header(["PEER_ID", "CLUSTER_KEY", "STATE", "PROTOCOL"]) peers.entrySet().each do |e| state = replication_admin.get_peer_state(e.key) - formatter.row([ e.key, e.value, state ]) + protocol = replication_admin.get_peer_protocol(e.key) + formatter.row([ e.key, e.value, state, protocol ]) end formatter.footer(now) diff --git a/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdmin.java b/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdmin.java index 6dabc27..cc60f58 100644 --- a/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdmin.java +++ b/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdmin.java @@ -25,6 +25,8 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.*; +import org.apache.hadoop.hbase.replication.ReplicationPeer; +import org.apache.hadoop.hbase.replication.ReplicationZookeeper; import org.apache.hadoop.hbase.replication.regionserver.ReplicationSourceManager; import org.junit.BeforeClass; import org.junit.Test; @@ -115,6 +117,41 @@ public class TestReplicationAdmin { assertEquals(1, admin.getPeersCount()); } + @Test + public void testAddRemovePeerWithProtocol() throws Exception { + assertEquals(0, manager.getSources().size()); + // Add a valid peer + admin.addPeer(ID_ONE, KEY_ONE, ReplicationZookeeper.PeerProtocol.THRIFT.name()); + // try adding the same (fails) + try { + admin.addPeer(ID_ONE, KEY_ONE); + } catch (IllegalArgumentException iae) { + // OK! + } + assertEquals(1, admin.getPeersCount()); + assertEquals(ReplicationZookeeper.PeerProtocol.THRIFT.name(), admin.getPeerProtocol(ID_ONE)); + // Try to remove an inexisting peer + try { + admin.removePeer(ID_SECOND); + fail(); + } catch (IllegalArgumentException iae) { + // OK! + } + assertEquals(1, admin.getPeersCount()); + // Add a second since multi-slave is supported + try { + admin.addPeer(ID_SECOND, KEY_SECOND); + } catch (IllegalStateException iae) { + fail(); + // OK! + } + assertEquals(2, admin.getPeersCount()); + assertEquals(ReplicationZookeeper.PeerProtocol.NATIVE.name(), admin.getPeerProtocol(ID_SECOND)); + // Remove the first peer we added + admin.removePeer(ID_ONE); + assertEquals(1, admin.getPeersCount()); + } + @org.junit.Rule public org.apache.hadoop.hbase.ResourceCheckerJUnitRule cu = new org.apache.hadoop.hbase.ResourceCheckerJUnitRule(); diff --git a/src/test/java/org/apache/hadoop/hbase/replication/TestMasterReplication.java b/src/test/java/org/apache/hadoop/hbase/replication/TestMasterReplication.java index 69a8f2f..4df478f 100644 --- a/src/test/java/org/apache/hadoop/hbase/replication/TestMasterReplication.java +++ b/src/test/java/org/apache/hadoop/hbase/replication/TestMasterReplication.java @@ -23,10 +23,8 @@ import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertEquals; import static org.junit.Assert.fail; -import java.io.Closeable; import java.io.IOException; import java.util.List; -import java.util.Random; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -57,22 +55,26 @@ public class TestMasterReplication { private static final Log LOG = LogFactory.getLog(TestReplicationBase.class); - private Configuration baseConfiguration; + protected Configuration baseConf = HBaseConfiguration.create(); + protected ReplicationZookeeper.PeerProtocol protocol = ReplicationZookeeper.PeerProtocol.NATIVE; + private Configuration conf1; + private Configuration conf2; + private Configuration conf3; + + private HBaseTestingUtility utility1; + private HBaseTestingUtility utility2; + private HBaseTestingUtility utility3; - private HBaseTestingUtility[] utilities; - private Configuration[] configurations; private MiniZooKeeperCluster miniZK; private static final long SLEEP_TIME = 500; - private static final int NB_RETRIES = 10; + private static final int NB_RETRIES = 100; private static final byte[] tableName = Bytes.toBytes("test"); private static final byte[] famName = Bytes.toBytes("f"); private static final byte[] row = Bytes.toBytes("row"); private static final byte[] row1 = Bytes.toBytes("row1"); private static final byte[] row2 = Bytes.toBytes("row2"); - private static final byte[] row3 = Bytes.toBytes("row3"); - private static final byte[] row4 = Bytes.toBytes("row4"); private static final byte[] noRepfamName = Bytes.toBytes("norep"); private static final byte[] count = Bytes.toBytes("count"); @@ -83,21 +85,44 @@ public class TestMasterReplication { @Before public void setUp() throws Exception { - baseConfiguration = HBaseConfiguration.create(); + conf1 = baseConf; + conf1.set(HConstants.ZOOKEEPER_ZNODE_PARENT, "/1"); // smaller block size and capacity to trigger more operations // and test them - baseConfiguration.setInt("hbase.regionserver.hlog.blocksize", 1024 * 20); - baseConfiguration.setInt("replication.source.size.capacity", 1024); - baseConfiguration.setLong("replication.source.sleepforretries", 100); - baseConfiguration.setInt("hbase.regionserver.maxlogs", 10); - baseConfiguration.setLong("hbase.master.logcleaner.ttl", 10); - baseConfiguration.setBoolean(HConstants.REPLICATION_ENABLE_KEY, true); - baseConfiguration.setBoolean("dfs.support.append", true); - baseConfiguration.setLong(HConstants.THREAD_WAKE_FREQUENCY, 100); - baseConfiguration.setStrings( - CoprocessorHost.USER_REGION_COPROCESSOR_CONF_KEY, + conf1.setInt("hbase.regionserver.hlog.blocksize", 1024*20); + conf1.setInt("replication.source.size.capacity", 1024); + conf1.setLong("replication.source.sleepforretries", 100); + conf1.setInt("hbase.regionserver.maxlogs", 10); + conf1.setLong("hbase.master.logcleaner.ttl", 10); + conf1.setBoolean(HConstants.REPLICATION_ENABLE_KEY, true); + conf1.setBoolean("dfs.support.append", true); + conf1.setLong(HConstants.THREAD_WAKE_FREQUENCY, 100); + conf1.setStrings(CoprocessorHost.USER_REGION_COPROCESSOR_CONF_KEY, CoprocessorCounter.class.getName()); + utility1 = new HBaseTestingUtility(conf1); + utility1.startMiniZKCluster(); + miniZK = utility1.getZkCluster(); + // By setting the mini ZK cluster through this method, even though this is + // already utility1's mini ZK cluster, we are telling utility1 not to shut + // the mini ZK cluster when we shut down the HBase cluster. + utility1.setZkCluster(miniZK); + new ZooKeeperWatcher(conf1, "cluster1", null, true); + + conf2 = new Configuration(conf1); + conf2.set(HConstants.ZOOKEEPER_ZNODE_PARENT, "/2"); + + utility2 = new HBaseTestingUtility(conf2); + utility2.setZkCluster(miniZK); + new ZooKeeperWatcher(conf2, "cluster2", null, true); + + conf3 = new Configuration(conf1); + conf3.set(HConstants.ZOOKEEPER_ZNODE_PARENT, "/3"); + + utility3 = new HBaseTestingUtility(conf3); + utility3.setZkCluster(miniZK); + new ZooKeeperWatcher(conf3, "cluster3", null, true); + table = new HTableDescriptor(tableName); HColumnDescriptor fam = new HColumnDescriptor(famName); fam.setScope(HConstants.REPLICATION_SCOPE_GLOBAL); @@ -106,258 +131,111 @@ public class TestMasterReplication { table.addFamily(fam); } - /** - * It tests the replication scenario involving 0 -> 1 -> 0. It does it by - * adding and deleting a row to a table in each cluster, checking if it's - * replicated. It also tests that the puts and deletes are not replicated back - * to the originating cluster. - */ - @Test(timeout = 300000) - public void testCyclicReplication1() throws Exception { - LOG.info("testSimplePutDelete"); - int numClusters = 2; - HTable[] htables = null; - try { - startMiniClusters(numClusters); - createTableOnClusters(table); - - htables = getHTablesOnClusters(tableName); - - // Test the replication scenarios of 0 -> 1 -> 0 - addPeer("1", 0, 1); - addPeer("1", 1, 0); - - int[] expectedCounts = new int[] { 2, 2 }; - - // add rows to both clusters, - // make sure they are both replication - putAndWait(row, famName, htables[0], htables[1]); - putAndWait(row1, famName, htables[1], htables[0]); - validateCounts(htables, put, expectedCounts); - - deleteAndWait(row, htables[0], htables[1]); - deleteAndWait(row1, htables[1], htables[0]); - validateCounts(htables, delete, expectedCounts); - } finally { - close(htables); - shutDownMiniClusters(); - } + @After + public void tearDown() throws IOException { + miniZK.shutdown(); } - /** - * Tests the cyclic replication scenario of 0 -> 1 -> 2 -> 0 by adding and - * deleting rows to a table in each clusters and ensuring that the each of - * these clusters get the appropriate mutations. It also tests the grouping - * scenario where a cluster needs to replicate the edits originating from - * itself and also the edits that it received using replication from a - * different cluster. The scenario is explained in HBASE-9158 - */ - @Test(timeout = 300000) - public void testCyclicReplication2() throws Exception { - LOG.info("testCyclicReplication1"); - int numClusters = 3; - HTable[] htables = null; - try { - startMiniClusters(numClusters); - createTableOnClusters(table); - - // Test the replication scenario of 0 -> 1 -> 2 -> 0 - addPeer("1", 0, 1); - addPeer("1", 1, 2); - addPeer("1", 2, 0); - - htables = getHTablesOnClusters(tableName); - - // put "row" and wait 'til it got around - putAndWait(row, famName, htables[0], htables[2]); - putAndWait(row1, famName, htables[1], htables[0]); - putAndWait(row2, famName, htables[2], htables[1]); - - deleteAndWait(row, htables[0], htables[2]); - deleteAndWait(row1, htables[1], htables[0]); - deleteAndWait(row2, htables[2], htables[1]); - - int[] expectedCounts = new int[] { 3, 3, 3 }; - validateCounts(htables, put, expectedCounts); - validateCounts(htables, delete, expectedCounts); - - // Test HBASE-9158 - disablePeer("1", 2); - // we now have an edit that was replicated into cluster originating from - // cluster 0 - putAndWait(row3, famName, htables[0], htables[1]); - // now add a local edit to cluster 1 - htables[1].put(new Put(row4).add(famName, row4, row4)); - // re-enable replication from cluster 2 to cluster 0 - enablePeer("1", 2); - // without HBASE-9158 the edit for row4 would have been marked with - // cluster 0's id - // and hence not replicated to cluster 0 - wait(row4, htables[0], true); - } finally { - close(htables); - shutDownMiniClusters(); - } + @Test(timeout=300000) + public void testCyclicReplication() throws Exception { + LOG.info("testCyclicReplication"); + utility1.startMiniCluster(); + utility2.startMiniCluster(); + utility3.startMiniCluster(); + ReplicationAdmin admin1 = new ReplicationAdmin(conf1); + ReplicationAdmin admin2 = new ReplicationAdmin(conf2); + ReplicationAdmin admin3 = new ReplicationAdmin(conf3); + + new HBaseAdmin(conf1).createTable(table); + new HBaseAdmin(conf2).createTable(table); + new HBaseAdmin(conf3).createTable(table); + HTable htable1 = new HTable(conf1, tableName); + htable1.setWriteBufferSize(1024); + HTable htable2 = new HTable(conf2, tableName); + htable2.setWriteBufferSize(1024); + HTable htable3 = new HTable(conf3, tableName); + htable3.setWriteBufferSize(1024); + + admin1.addPeer("1", utility2.getClusterKey(), protocol.name()); + admin2.addPeer("1", utility3.getClusterKey(), protocol.name()); + admin3.addPeer("1", utility1.getClusterKey(), protocol.name()); + + // put "row" and wait 'til it got around + putAndWait(row, famName, htable1, htable3); + // it should have passed through table2 + check(row,famName,htable2); + + putAndWait(row1, famName, htable2, htable1); + check(row,famName,htable3); + putAndWait(row2, famName, htable3, htable2); + check(row,famName,htable1); + + deleteAndWait(row,htable1,htable3); + deleteAndWait(row1,htable2,htable1); + deleteAndWait(row2,htable3,htable2); + + assertEquals("Puts were replicated back ", 3, getCount(htable1, put)); + assertEquals("Puts were replicated back ", 3, getCount(htable2, put)); + assertEquals("Puts were replicated back ", 3, getCount(htable3, put)); + assertEquals("Deletes were replicated back ", 3, getCount(htable1, delete)); + assertEquals("Deletes were replicated back ", 3, getCount(htable2, delete)); + assertEquals("Deletes were replicated back ", 3, getCount(htable3, delete)); + utility3.shutdownMiniCluster(); + utility2.shutdownMiniCluster(); + utility1.shutdownMiniCluster(); } /** - * Tests cyclic replication scenario of 0 -> 1 -> 2 -> 1. + * Add a row to a table in each cluster, check it's replicated, + * delete it, check's gone + * Also check the puts and deletes are not replicated back to + * the originating cluster. */ - @Test(timeout = 300000) - public void testCyclicReplication3() throws Exception { - LOG.info("testCyclicReplication2"); - int numClusters = 3; - HTable[] htables = null; - try { - startMiniClusters(numClusters); - createTableOnClusters(table); - - // Test the replication scenario of 0 -> 1 -> 2 -> 1 - addPeer("1", 0, 1); - addPeer("1", 1, 2); - addPeer("1", 2, 1); - - htables = getHTablesOnClusters(tableName); - - // put "row" and wait 'til it got around - putAndWait(row, famName, htables[0], htables[2]); - putAndWait(row1, famName, htables[1], htables[2]); - putAndWait(row2, famName, htables[2], htables[1]); - - deleteAndWait(row, htables[0], htables[2]); - deleteAndWait(row1, htables[1], htables[2]); - deleteAndWait(row2, htables[2], htables[1]); - - int[] expectedCounts = new int[] { 1, 3, 3 }; - validateCounts(htables, put, expectedCounts); - validateCounts(htables, delete, expectedCounts); - } finally { - close(htables); - shutDownMiniClusters(); - } - } + @Test(timeout=300000) + public void testSimplePutDelete() throws Exception { + LOG.info("testSimplePutDelete"); + utility1.startMiniCluster(); + utility2.startMiniCluster(); - @After - public void tearDown() throws IOException { - configurations = null; - utilities = null; - } + ReplicationAdmin admin1 = new ReplicationAdmin(conf1); + ReplicationAdmin admin2 = new ReplicationAdmin(conf2); - @SuppressWarnings("resource") - private void startMiniClusters(int numClusters) throws Exception { - Random random = new Random(); - utilities = new HBaseTestingUtility[numClusters]; - configurations = new Configuration[numClusters]; - for (int i = 0; i < numClusters; i++) { - Configuration conf = new Configuration(baseConfiguration); - conf.set(HConstants.ZOOKEEPER_ZNODE_PARENT, "/" + i + random.nextInt()); - HBaseTestingUtility utility = new HBaseTestingUtility(conf); - if (i == 0) { - utility.startMiniZKCluster(); - miniZK = utility.getZkCluster(); - } else { - utility.setZkCluster(miniZK); - } - utility.startMiniCluster(); - utilities[i] = utility; - configurations[i] = conf; - new ZooKeeperWatcher(conf, "cluster" + i, null, true); - } - } - - private void shutDownMiniClusters() throws Exception { - int numClusters = utilities.length; - for (int i = numClusters - 1; i >= 0; i--) { - if (utilities[i] != null) { - utilities[i].shutdownMiniCluster(); - } - } - miniZK.shutdown(); - } + new HBaseAdmin(conf1).createTable(table); + new HBaseAdmin(conf2).createTable(table); + HTable htable1 = new HTable(conf1, tableName); + htable1.setWriteBufferSize(1024); + HTable htable2 = new HTable(conf2, tableName); + htable2.setWriteBufferSize(1024); - private void createTableOnClusters(HTableDescriptor table) throws Exception { - int numClusters = configurations.length; - for (int i = 0; i < numClusters; i++) { - HBaseAdmin hbaseAdmin = null; - try { - hbaseAdmin = new HBaseAdmin(configurations[i]); - hbaseAdmin.createTable(table); - } finally { - close(hbaseAdmin); - } - } - } + // set M-M + admin1.addPeer("1", utility2.getClusterKey(), protocol.name()); + admin2.addPeer("1", utility1.getClusterKey(), protocol.name()); + LOG.info("Using peer protocol: " + protocol); - private void addPeer(String id, int masterClusterNumber, - int slaveClusterNumber) throws Exception { - ReplicationAdmin replicationAdmin = null; - try { - replicationAdmin = new ReplicationAdmin( - configurations[masterClusterNumber]); - replicationAdmin.addPeer(id, - utilities[slaveClusterNumber].getClusterKey()); - } finally { - close(replicationAdmin); - } - } + // add rows to both clusters, + // make sure they are both replication + putAndWait(row, famName, htable1, htable2); + putAndWait(row1, famName, htable2, htable1); - private void disablePeer(String id, int masterClusterNumber) throws Exception { - ReplicationAdmin replicationAdmin = null; - try { - replicationAdmin = new ReplicationAdmin( - configurations[masterClusterNumber]); - replicationAdmin.disablePeer(id); - } finally { - close(replicationAdmin); - } - } + // make sure "row" did not get replicated back. + assertEquals("Puts were replicated back ", 2, getCount(htable1, put)); - private void enablePeer(String id, int masterClusterNumber) throws Exception { - ReplicationAdmin replicationAdmin = null; - try { - replicationAdmin = new ReplicationAdmin( - configurations[masterClusterNumber]); - replicationAdmin.enablePeer(id); - } finally { - close(replicationAdmin); - } - } + // delete "row" and wait + deleteAndWait(row, htable1, htable2); - private void close(Closeable... closeables) { - try { - if (closeables != null) { - for (Closeable closeable : closeables) { - closeable.close(); - } - } - } catch (Exception e) { - LOG.warn("Exception occured while closing the object:", e); - } - } + // make the 2nd cluster replicated back + assertEquals("Puts were replicated back ", 2, getCount(htable2, put)); - @SuppressWarnings("resource") - private HTable[] getHTablesOnClusters(byte[] tableName) throws Exception { - int numClusters = utilities.length; - HTable[] htables = new HTable[numClusters]; - for (int i = 0; i < numClusters; i++) { - HTable htable = new HTable(configurations[i], tableName); - htable.setWriteBufferSize(1024); - htables[i] = htable; - } - return htables; - } + deleteAndWait(row1, htable2, htable1); - private void validateCounts(HTable[] htables, byte[] type, - int[] expectedCounts) throws IOException { - for (int i = 0; i < htables.length; i++) { - assertEquals(Bytes.toString(type) + " were replicated back ", - expectedCounts[i], getCount(htables[i], type)); - } + assertEquals("Deletes were replicated back ", 2, getCount(htable1, delete)); + utility2.shutdownMiniCluster(); + utility1.shutdownMiniCluster(); } - private int getCount(HTable t, byte[] type) throws IOException { + private int getCount(HTable t, byte[] type) throws IOException { Get test = new Get(row); - test.setAttribute("count", new byte[] {}); + test.setAttribute("count", new byte[]{}); Result res = t.get(test); return Bytes.toInt(res.getValue(count, type)); } @@ -366,7 +244,28 @@ public class TestMasterReplication { throws Exception { Delete del = new Delete(row); source.delete(del); - wait(row, target, true); + + Get get = new Get(row); + for (int i = 0; i < NB_RETRIES; i++) { + if (i==NB_RETRIES-1) { + fail("Waited too much time for del replication"); + } + Result res = target.get(get); + if (res.size() >= 1) { + LOG.info("Row not deleted"); + Thread.sleep(SLEEP_TIME); + } else { + break; + } + } + } + + private void check(byte[] row, byte[] fam, HTable t) throws IOException { + Get get = new Get(row); + Result res = t.get(get); + if (res.size() == 0) { + fail("Row is missing"); + } } private void putAndWait(byte[] row, byte[] fam, HTable source, HTable target) @@ -374,29 +273,18 @@ public class TestMasterReplication { Put put = new Put(row); put.add(fam, row, row); source.put(put); - wait(row, target, false); - } - private void wait(byte[] row, HTable target, boolean isDeleted) - throws Exception { Get get = new Get(row); for (int i = 0; i < NB_RETRIES; i++) { - if (i == NB_RETRIES - 1) { - fail("Waited too much time for replication. Row:" + Bytes.toString(row) - + ". IsDeleteReplication:" + isDeleted); + if (i==NB_RETRIES-1) { + fail("Waited too much time for put replication"); } Result res = target.get(get); - boolean sleep = isDeleted ? res.size() > 0 : res.size() == 0; - if (sleep) { - LOG.info("Waiting for more time for replication. Row:" - + Bytes.toString(row) + ". IsDeleteReplication:" + isDeleted); + if (res.size() == 0) { + LOG.info("Row not available"); Thread.sleep(SLEEP_TIME); } else { - if (!isDeleted) { - assertArrayEquals(res.value(), row); - } - LOG.info("Obtained row:" - + Bytes.toString(row) + ". IsDeleteReplication:" + isDeleted); + assertArrayEquals(res.value(), row); break; } } @@ -413,14 +301,14 @@ public class TestMasterReplication { @Override public void prePut(final ObserverContext e, - final Put put, final WALEdit edit, + final Put put, final WALEdit edit, final boolean writeToWAL) throws IOException { nCount++; } @Override public void postDelete(final ObserverContext c, - final Delete delete, final WALEdit edit, + final Delete delete, final WALEdit edit, final boolean writeToWAL) throws IOException { nDelete++; @@ -440,6 +328,6 @@ public class TestMasterReplication { @org.junit.Rule public org.apache.hadoop.hbase.ResourceCheckerJUnitRule cu = - new org.apache.hadoop.hbase.ResourceCheckerJUnitRule(); + new org.apache.hadoop.hbase.ResourceCheckerJUnitRule(); } diff --git a/src/test/java/org/apache/hadoop/hbase/replication/TestMultiSlaveReplication.java b/src/test/java/org/apache/hadoop/hbase/replication/TestMultiSlaveReplication.java index 1672d97..c631ebf 100644 --- a/src/test/java/org/apache/hadoop/hbase/replication/TestMultiSlaveReplication.java +++ b/src/test/java/org/apache/hadoop/hbase/replication/TestMultiSlaveReplication.java @@ -48,7 +48,9 @@ import org.junit.experimental.categories.Category; public class TestMultiSlaveReplication { private static final Log LOG = LogFactory.getLog(TestReplicationBase.class); + protected static ReplicationZookeeper.PeerProtocol protocol = ReplicationZookeeper.PeerProtocol.NATIVE; + protected static Configuration baseConf = HBaseConfiguration.create(); private static Configuration conf1; private static Configuration conf2; private static Configuration conf3; @@ -71,7 +73,7 @@ public class TestMultiSlaveReplication { @BeforeClass public static void setUpBeforeClass() throws Exception { - conf1 = HBaseConfiguration.create(); + conf1 = baseConf; conf1.set(HConstants.ZOOKEEPER_ZNODE_PARENT, "/1"); // smaller block size and capacity to trigger more operations // and test them @@ -131,7 +133,7 @@ public class TestMultiSlaveReplication { HTable htable3 = new HTable(conf3, tableName); htable3.setWriteBufferSize(1024); - admin1.addPeer("1", utility2.getClusterKey()); + admin1.addPeer("1", utility2.getClusterKey(), protocol.name()); // put "row" and wait 'til it got around, then delete putAndWait(row, famName, htable1, htable2); @@ -146,7 +148,7 @@ public class TestMultiSlaveReplication { // after the log was rolled put a new row putAndWait(row3, famName, htable1, htable2); - admin1.addPeer("2", utility3.getClusterKey()); + admin1.addPeer("2", utility3.getClusterKey(), protocol.name()); // put a row, check it was replicated to all clusters putAndWait(row1, famName, htable1, htable2, htable3); diff --git a/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationBase.java b/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationBase.java index aabaedc..b2d6013 100644 --- a/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationBase.java +++ b/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationBase.java @@ -35,6 +35,9 @@ import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; import org.junit.AfterClass; import org.junit.BeforeClass; +import java.io.IOException; +import java.net.ServerSocket; + /** * This class is only a base for other integration-level replication tests. * Do not add tests here. @@ -45,6 +48,7 @@ public class TestReplicationBase { private static final Log LOG = LogFactory.getLog(TestReplicationBase.class); + protected static ReplicationZookeeper.PeerProtocol protocol = ReplicationZookeeper.PeerProtocol.NATIVE; protected static Configuration conf1 = HBaseConfiguration.create(); protected static Configuration conf2; protected static Configuration CONF_WITH_LOCALFS; @@ -111,7 +115,8 @@ public class TestReplicationBase { utility2.setZkCluster(miniZK); zkw2 = new ZooKeeperWatcher(conf2, "cluster2", null, true); - admin.addPeer("2", utility2.getClusterKey()); + admin.addPeer("2", utility2.getClusterKey(), protocol.name()); + LOG.info("Using replication protocol: " + protocol); setIsReplication(true); LOG.info("Setup second Zk"); diff --git a/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSmallTests.java b/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSmallTests.java index 1b47ab6..a05edba 100644 --- a/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSmallTests.java +++ b/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSmallTests.java @@ -405,7 +405,7 @@ public class TestReplicationSmallTests extends TestReplicationBase { } } - admin.addPeer("2", utility2.getClusterKey()); + admin.addPeer("2", utility2.getClusterKey(), protocol.name()); Thread.sleep(SLEEP_TIME); rowKey = Bytes.toBytes("do rep"); put = new Put(rowKey); diff --git a/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationZookeeper.java b/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationZookeeper.java index 553b5cd..ca8bb7a 100644 --- a/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationZookeeper.java +++ b/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationZookeeper.java @@ -75,6 +75,12 @@ public class TestReplicationZookeeper { assertEquals(0, repZk.getSlavesAddresses("1").size()); } + @Test + public void testDifferentProtocols() throws Exception { + repZk.addPeer("2", slaveClusterKey, ReplicationZookeeper.PeerProtocol.THRIFT); + assertEquals(ReplicationZookeeper.PeerProtocol.THRIFT, repZk.getPeerProtocol("2")); + } + static class DummyServer implements Server { @Override diff --git a/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSink.java b/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSink.java index 18eb530..a400636 100644 --- a/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSink.java +++ b/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSink.java @@ -22,12 +22,14 @@ package org.apache.hadoop.hbase.replication.regionserver; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; +import java.util.UUID; import java.util.concurrent.atomic.AtomicBoolean; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.*; +import org.apache.hadoop.hbase.catalog.CatalogTracker; import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Result; @@ -37,6 +39,7 @@ import org.apache.hadoop.hbase.regionserver.wal.HLog; import org.apache.hadoop.hbase.regionserver.wal.HLogKey; import org.apache.hadoop.hbase.regionserver.wal.WALEdit; import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; import org.junit.AfterClass; import org.junit.Before; import org.junit.BeforeClass; @@ -62,7 +65,7 @@ public class TestReplicationSink { private static final byte[] FAM_NAME2 = Bytes.toBytes("info2"); private static HTable table1; - private static Stoppable STOPPABLE = new Stoppable() { + private static Server SERVER = new Server() { final AtomicBoolean stop = new AtomicBoolean(false); @Override @@ -75,7 +78,36 @@ public class TestReplicationSink { LOG.info("STOPPING BECAUSE: " + why); this.stop.set(true); } - + + @Override + public Configuration getConfiguration() { + return null; + } + + @Override + public ZooKeeperWatcher getZooKeeper() { + return null; + } + + @Override + public CatalogTracker getCatalogTracker() { + return null; + } + + @Override + public ServerName getServerName() { + return new ServerName("foo",9,-1); + } + + @Override + public void abort(String why, Throwable e) { + //To change body of implemented methods use File | Settings | File Templates. + } + + @Override + public boolean isAborted() { + return false; //To change body of implemented methods use File | Settings | File Templates. + } }; private static HTable table2; @@ -89,7 +121,8 @@ public class TestReplicationSink { TEST_UTIL.getConfiguration().setBoolean(HConstants.REPLICATION_ENABLE_KEY, true); TEST_UTIL.startMiniCluster(3); SINK = - new ReplicationSink(new Configuration(TEST_UTIL.getConfiguration()), STOPPABLE); + new ReplicationSink(new Configuration(TEST_UTIL.getConfiguration()), SERVER, + UUID.randomUUID().toString()); table1 = TEST_UTIL.createTable(TABLE_NAME1, FAM_NAME1); table2 = TEST_UTIL.createTable(TABLE_NAME2, FAM_NAME2); } @@ -99,7 +132,7 @@ public class TestReplicationSink { */ @AfterClass public static void tearDownAfterClass() throws Exception { - STOPPABLE.stop("Shutting down"); + SERVER.stop("Shutting down"); TEST_UTIL.shutdownMiniCluster(); } diff --git a/src/test/java/org/apache/hadoop/hbase/replication/thrift/ReplicationTestUtils.java b/src/test/java/org/apache/hadoop/hbase/replication/thrift/ReplicationTestUtils.java new file mode 100644 index 0000000..98c6b10 --- /dev/null +++ b/src/test/java/org/apache/hadoop/hbase/replication/thrift/ReplicationTestUtils.java @@ -0,0 +1,167 @@ +package org.apache.hadoop.hbase.replication.thrift; + +import com.google.common.base.Throwables; +import com.google.common.collect.Iterables; +import com.google.common.collect.Lists; +import com.google.common.collect.Sets; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.*; +import org.apache.hadoop.hbase.client.*; +import org.apache.hadoop.hbase.client.replication.ReplicationAdmin; +import org.apache.hadoop.hbase.replication.regionserver.ReplicationSink; +import org.apache.hadoop.hbase.util.Bytes; + +import java.io.IOException; +import java.util.List; +import java.util.Set; +import java.util.UUID; + +import static com.google.common.collect.Iterables.getOnlyElement; +import static org.apache.hadoop.hbase.replication.ReplicationZookeeper.*; +import static org.junit.Assert.assertArrayEquals; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.fail; + +public class ReplicationTestUtils { + + + private static final Log LOG = LogFactory.getLog(ReplicationTestUtils.class); + public static final byte[] DEFAULT_FAMILY = Bytes.toBytes("test_family"); + public static final byte[] DEFAULT_QUALIFIER = Bytes.toBytes("test_qual"); + static final long SLEEP_TIME = 100; + static final int NB_RETRIES = 200; + + public static Configuration setupConfiguration(HBaseTestingUtility cluster, int thriftServerPort) { + Configuration configuration = cluster.getConfiguration(); + configuration.setInt("hbase.replication.thrift.server.port", thriftServerPort); + configuration.setBoolean(ReplicationSink.CONF_KEY_REPLICATION_THRIFT, true); + configuration.setBoolean("hbase.replication", true); + return configuration; + } + + public static void addPeerThriftPort(HBaseTestingUtility cluster, String peerId, int port) { + cluster.getConfiguration().setInt("hbase.replication.thrift.peer." + peerId + ".port", port); + } + + public static HTableDescriptor createTestTable() throws Exception { + HTableDescriptor table = new HTableDescriptor("test_table"); + HColumnDescriptor fam = new HColumnDescriptor(DEFAULT_FAMILY); + fam.setScope(HConstants.REPLICATION_SCOPE_GLOBAL); + fam.setMaxVersions(1); + table.addFamily(fam); + return table; + } + + public static void createTableOnCluster(HBaseTestingUtility cluster, HTableDescriptor table) + throws IOException { + new HBaseAdmin(cluster.getConfiguration()).createTable(table); + } + + public static HTable getTestTable(HBaseTestingUtility cluster, HTableDescriptor table) + throws IOException { + HTable result = new HTable(cluster.getConfiguration(), table.getName()); + result.setWriteBufferSize(1024); + return result; + } + + public static void addReplicationPeer(String peerId, HBaseTestingUtility source, + HBaseTestingUtility destination) throws IOException { + Configuration configuration = source.getConfiguration(); + + ReplicationAdmin admin = new ReplicationAdmin(configuration); + admin.addPeer(peerId, destination.getClusterKey(), PeerProtocol.THRIFT.name()); + } + + public static Result putAndWait(Put put, String value, HTable source, HTable target) throws Exception { + return putAndWait(put, value, false, source, target); + } + + public static Result putAndWait(Put put, String value, boolean compareTimestamps, HTable source, HTable target) throws Exception { + source.put(put); + + Get get = new Get(put.getRow()); + for (int i = 0; i < NB_RETRIES; i++) { + Result res = target.get(get); + if (res.size() == 0) { + LOG.info("Row not available"); + Thread.sleep(SLEEP_TIME); + } else { + if (compareTimestamps) { + if (getOnlyElement(res.getColumn(DEFAULT_FAMILY, DEFAULT_QUALIFIER)).getTimestamp() + != getOnlyElement(put.get(DEFAULT_FAMILY, DEFAULT_QUALIFIER)).getTimestamp()) { + LOG.info("Cell timestamps don't match... wait some more"); + Thread.sleep(SLEEP_TIME); + continue; + } + assertArrayEquals(res.value(), Bytes.toBytes(value)); + } + return res; + } + } + throw new RuntimeException("Waited too much time for put replication"); + } + + + public static void deleteAndWait(byte[] row, HTable source, HTable target) + throws Exception { + Delete del = new Delete(row); + source.delete(del); + + Get get = new Get(row); + for (int i = 0; i < NB_RETRIES; i++) { + if (i==NB_RETRIES-1) { + fail("Waited too much time for del replication"); + } + Result res = target.get(get); + if (res.size() >= 1) { + LOG.info("Row not deleted"); + Thread.sleep(SLEEP_TIME); + } else { + break; + } + } + } + + public static void assertContainsOnly(HTable table, Set values) throws Exception { + for (int i = 0; i < NB_RETRIES; i++) { + Set valuesCopy = Sets.newHashSet(values); + ResultScanner scanner = table.getScanner(new Scan()); + Result result; + int tableSize = 0; + int valuesSize = valuesCopy.size(); + List inTableNotExpected = Lists.newArrayList(); + while ((result = scanner.next()) != null) { + String value = Bytes.toString(result.getValue(DEFAULT_FAMILY, DEFAULT_QUALIFIER)); + boolean removed = valuesCopy.remove(value); + if (!removed) { + inTableNotExpected.add(value); + } + tableSize++; + } + if (!valuesCopy.isEmpty()) { + LOG.warn("Table did not have expected values: " + valuesCopy); + Thread.sleep(SLEEP_TIME); + } else if (tableSize != valuesSize) { + LOG.warn("Table had more values (" + tableSize + ") than expected: " + inTableNotExpected); + Thread.sleep(SLEEP_TIME); + } else { + return; + } + } + fail("Waited too much time for replication to sync up"); + } + + public static Put generateRandomPut(String value) { + return generateRandomPut(value, System.currentTimeMillis()); + } + + public static Put generateRandomPut(String value, long timestamp) { + Put put = new Put(Bytes.toBytes(UUID.randomUUID().toString())); + put.add(DEFAULT_FAMILY, DEFAULT_QUALIFIER, timestamp, Bytes.toBytes(value)); + return put; + } + + +} diff --git a/src/test/java/org/apache/hadoop/hbase/replication/thrift/TestThriftCyclicReplication.java b/src/test/java/org/apache/hadoop/hbase/replication/thrift/TestThriftCyclicReplication.java new file mode 100644 index 0000000..a3a9d4a --- /dev/null +++ b/src/test/java/org/apache/hadoop/hbase/replication/thrift/TestThriftCyclicReplication.java @@ -0,0 +1,114 @@ +package org.apache.hadoop.hbase.replication.thrift; + +import com.google.common.collect.Iterables; +import com.google.common.collect.Lists; +import com.google.common.collect.Sets; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.HTableDescriptor; +import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.MediumTests; +import org.apache.hadoop.hbase.client.*; +import org.apache.hadoop.hbase.util.Bytes; +import org.junit.*; +import org.junit.experimental.categories.Category; + +import java.io.IOException; +import java.util.*; + +import static org.apache.hadoop.hbase.replication.thrift.ReplicationTestUtils.*; +import static org.junit.Assert.*; + +@Category(MediumTests.class) +public class TestThriftCyclicReplication extends TestThriftReplicationBase { + + private final static HBaseTestingUtility clusterA = new HBaseTestingUtility(); + private final static HBaseTestingUtility clusterB = new HBaseTestingUtility(); + private final static HBaseTestingUtility clusterC = new HBaseTestingUtility(); + + private static HTable tableA; + private static HTable tableB; + private static HTable tableC; + + @BeforeClass + public static void setUpClazz() throws Exception { + + int clusterAServerPort = HBaseTestingUtility.randomFreePort(); + int clusterBServerPort = HBaseTestingUtility.randomFreePort(); + int clusterCServerPort = HBaseTestingUtility.randomFreePort(); + + setupConfiguration(clusterA, clusterAServerPort); + setupConfiguration(clusterB, clusterBServerPort); + setupConfiguration(clusterC, clusterCServerPort); + + addPeerThriftPort(clusterA, "1", clusterBServerPort); + addPeerThriftPort(clusterB, "1", clusterCServerPort); + addPeerThriftPort(clusterC, "1", clusterAServerPort); + + HTableDescriptor table = createTestTable(); + + clusterA.startMiniCluster(); + clusterB.startMiniCluster(); + clusterC.startMiniCluster(); + + createTableOnCluster(clusterA, table); + createTableOnCluster(clusterB, table); + createTableOnCluster(clusterC, table); + + tableA = getTestTable(clusterA, table); + tableB = getTestTable(clusterB, table); + tableC = getTestTable(clusterC, table); + + addReplicationPeer("1", clusterA, clusterB); + addReplicationPeer("1", clusterB, clusterC); + addReplicationPeer("1", clusterC, clusterA); + } + + @Before + public void setUp() throws Exception { + clusterA.truncateTable(tableA.getTableName()); + clusterB.truncateTable(tableB.getTableName()); + + } + + @AfterClass + public static void tearDown() throws Exception { + clusterA.shutdownMiniCluster(); + clusterB.shutdownMiniCluster(); + clusterC.shutdownMiniCluster(); + } + + @Test + public void testCyclicReplication() throws Exception { + String firstRow = "firstRow"; + putAndWait(generateRandomPut(firstRow), firstRow, tableA, tableB); + + String secondRow = "secondRow"; + putAndWait(generateRandomPut(secondRow), secondRow, tableB, tableC); + + String thirdRow = "thirdRow"; + Put lastPut = generateRandomPut(thirdRow); + putAndWait(lastPut, thirdRow, tableC, tableA); + + Set expected = Sets.newHashSet(firstRow, secondRow, thirdRow); + assertContainsOnly(tableA, expected); + assertContainsOnly(tableB, expected); + assertContainsOnly(tableC, expected); + + // lets delete one of those rows and verify it goes around + + Delete delete = new Delete(lastPut.getRow()); + deleteAndWait(delete.getRow(), tableB, tableC); + + expected = Sets.newHashSet(firstRow, secondRow); + assertContainsOnly(tableB, expected); + assertContainsOnly(tableC, expected); + assertContainsOnly(tableA, expected); + } + + @org.junit.Rule + public org.apache.hadoop.hbase.ResourceCheckerJUnitRule cu = + new org.apache.hadoop.hbase.ResourceCheckerJUnitRule(); + +} diff --git a/src/test/java/org/apache/hadoop/hbase/replication/thrift/TestThriftMasterMasterReplication.java b/src/test/java/org/apache/hadoop/hbase/replication/thrift/TestThriftMasterMasterReplication.java new file mode 100644 index 0000000..1979554 --- /dev/null +++ b/src/test/java/org/apache/hadoop/hbase/replication/thrift/TestThriftMasterMasterReplication.java @@ -0,0 +1,94 @@ +package org.apache.hadoop.hbase.replication.thrift; + +import com.google.common.collect.Iterables; +import com.google.common.collect.Sets; +import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.HTableDescriptor; +import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.MediumTests; +import org.apache.hadoop.hbase.client.Delete; +import org.apache.hadoop.hbase.client.HTable; +import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.util.Bytes; +import org.junit.*; +import org.junit.experimental.categories.Category; + +import java.util.Iterator; +import java.util.Set; + +import static org.apache.hadoop.hbase.replication.thrift.ReplicationTestUtils.*; +import static org.junit.Assert.assertEquals; + +@Category(MediumTests.class) +public class TestThriftMasterMasterReplication extends TestThriftReplicationBase { + + private final static HBaseTestingUtility clusterA = new HBaseTestingUtility(); + private final static HBaseTestingUtility clusterB = new HBaseTestingUtility(); + + private static HTable tableA; + private static HTable tableB; + + @BeforeClass + public static void setUpClazz() throws Exception { + + int clusterAServerPort = HBaseTestingUtility.randomFreePort(); + int clusterBServerPort = HBaseTestingUtility.randomFreePort(); + + setupConfiguration(clusterA, clusterAServerPort); + setupConfiguration(clusterB, clusterBServerPort); + + addPeerThriftPort(clusterA, "1", clusterBServerPort); + addPeerThriftPort(clusterB, "1", clusterAServerPort); + + HTableDescriptor table = createTestTable(); + + clusterA.startMiniCluster(); + clusterB.startMiniCluster(); + + createTableOnCluster(clusterA, table); + createTableOnCluster(clusterB, table); + + tableA = getTestTable(clusterA, table); + tableB = getTestTable(clusterB, table); + + addReplicationPeer("1", clusterA, clusterB); + addReplicationPeer("1", clusterB, clusterA); + } + + @Before + public void setUp() throws Exception { + clusterA.truncateTable(tableA.getTableName()); + clusterB.truncateTable(tableB.getTableName()); + + } + + @AfterClass + public static void tearDown() throws Exception { + clusterA.shutdownMiniCluster(); + clusterB.shutdownMiniCluster(); + } + + @Test + public void testMasterMasterReplication() throws Exception { + long originalTimestamp = 1l; + String rowKey = "master-master-key"; + String value = "testMasterMaster"; + Put originalPut = new Put(Bytes.toBytes(rowKey)); + originalPut.add(DEFAULT_FAMILY, DEFAULT_QUALIFIER, originalTimestamp, Bytes.toBytes(value)); + Result originalResult = putAndWait(originalPut, value, true, tableA, tableB); + KeyValue originalKeyVal = + Iterables.getOnlyElement(originalResult.getColumn(DEFAULT_FAMILY, DEFAULT_QUALIFIER)); + assertEquals(originalTimestamp, originalKeyVal.getTimestamp()); + + long newTimestamp = 2l; + Put overwritePut = new Put(Bytes.toBytes(rowKey)); + overwritePut.add(DEFAULT_FAMILY, DEFAULT_QUALIFIER, newTimestamp, Bytes.toBytes(value)); + putAndWait(overwritePut, value, true, tableB, tableA); + } + + @Rule + public org.apache.hadoop.hbase.ResourceCheckerJUnitRule cu = + new org.apache.hadoop.hbase.ResourceCheckerJUnitRule(); + +} diff --git a/src/test/java/org/apache/hadoop/hbase/replication/thrift/TestThriftMultiSlaveReplication.java b/src/test/java/org/apache/hadoop/hbase/replication/thrift/TestThriftMultiSlaveReplication.java new file mode 100644 index 0000000..93cf376 --- /dev/null +++ b/src/test/java/org/apache/hadoop/hbase/replication/thrift/TestThriftMultiSlaveReplication.java @@ -0,0 +1,123 @@ +/** + * 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.hbase.replication.thrift; + +import com.google.common.collect.Sets; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.HTableDescriptor; +import org.apache.hadoop.hbase.MediumTests; +import org.apache.hadoop.hbase.client.Delete; +import org.apache.hadoop.hbase.client.HTable; +import org.apache.hadoop.hbase.client.Put; +import org.junit.*; +import org.junit.experimental.categories.Category; + +import java.util.Iterator; +import java.util.Set; + +import static org.apache.hadoop.hbase.replication.thrift.ReplicationTestUtils.*; +import static org.apache.hadoop.hbase.replication.thrift.ReplicationTestUtils.addReplicationPeer; +@Category(MediumTests.class) +public class TestThriftMultiSlaveReplication extends TestThriftReplicationBase { + + private static final Log LOG = LogFactory.getLog(TestThriftCyclicReplication.class); + + + private final static HBaseTestingUtility clusterA = new HBaseTestingUtility(); + private final static HBaseTestingUtility clusterB = new HBaseTestingUtility(); + private final static HBaseTestingUtility clusterC = new HBaseTestingUtility(); + + private static HTable tableA; + private static HTable tableB; + private static HTable tableC; + + @BeforeClass + public static void setUp() throws Exception { + + int clusterAServerPort = HBaseTestingUtility.randomFreePort(); + int clusterBServerPort = HBaseTestingUtility.randomFreePort(); + int clusterCServerPort = HBaseTestingUtility.randomFreePort(); + + setupConfiguration(clusterA, clusterAServerPort); + setupConfiguration(clusterB, clusterBServerPort); + setupConfiguration(clusterC, clusterCServerPort); + + addPeerThriftPort(clusterA, "1", clusterBServerPort); + addPeerThriftPort(clusterA, "2", clusterCServerPort); + + HTableDescriptor table = createTestTable(); + + clusterA.startMiniCluster(); + clusterB.startMiniCluster(); + clusterC.startMiniCluster(); + + createTableOnCluster(clusterA, table); + createTableOnCluster(clusterB, table); + createTableOnCluster(clusterC, table); + + tableA = getTestTable(clusterA, table); + tableB = getTestTable(clusterB, table); + tableC = getTestTable(clusterC, table); + + addReplicationPeer("1", clusterA, clusterB); + addReplicationPeer("2", clusterA, clusterC); + } + + @Test + public void testReplicatingToMultipleSlaves() throws Exception { + String firstRow = "firstRow"; + putAndWait(generateRandomPut(firstRow), firstRow, tableA, tableB); + + // make sure all tables have that one element only + Set expected = Sets.newHashSet(firstRow); + assertContainsOnly(tableA, expected); + assertContainsOnly(tableB, expected); + assertContainsOnly(tableC, expected); + + String secondRow = "secondRow"; + Put lastPut = generateRandomPut(secondRow); + putAndWait(lastPut, secondRow, tableA, tableB); + + // all tables should have both elements + Set otherExpected = Sets.newHashSet(firstRow, secondRow); + assertContainsOnly(tableA, otherExpected); + assertContainsOnly(tableB, otherExpected); + assertContainsOnly(tableC, otherExpected); + + // lets delete one of those rows and verify it goes replicated to both slaves + Delete delete = new Delete(lastPut.getRow()); + deleteAndWait(delete.getRow(), tableA, tableB); + + assertContainsOnly(tableA, expected); + assertContainsOnly(tableB, expected); + assertContainsOnly(tableC, expected); + } + + @AfterClass + public static void tearDown() throws Exception { + clusterA.shutdownMiniCluster(); + clusterB.shutdownMiniCluster(); + clusterC.shutdownMiniCluster(); + } + + @org.junit.Rule + public org.apache.hadoop.hbase.ResourceCheckerJUnitRule cu = + new org.apache.hadoop.hbase.ResourceCheckerJUnitRule(); +} diff --git a/src/test/java/org/apache/hadoop/hbase/replication/thrift/TestThriftReplicationBase.java b/src/test/java/org/apache/hadoop/hbase/replication/thrift/TestThriftReplicationBase.java new file mode 100644 index 0000000..e111df6 --- /dev/null +++ b/src/test/java/org/apache/hadoop/hbase/replication/thrift/TestThriftReplicationBase.java @@ -0,0 +1,14 @@ +package org.apache.hadoop.hbase.replication.thrift; + +import org.junit.BeforeClass; + +public class TestThriftReplicationBase { + + @BeforeClass + public static void setUpKlazz() throws Exception { + // Error level to skip some warnings specific to the minicluster. See HBASE-4709 + org.apache.log4j.Logger.getLogger(org.apache.hadoop.metrics2.util.MBeans.class).setLevel( org.apache.log4j.Level.ERROR); + org.apache.log4j.Logger.getLogger(org.apache.hadoop.metrics2.impl.MetricsSystemImpl.class).setLevel(org.apache.log4j.Level.ERROR); + + } +} diff --git a/src/test/java/org/apache/hadoop/hbase/replication/thrift/TestThriftReplicationSink.java b/src/test/java/org/apache/hadoop/hbase/replication/thrift/TestThriftReplicationSink.java new file mode 100644 index 0000000..4b7d307 --- /dev/null +++ b/src/test/java/org/apache/hadoop/hbase/replication/thrift/TestThriftReplicationSink.java @@ -0,0 +1,275 @@ +package org.apache.hadoop.hbase.replication.thrift; + +import com.google.common.base.Throwables; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.*; +import org.apache.hadoop.hbase.catalog.CatalogTracker; +import org.apache.hadoop.hbase.client.*; +import org.apache.hadoop.hbase.regionserver.wal.HLog; +import org.apache.hadoop.hbase.regionserver.wal.HLogKey; +import org.apache.hadoop.hbase.regionserver.wal.WALEdit; +import org.apache.hadoop.hbase.replication.regionserver.ReplicationSink; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; +import org.junit.AfterClass; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +import java.net.InetAddress; +import java.net.UnknownHostException; +import java.util.UUID; +import java.util.concurrent.atomic.AtomicBoolean; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; +@Category(MediumTests.class) +public class TestThriftReplicationSink extends TestThriftReplicationBase { + + private static final Log LOG = LogFactory.getLog(TestThriftReplicationSink.class); + private static final int BATCH_SIZE = 10; + + private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); + + private static ReplicationSink SINK; + + private static final byte[] TABLE_NAME1 = Bytes.toBytes("table1"); + private static final byte[] TABLE_NAME2 = Bytes.toBytes("table2"); + + private static final byte[] FAM_NAME1 = Bytes.toBytes("info1"); + private static final byte[] FAM_NAME2 = Bytes.toBytes("info2"); + + private static HTable table1; + private static HTable table2; + + /** + * @throws java.lang.Exception + */ + @BeforeClass + public static void setUpBeforeClass() throws Exception { + TEST_UTIL.getConfiguration().setBoolean("dfs.support.append", true); + TEST_UTIL.getConfiguration().setBoolean(HConstants.REPLICATION_ENABLE_KEY, true); + ReplicationTestUtils.setupConfiguration(TEST_UTIL, HBaseTestingUtility.randomFreePort()); + TEST_UTIL.startMiniCluster(1); + + Configuration sinkConfiguration = new Configuration(TEST_UTIL.getConfiguration()); + sinkConfiguration.setInt("hbase.replication.thrift.server.port", HBaseTestingUtility.randomFreePort()); + SINK = new ReplicationSink(sinkConfiguration, SERVER, UUID.randomUUID().toString()); + table1 = TEST_UTIL.createTable(TABLE_NAME1, FAM_NAME1); + table2 = TEST_UTIL.createTable(TABLE_NAME2, FAM_NAME2); + } + + /** + * @throws java.lang.Exception + */ + @AfterClass + public static void tearDownAfterClass() throws Exception { + SERVER.stop("Shutting down"); + TEST_UTIL.shutdownMiniCluster(); + } + + /** + * @throws java.lang.Exception + */ + @Before + public void setUp() throws Exception { + table1 = TEST_UTIL.truncateTable(TABLE_NAME1); + table2 = TEST_UTIL.truncateTable(TABLE_NAME2); + } + + /** + * Insert a whole batch of entries + * @throws Exception + */ + @Test + public void testBatchSink() throws Exception { + HLog.Entry[] entries = new HLog.Entry[BATCH_SIZE]; + for(int i = 0; i < BATCH_SIZE; i++) { + entries[i] = createEntry(TABLE_NAME1, i, KeyValue.Type.Put); + } + SINK.replicate(ThriftAdaptors.REPLICATION_BATCH_ADAPTOR.toThrift(entries)); + Scan scan = new Scan(); + ResultScanner scanRes = table1.getScanner(scan); + assertEquals(BATCH_SIZE, scanRes.next(BATCH_SIZE).length); + } + + /** + * Insert a mix of puts and deletes + * @throws Exception + */ + @Test + public void testMixedPutDelete() throws Exception { + HLog.Entry[] entries = new HLog.Entry[BATCH_SIZE/2]; + for(int i = 0; i < BATCH_SIZE/2; i++) { + entries[i] = createEntry(TABLE_NAME1, i, KeyValue.Type.Put); + } + SINK.replicate(ThriftAdaptors.REPLICATION_BATCH_ADAPTOR.toThrift(entries)); + + entries = new HLog.Entry[BATCH_SIZE]; + for(int i = 0; i < BATCH_SIZE; i++) { + entries[i] = createEntry(TABLE_NAME1, i, + i % 2 != 0 ? KeyValue.Type.Put: KeyValue.Type.DeleteColumn); + } + + SINK.replicate(ThriftAdaptors.REPLICATION_BATCH_ADAPTOR.toThrift(entries)); + Scan scan = new Scan(); + ResultScanner scanRes = table1.getScanner(scan); + assertEquals(BATCH_SIZE/2, scanRes.next(BATCH_SIZE).length); + } + + /** + * Insert to 2 different tables + * @throws Exception + */ + @Test + public void testMixedPutTables() throws Exception { + HLog.Entry[] entries = new HLog.Entry[BATCH_SIZE]; + for(int i = 0; i < BATCH_SIZE; i++) { + entries[i] = + createEntry( i % 2 == 0 ? TABLE_NAME2 : TABLE_NAME1, + i, KeyValue.Type.Put); + } + + SINK.replicate(ThriftAdaptors.REPLICATION_BATCH_ADAPTOR.toThrift(entries)); + Scan scan = new Scan(); + ResultScanner scanRes = table2.getScanner(scan); + for(Result res : scanRes) { + assertTrue(Bytes.toInt(res.getRow()) % 2 == 0); + } + } + + /** + * Insert then do different types of deletes + * @throws Exception + */ + @Test + public void testMixedDeletes() throws Exception { + HLog.Entry[] entries = new HLog.Entry[3]; + for(int i = 0; i < 3; i++) { + entries[i] = createEntry(TABLE_NAME1, i, KeyValue.Type.Put); + } + SINK.replicate(ThriftAdaptors.REPLICATION_BATCH_ADAPTOR.toThrift(entries)); + entries = new HLog.Entry[3]; + + entries[0] = createEntry(TABLE_NAME1, 0, KeyValue.Type.DeleteColumn); + entries[1] = createEntry(TABLE_NAME1, 1, KeyValue.Type.DeleteFamily); + entries[2] = createEntry(TABLE_NAME1, 2, KeyValue.Type.DeleteColumn); + + SINK.replicate(ThriftAdaptors.REPLICATION_BATCH_ADAPTOR.toThrift(entries)); + + Scan scan = new Scan(); + ResultScanner scanRes = table1.getScanner(scan); + assertEquals(0, scanRes.next(3).length); + } + + /** + * Puts are buffered, but this tests when a delete (not-buffered) is applied + * before the actual Put that creates it. + * @throws Exception + */ + @Test + public void testApplyDeleteBeforePut() throws Exception { + HLog.Entry[] entries = new HLog.Entry[5]; + for(int i = 0; i < 2; i++) { + entries[i] = createEntry(TABLE_NAME1, i, KeyValue.Type.Put); + } + entries[2] = createEntry(TABLE_NAME1, 1, KeyValue.Type.DeleteFamily); + for(int i = 3; i < 5; i++) { + entries[i] = createEntry(TABLE_NAME1, i, KeyValue.Type.Put); + } + SINK.replicate(ThriftAdaptors.REPLICATION_BATCH_ADAPTOR.toThrift(entries)); + Get get = new Get(Bytes.toBytes(1)); + Result res = table1.get(get); + assertEquals(0, res.size()); + } + + private HLog.Entry createEntry(byte [] table, int row, KeyValue.Type type) { + byte[] fam = Bytes.equals(table, TABLE_NAME1) ? FAM_NAME1 : FAM_NAME2; + byte[] rowBytes = Bytes.toBytes(row); + // Just make sure we don't get the same ts for two consecutive rows with + // same key + try { + Thread.sleep(1); + } catch (InterruptedException e) { + LOG.info("Was interrupted while sleep, meh", e); + } + final long now = System.currentTimeMillis(); + KeyValue kv = null; + if(type.getCode() == KeyValue.Type.Put.getCode()) { + kv = new KeyValue(rowBytes, fam, fam, now, + KeyValue.Type.Put, Bytes.toBytes(row)); + } else if (type.getCode() == KeyValue.Type.DeleteColumn.getCode()) { + kv = new KeyValue(rowBytes, fam, fam, + now, KeyValue.Type.DeleteColumn); + } else if (type.getCode() == KeyValue.Type.DeleteFamily.getCode()) { + kv = new KeyValue(rowBytes, fam, null, + now, KeyValue.Type.DeleteFamily); + } + + HLogKey key = new HLogKey(table, table, now, now, + HConstants.DEFAULT_CLUSTER_ID); + + WALEdit edit = new WALEdit(); + edit.add(kv); + + return new HLog.Entry(key, edit); + } + + @org.junit.Rule + public org.apache.hadoop.hbase.ResourceCheckerJUnitRule cu = + new org.apache.hadoop.hbase.ResourceCheckerJUnitRule(); + + private static Server SERVER = new Server() { + final AtomicBoolean stop = new AtomicBoolean(false); + + @Override + public boolean isStopped() { + return this.stop.get(); + } + + @Override + public void stop(String why) { + LOG.info("STOPPING BECAUSE: " + why); + this.stop.set(true); + } + + @Override + public Configuration getConfiguration() { + return null; + } + + @Override + public ZooKeeperWatcher getZooKeeper() { + return null; + } + + @Override + public CatalogTracker getCatalogTracker() { + return null; + } + + @Override + public ServerName getServerName() { + try { + return new ServerName(InetAddress.getLocalHost().getHostName(),9,-1); + } catch (UnknownHostException e) { + throw Throwables.propagate(e); + } + } + + @Override + public void abort(String why, Throwable e) { + //To change body of implemented methods use File | Settings | File Templates. + } + + @Override + public boolean isAborted() { + return false; //To change body of implemented methods use File | Settings | File Templates. + } + }; + + +} diff --git a/src/test/java/org/apache/hadoop/hbase/replication/thrift/TestThriftReplicationSmallTests.java b/src/test/java/org/apache/hadoop/hbase/replication/thrift/TestThriftReplicationSmallTests.java new file mode 100644 index 0000000..42ab5d9 --- /dev/null +++ b/src/test/java/org/apache/hadoop/hbase/replication/thrift/TestThriftReplicationSmallTests.java @@ -0,0 +1,594 @@ +/** + * 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.hbase.replication.thrift; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hbase.*; +import org.apache.hadoop.hbase.client.*; +import org.apache.hadoop.hbase.client.replication.ReplicationAdmin; +import org.apache.hadoop.hbase.replication.TestReplicationSmallTests; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; +import org.apache.hadoop.hbase.util.JVMClusterUtil; +import org.junit.AfterClass; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +import java.io.IOException; +import java.util.HashMap; +import java.util.List; + +import static org.apache.hadoop.hbase.replication.thrift.ReplicationTestUtils.*; +import static org.junit.Assert.*; + +@Category(MediumTests.class) +public class TestThriftReplicationSmallTests extends TestThriftReplicationBase { + + private static final Log LOG = LogFactory.getLog(TestReplicationSmallTests.class); + + protected static final int NB_ROWS_IN_BATCH = 100; + protected static final int NB_ROWS_IN_BIG_BATCH = NB_ROWS_IN_BATCH * 10; + protected static final int NB_RETRIES_FOR_BIG_BATCH = 30; + + private final static HBaseTestingUtility clusterA = new HBaseTestingUtility(); + private final static HBaseTestingUtility clusterB = new HBaseTestingUtility(); + private static HTable tableA; + private static HTable tableB; + protected static final byte[] row = Bytes.toBytes("row"); + + @BeforeClass + public static void setUpClass() throws Exception { + int clusterAServerPort = HBaseTestingUtility.randomFreePort(); + int clusterBServerPort = HBaseTestingUtility.randomFreePort(); + + setupConfiguration(clusterA, clusterAServerPort); + setupConfiguration(clusterB, clusterBServerPort); + + addPeerThriftPort(clusterA, "1", clusterBServerPort); + addPeerThriftPort(clusterB, "1", clusterAServerPort); + + HTableDescriptor table = createTestTable(); + table.addFamily(new HColumnDescriptor("NO_REP_FAMILY")); + + clusterA.startMiniCluster(); + clusterB.startMiniCluster(); + + createTableOnCluster(clusterA, table); + createTableOnCluster(clusterB, table); + + tableA = getTestTable(clusterA, table); + tableB = getTestTable(clusterB, table); + + addReplicationPeer("1", clusterA, clusterB); + addReplicationPeer("1", clusterB, clusterA); + } + + @Before + public void setUp() throws Exception { + tableA.setAutoFlush(true); + tableB.setAutoFlush(true); + // Starting and stopping replication can make us miss new logs, + // rolling like this makes sure the most recent one gets added to the queue + for ( JVMClusterUtil.RegionServerThread r : + clusterA.getHBaseCluster().getRegionServerThreads()) { + r.getRegionServer().getWAL().rollWriter(); + } + clusterA.truncateTable(tableA.getTableName()); + // truncating the table will send one Delete per row to the slave cluster + // in an async fashion, which is why we cannot just call truncateTable on + // utility2 since late writes could make it to the slave in some way. + // Instead, we truncate the first table and wait for all the Deletes to + // make it to the slave. + Scan scan = new Scan(); + int lastCount = 0; + for (int i = 0; i < NB_RETRIES; i++) { + if (i==NB_RETRIES-1) { + fail("Waited too much time for truncate"); + } + ResultScanner scanner = tableB.getScanner(scan); + Result[] res = scanner.next(NB_ROWS_IN_BIG_BATCH); + scanner.close(); + if (res.length != 0) { + if (res.length < lastCount) { + i--; // Don't increment timeout if we make progress + } + lastCount = res.length; + LOG.info("Still got " + res.length + " rows"); + Thread.sleep(SLEEP_TIME); + } else { + break; + } + } + } + + @AfterClass + public static void tearDown() throws Exception { + clusterA.shutdownMiniCluster(); + clusterB.shutdownMiniCluster(); + } + + /** + * Verify that version and column delete marker types are replicated + * correctly. + * @throws Exception + */ + @Test(timeout=300000) + public void testDeleteTypes() throws Exception { + LOG.info("testDeleteTypes"); + final byte[] v1 = Bytes.toBytes("v1"); + final byte[] v2 = Bytes.toBytes("v2"); + final byte[] v3 = Bytes.toBytes("v3"); + + long t = EnvironmentEdgeManager.currentTimeMillis(); + // create three versions for "row" + Put put = new Put(row); + put.add(ReplicationTestUtils.DEFAULT_FAMILY, row, t, v1); + tableA.put(put); + + put = new Put(row); + put.add(ReplicationTestUtils.DEFAULT_FAMILY, row, t+1, v2); + tableA.put(put); + + put = new Put(row); + put.add(ReplicationTestUtils.DEFAULT_FAMILY, row, t+2, v3); + tableA.put(put); + + Get get = new Get(row); + get.setMaxVersions(); + for (int i = 0; i < NB_RETRIES; i++) { + if (i==NB_RETRIES-1) { + fail("Waited too much time for put replication"); + } + Result res = tableB.get(get); + if (res.size() < 3) { + LOG.info("Rows not available"); + Thread.sleep(SLEEP_TIME); + } else { + assertArrayEquals(res.raw()[0].getValue(), v3); + assertArrayEquals(res.raw()[1].getValue(), v2); + assertArrayEquals(res.raw()[2].getValue(), v1); + break; + } + } + // place a version delete marker (delete last version) + Delete d = new Delete(row); + d.deleteColumn(ReplicationTestUtils.DEFAULT_FAMILY, row, t); + tableA.delete(d); + + get = new Get(row); + get.setMaxVersions(); + for (int i = 0; i < NB_RETRIES; i++) { + if (i==NB_RETRIES-1) { + fail("Waited too much time for put replication"); + } + Result res = tableB.get(get); + if (res.size() > 2) { + LOG.info("Version not deleted"); + Thread.sleep(SLEEP_TIME); + } else { + assertArrayEquals(res.raw()[0].getValue(), v3); + assertArrayEquals(res.raw()[1].getValue(), v2); + break; + } + } + + // place a column delete marker + d = new Delete(row); + d.deleteColumns(DEFAULT_FAMILY, row, t+2); + tableA.delete(d); + + // now *both* of the remaining version should be deleted + // at the replica + get = new Get(row); + for (int i = 0; i < NB_RETRIES; i++) { + if (i==NB_RETRIES-1) { + fail("Waited too much time for del replication"); + } + Result res = tableB.get(get); + if (res.size() >= 1) { + LOG.info("Rows not deleted"); + Thread.sleep(SLEEP_TIME); + } else { + break; + } + } + } + + /** + * Add a row, check it's replicated, delete it, check's gone + * @throws Exception + */ + @Test(timeout=300000) + public void testSimplePutDelete() throws Exception { + LOG.info("testSimplePutDelete"); + Put put = new Put(row); + put.add(DEFAULT_FAMILY, row, row); + + tableA.put(put); + + Get get = new Get(row); + for (int i = 0; i < NB_RETRIES; i++) { + if (i==NB_RETRIES-1) { + fail("Waited too much time for put replication"); + } + Result res = tableB.get(get); + if (res.size() == 0) { + LOG.info("Row not available"); + Thread.sleep(SLEEP_TIME); + } else { + assertArrayEquals(res.value(), row); + break; + } + } + + Delete del = new Delete(row); + tableA.delete(del); + + get = new Get(row); + for (int i = 0; i < NB_RETRIES; i++) { + if (i==NB_RETRIES-1) { + fail("Waited too much time for del replication"); + } + Result res = tableB.get(get); + if (res.size() >= 1) { + LOG.info("Row not deleted"); + Thread.sleep(SLEEP_TIME); + } else { + break; + } + } + } + + /** + * Try a small batch upload using the write buffer, check it's replicated + * @throws Exception + */ + @Test(timeout=300000) + public void testSmallBatch() throws Exception { + LOG.info("testSmallBatch"); + Put put; + // normal Batch tests + tableA.setAutoFlush(false); + for (int i = 0; i < NB_ROWS_IN_BATCH; i++) { + put = new Put(Bytes.toBytes(i)); + put.add(DEFAULT_FAMILY, row, row); + tableA.put(put); + } + tableA.flushCommits(); + + Scan scan = new Scan(); + + ResultScanner scanner1 = tableA.getScanner(scan); + Result[] res1 = scanner1.next(NB_ROWS_IN_BATCH); + scanner1.close(); + assertEquals(NB_ROWS_IN_BATCH, res1.length); + + for (int i = 0; i < NB_RETRIES; i++) { + if (i==NB_RETRIES-1) { + fail("Waited too much time for normal batch replication"); + } + ResultScanner scanner = tableB.getScanner(scan); + Result[] res = scanner.next(NB_ROWS_IN_BATCH); + scanner.close(); + if (res.length != NB_ROWS_IN_BATCH) { + LOG.info("Only got " + res.length + " rows"); + Thread.sleep(SLEEP_TIME); + } else { + break; + } + } + } + + /** + * Test stopping replication, trying to insert, make sure nothing's + * replicated, enable it, try replicating and it should work + * @throws Exception + */ + @Test(timeout=300000) + public void testStartStop() throws Exception { + + // Test stopping replication + changeReplicationState(clusterA, false); + + Put put = new Put(Bytes.toBytes("stop start")); + put.add(DEFAULT_FAMILY, row, row); + tableA.put(put); + + Get get = new Get(Bytes.toBytes("stop start")); + for (int i = 0; i < NB_RETRIES; i++) { + if (i == NB_RETRIES - 1) { + break; + } + Result res = tableB.get(get); + if (res.size() >= 1) { + fail("Replication wasn't stopped"); + + } else { + LOG.info("Row not replicated, let's wait a bit more..."); + Thread.sleep(SLEEP_TIME); + } + } + + // Test restart replication + changeReplicationState(clusterA, true); + + tableA.put(put); + + for (int i = 0; i < NB_RETRIES; i++) { + if (i == NB_RETRIES - 1) { + fail("Waited too much time for put replication"); + } + Result res = tableB.get(get); + if (res.size() == 0) { + LOG.info("Row not available"); + Thread.sleep(SLEEP_TIME); + } else { + assertArrayEquals(res.value(), row); + break; + } + } + + put = new Put(Bytes.toBytes("do not rep")); + put.add("NO_REP_FAMILY".getBytes(), row, row); + tableA.put(put); + + get = new Get(Bytes.toBytes("do not rep")); + for (int i = 0; i < NB_RETRIES; i++) { + if (i == NB_RETRIES - 1) { + break; + } + Result res = tableB.get(get); + if (res.size() >= 1) { + fail("Not supposed to be replicated"); + } else { + LOG.info("Row not replicated, let's wait a bit more..."); + Thread.sleep(SLEEP_TIME); + } + } + + } + + + /** + * Test disable/enable replication, trying to insert, make sure nothing's + * replicated, enable it, the insert should be replicated + * + * @throws Exception + */ + @Test(timeout = 300000) + public void testDisableEnable() throws Exception { + + // Test disabling replication + ReplicationAdmin clusterAAdmin = getReplicationAdmin(clusterA); + clusterAAdmin.disablePeer("1"); + + byte[] rowkey = Bytes.toBytes("disable enable"); + Put put = new Put(rowkey); + put.add(DEFAULT_FAMILY, row, row); + tableA.put(put); + + Get get = new Get(rowkey); + for (int i = 0; i < NB_RETRIES; i++) { + Result res = tableB.get(get); + if (res.size() >= 1) { + fail("Replication wasn't disabled"); + } else { + LOG.info("Row not replicated, let's wait a bit more..."); + Thread.sleep(SLEEP_TIME); + } + } + + // Test enable replication + clusterAAdmin.enablePeer("1"); + + for (int i = 0; i < NB_RETRIES; i++) { + Result res = tableB.get(get); + if (res.size() == 0) { + LOG.info("Row not available"); + Thread.sleep(SLEEP_TIME); + } else { + assertArrayEquals(res.value(), row); + return; + } + } + fail("Waited too much time for put replication"); + } + + /** + * Integration test for TestReplicationAdmin, removes and re-add a peer + * cluster + * + * @throws Exception + */ + @Test(timeout=300000) + public void testAddAndRemoveClusters() throws Exception { + LOG.info("testAddAndRemoveClusters"); + ReplicationAdmin clusterAAdmin = getReplicationAdmin(clusterA); + clusterAAdmin.removePeer("1"); + Thread.sleep(SLEEP_TIME * 3); + byte[] rowKey = Bytes.toBytes("Won't be replicated"); + Put put = new Put(rowKey); + put.add(DEFAULT_FAMILY, row, row); + tableA.put(put); + + Get get = new Get(rowKey); + for (int i = 0; i < NB_RETRIES; i++) { + if (i == NB_RETRIES-1) { + break; + } + Result res = tableB.get(get); + if (res.size() >= 1) { + fail("Not supposed to be replicated"); + } else { + LOG.info("Row not replicated, let's wait a bit more..."); + Thread.sleep(SLEEP_TIME); + } + } + + ReplicationTestUtils.addReplicationPeer("1", clusterA, clusterB); + Thread.sleep(SLEEP_TIME*3); + rowKey = Bytes.toBytes("do rep"); + put = new Put(rowKey); + put.add(DEFAULT_FAMILY, row, row); + LOG.info("Adding new row"); + tableA.put(put); + + get = new Get(rowKey); + for (int i = 0; i < NB_RETRIES/2; i++) { + if (i==NB_RETRIES-1) { + fail("Waited too much time for put replication"); + } + Result res = tableB.get(get); + if (res.size() == 0) { + LOG.info("Row not available"); + Thread.sleep(SLEEP_TIME*i); + } else { + assertArrayEquals(res.value(), row); + break; + } + } + } + + + /** + * Do a more intense version testSmallBatch, one that will trigger + * hlog rolling and other non-trivial code paths + * @throws Exception + */ + @Test(timeout=300000) + public void loadTesting() throws Exception { + tableA.setWriteBufferSize(1024); + tableA.setAutoFlush(false); + for (int i = 0; i < NB_ROWS_IN_BIG_BATCH; i++) { + Put put = new Put(Bytes.toBytes(i)); + put.add(DEFAULT_FAMILY, row, row); + tableA.put(put); + } + tableA.flushCommits(); + + Scan scan = new Scan(); + + ResultScanner scanner = tableA.getScanner(scan); + Result[] res = scanner.next(NB_ROWS_IN_BIG_BATCH); + scanner.close(); + + assertEquals(NB_ROWS_IN_BIG_BATCH, res.length); + + scan = new Scan(); + + for (int i = 0; i < NB_RETRIES_FOR_BIG_BATCH; i++) { + + scanner = tableB.getScanner(scan); + res = scanner.next(NB_ROWS_IN_BIG_BATCH); + scanner.close(); + if (res.length != NB_ROWS_IN_BIG_BATCH) { + if (i == NB_RETRIES_FOR_BIG_BATCH-1) { + int lastRow = -1; + for (Result result : res) { + int currentRow = Bytes.toInt(result.getRow()); + for (int row = lastRow+1; row < currentRow; row++) { + LOG.error("Row missing: " + row); + } + lastRow = currentRow; + } + LOG.error("Last row: " + lastRow); + fail("Waited too much time for normal batch replication, " + + res.length + " instead of " + NB_ROWS_IN_BIG_BATCH); + } else { + LOG.info("Only got " + res.length + " rows"); + Thread.sleep(SLEEP_TIME); + } + } else { + break; + } + } + } + + /** + * Test for HBASE-8663 + * Create two new Tables with colfamilies enabled for replication then run + * ReplicationAdmin.listReplicated(). Finally verify the table:colfamilies. Note: + * TestReplicationAdmin is a better place for this testing but it would need mocks. + * @throws Exception + */ + @Test(timeout = 300000) + public void testVerifyListReplicatedTable() throws Exception { + LOG.info("testVerifyListReplicatedTable"); + + final String tName = "VerifyListReplicated_"; + final String colFam = "cf1"; + final int numOfTables = 3; + + HBaseAdmin hadmin = clusterA.getHBaseAdmin(); + + // Create Tables + for (int i = 0; i < numOfTables; i++) { + HTableDescriptor ht = new HTableDescriptor(tName + i); + HColumnDescriptor cfd = new HColumnDescriptor(colFam); + cfd.setScope(HConstants.REPLICATION_SCOPE_GLOBAL); + ht.addFamily(cfd); + hadmin.createTable(ht); + } + + // verify the result + List> replicationColFams = getReplicationAdmin(clusterA).listReplicated(); + int[] match = new int[numOfTables]; // array of 3 with init value of zero + + for (int i = 0; i < replicationColFams.size(); i++) { + HashMap replicationEntry = replicationColFams.get(i); + String tn = replicationEntry.get(ReplicationAdmin.TNAME); + if ((tn.startsWith(tName)) && replicationEntry.get(ReplicationAdmin.CFNAME).equals(colFam)) { + int m = Integer.parseInt(tn.substring(tn.length() - 1)); // get the last digit + match[m]++; // should only increase once + } + } + + // check the matching result + for (int i = 0; i < match.length; i++) { + assertTrue("listReplicated() does not match table " + i, (match[i] == 1)); + } + + // drop tables + for (int i = 0; i < numOfTables; i++) { + String ht = tName + i; + hadmin.disableTable(ht); + hadmin.deleteTable(ht); + } + + hadmin.close(); + } + + private void changeReplicationState(HBaseTestingUtility cluster, boolean newState) + throws IOException, InterruptedException { + new ReplicationAdmin(cluster.getConfiguration()).setReplicating(newState); + Thread.sleep(SLEEP_TIME*3); + } + + private ReplicationAdmin getReplicationAdmin(HBaseTestingUtility cluster) throws IOException { + return new ReplicationAdmin(cluster.getConfiguration()); + } + + @org.junit.Rule + public org.apache.hadoop.hbase.ResourceCheckerJUnitRule cu = + new org.apache.hadoop.hbase.ResourceCheckerJUnitRule(); + + +} diff --git a/src/test/java/org/apache/hadoop/hbase/replication/thrift/ThriftAdaptorsTest.java b/src/test/java/org/apache/hadoop/hbase/replication/thrift/ThriftAdaptorsTest.java new file mode 100644 index 0000000..4aa31c1 --- /dev/null +++ b/src/test/java/org/apache/hadoop/hbase/replication/thrift/ThriftAdaptorsTest.java @@ -0,0 +1,90 @@ +package org.apache.hadoop.hbase.replication.thrift; + +import com.google.common.base.Function; +import com.google.common.collect.Iterables; +import com.google.common.collect.Lists; +import org.apache.commons.lang.builder.EqualsBuilder; +import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.SmallTests; +import org.apache.hadoop.hbase.regionserver.wal.HLog; +import org.apache.hadoop.hbase.regionserver.wal.HLogKey; +import org.apache.hadoop.hbase.regionserver.wal.WALEdit; +import org.apache.hadoop.hbase.replication.thrift.generated.*; +import org.apache.hadoop.hbase.util.Bytes; +import org.hamcrest.CoreMatchers; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +import java.util.List; +import java.util.UUID; + +import static org.apache.hadoop.hbase.replication.thrift.ThriftAdaptors.*; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +@Category(SmallTests.class) +public class ThriftAdaptorsTest { + + @Test + public void testSerializingHLogEntryAndBack() throws Exception { + long now = System.currentTimeMillis(); + UUID uuid = UUID.randomUUID(); + + List keyValues = makeKeyValues(1, 10, "CF", "CQ", KeyValue.Type.Put, now); + + HLogKey key = createKey("region", "table", now, uuid); + WALEdit walEdit = createWALEdit(Lists.newArrayList(), keyValues); + HLog.Entry entry = new HLog.Entry( + key, + walEdit + ); + + TBatchEdit result = + REPLICATION_BATCH_ADAPTOR.toThrift(new HLog.Entry[] { entry }); + + assertEquals(1, result.getEditsSize()); + + // change this back to key values and compare each one to ensure they match + HLog.Entry[] entries = REPLICATION_BATCH_ADAPTOR.fromThrift(result); + HLog.Entry onlyEntry = entries[0]; + assertTrue(customHLogKeyEquality(onlyEntry.getKey(), key).isEquals()); + List transformedKeyValues = onlyEntry.getEdit().getKeyValues(); + assertThat(transformedKeyValues, CoreMatchers.hasItems( + keyValues.toArray(new KeyValue[keyValues.size()]))); + } + + private HLogKey createKey(String region, String table, long timestamp, UUID uuid) { + return new HLogKey(Bytes.toBytes(region), Bytes.toBytes(table), -1, timestamp, uuid); + } + + private WALEdit createWALEdit(List clusterIds, List keyValues) { + WALEdit edit = new WALEdit(); + edit.addClusterIds(clusterIds); + for (KeyValue keyValue : keyValues) { + edit.add(keyValue); + } + return edit; + } + + private List makeKeyValues(int from, int to, String family, String cq, + KeyValue.Type type, long ts) { + List result = Lists.newArrayList(); + while (from < to) { + byte[] rowkey = Bytes.toBytes("row-" + from); + result.add( + new KeyValue(rowkey, Bytes.toBytes(family), Bytes.toBytes(cq), ts, type, rowkey)); + from++; + } + return result; + } + + // have to write custom equality because all we nee for replication is + // clusterId and tableName and writeTime + private EqualsBuilder customHLogKeyEquality(HLogKey left, HLogKey right) { + return new EqualsBuilder() + .append(left.getClusterId(), right.getClusterId()) + .append(left.getTablename(), right.getTablename()); + } + +} -- 2.1.0