HBASE-16604 Scanner retries on IOException can cause the scans to miss data

This commit is contained in:
Enis Soztutar 2016-09-22 12:06:11 -07:00
parent 1384c9a08d
commit 83cf44cd3f
69 changed files with 112176 additions and 53 deletions

View File

@ -45,4 +45,8 @@ public class UnknownScannerException extends DoNotRetryIOException {
public UnknownScannerException(String s) {
super(s);
}
public UnknownScannerException(String s, Exception e) {
super(s, e);
}
}

View File

@ -33,6 +33,7 @@ import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.UnknownScannerException;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.exceptions.OutOfOrderScannerNextException;
import org.apache.hadoop.hbase.exceptions.ScannerResetException;
import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.MapReduceProtos;
@ -428,7 +429,8 @@ public abstract class ClientScanner extends AbstractClientScanner {
if ((cause != null && cause instanceof NotServingRegionException) ||
(cause != null && cause instanceof RegionServerStoppedException) ||
e instanceof OutOfOrderScannerNextException ||
e instanceof UnknownScannerException ) {
e instanceof UnknownScannerException ||
e instanceof ScannerResetException) {
// Pass. It is easier writing the if loop test as list of what is allowed rather than
// as a list of what is not allowed... so if in here, it means we do not throw.
} else {

View File

@ -40,7 +40,7 @@ import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.UnknownScannerException;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.client.metrics.ScanMetrics;
import org.apache.hadoop.hbase.ipc.HBaseRpcController;
import org.apache.hadoop.hbase.exceptions.ScannerResetException;
import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.RequestConverter;
@ -102,7 +102,7 @@ public class ScannerCallable extends RegionServerCallable<Result[]> {
* @param scan the scan to execute
* @param scanMetrics the ScanMetrics to used, if it is null, ScannerCallable won't collect
* metrics
* @param rpcControllerFactory factory to use when creating
* @param rpcControllerFactory factory to use when creating
* {@link com.google.protobuf.RpcController}
*/
public ScannerCallable(ClusterConnection connection, TableName tableName, Scan scan,
@ -174,6 +174,7 @@ public class ScannerCallable extends RegionServerCallable<Result[]> {
}
}
@Override
protected Result [] rpcCall() throws Exception {
if (Thread.interrupted()) {
throw new InterruptedIOException();
@ -245,14 +246,19 @@ public class ScannerCallable extends RegionServerCallable<Result[]> {
if (e instanceof RemoteException) {
ioe = ((RemoteException) e).unwrapRemoteException();
}
if (logScannerActivity && (ioe instanceof UnknownScannerException)) {
try {
HRegionLocation location =
getConnection().relocateRegion(getTableName(), scan.getStartRow());
LOG.info("Scanner=" + scannerId + " expired, current region location is " +
location.toString());
} catch (Throwable t) {
LOG.info("Failed to relocate region", t);
if (logScannerActivity) {
if (ioe instanceof UnknownScannerException) {
try {
HRegionLocation location =
getConnection().relocateRegion(getTableName(), scan.getStartRow());
LOG.info("Scanner=" + scannerId + " expired, current region location is " +
location.toString());
} catch (Throwable t) {
LOG.info("Failed to relocate region", t);
}
} else if (ioe instanceof ScannerResetException) {
LOG.info("Scanner=" + scannerId + " has received an exception, and the server "
+ "asked us to reset the scanner state.", ioe);
}
}
// The below convertion of exceptions into DoNotRetryExceptions is a little strange.

View File

@ -0,0 +1,50 @@
/**
* 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.exceptions;
import org.apache.hadoop.hbase.DoNotRetryIOException;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.classification.InterfaceStability;
/**
* Thrown when the server side has received an Exception, and asks the Client to reset the scanner
* state by closing the current region scanner, and reopening from the start of last seen row.
*/
@InterfaceAudience.Public
@InterfaceStability.Stable
public class ScannerResetException extends DoNotRetryIOException {
private static final long serialVersionUID = -5649728171144849619L;
/** constructor */
public ScannerResetException() {
super();
}
/**
* Constructor
* @param s message
*/
public ScannerResetException(String s) {
super(s);
}
public ScannerResetException(String s, Exception e) {
super(s, e);
}
}

View File

@ -79,6 +79,7 @@ public interface MetricsHBaseServerSource extends BaseSource {
String EXCEPTIONS_OOO_NAME="exceptions.OutOfOrderScannerNextException";
String EXCEPTIONS_BUSY_NAME="exceptions.RegionTooBusyException";
String EXCEPTIONS_UNKNOWN_NAME="exceptions.UnknownScannerException";
String EXCEPTIONS_SCANNER_RESET_NAME="exceptions.ScannerResetException";
String EXCEPTIONS_SANITY_NAME="exceptions.FailedSanityCheckException";
String EXCEPTIONS_MOVED_NAME="exceptions.RegionMovedException";
String EXCEPTIONS_NSRE_NAME="exceptions.NotServingRegionException";
@ -108,6 +109,7 @@ public interface MetricsHBaseServerSource extends BaseSource {
void movedRegionException();
void notServingRegionException();
void unknownScannerException();
void scannerResetException();
void tooBusyException();
void multiActionTooLargeException();
void callQueueTooBigException();

View File

@ -45,6 +45,7 @@ public class MetricsHBaseServerSourceImpl extends BaseSourceImpl
private final MutableFastCounter exceptionsOOO;
private final MutableFastCounter exceptionsBusy;
private final MutableFastCounter exceptionsUnknown;
private final MutableFastCounter exceptionsScannerReset;
private final MutableFastCounter exceptionsSanity;
private final MutableFastCounter exceptionsNSRE;
private final MutableFastCounter exceptionsMoved;
@ -78,6 +79,8 @@ public class MetricsHBaseServerSourceImpl extends BaseSourceImpl
.newCounter(EXCEPTIONS_BUSY_NAME, EXCEPTIONS_TYPE_DESC, 0L);
this.exceptionsUnknown = this.getMetricsRegistry()
.newCounter(EXCEPTIONS_UNKNOWN_NAME, EXCEPTIONS_TYPE_DESC, 0L);
this.exceptionsScannerReset = this.getMetricsRegistry()
.newCounter(EXCEPTIONS_SCANNER_RESET_NAME, EXCEPTIONS_TYPE_DESC, 0L);
this.exceptionsSanity = this.getMetricsRegistry()
.newCounter(EXCEPTIONS_SANITY_NAME, EXCEPTIONS_TYPE_DESC, 0L);
this.exceptionsMoved = this.getMetricsRegistry()
@ -161,6 +164,11 @@ public class MetricsHBaseServerSourceImpl extends BaseSourceImpl
exceptionsUnknown.incr();
}
@Override
public void scannerResetException() {
exceptionsScannerReset.incr();
}
@Override
public void tooBusyException() {
exceptionsBusy.incr();

View File

@ -29,6 +29,7 @@ import org.apache.hadoop.hbase.CompatibilitySingletonFactory;
import org.apache.hadoop.hbase.exceptions.FailedSanityCheckException;
import org.apache.hadoop.hbase.exceptions.OutOfOrderScannerNextException;
import org.apache.hadoop.hbase.exceptions.RegionMovedException;
import org.apache.hadoop.hbase.exceptions.ScannerResetException;
@InterfaceAudience.Private
public class MetricsHBaseServer {
@ -103,6 +104,8 @@ public class MetricsHBaseServer {
source.tooBusyException();
} else if (throwable instanceof UnknownScannerException) {
source.unknownScannerException();
} else if (throwable instanceof ScannerResetException) {
source.scannerResetException();
} else if (throwable instanceof RegionMovedException) {
source.movedRegionException();
} else if (throwable instanceof NotServingRegionException) {

View File

@ -82,6 +82,7 @@ import org.apache.hadoop.hbase.conf.ConfigurationObserver;
import org.apache.hadoop.hbase.exceptions.FailedSanityCheckException;
import org.apache.hadoop.hbase.exceptions.MergeRegionException;
import org.apache.hadoop.hbase.exceptions.OutOfOrderScannerNextException;
import org.apache.hadoop.hbase.exceptions.ScannerResetException;
import org.apache.hadoop.hbase.filter.ByteArrayComparable;
import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
import org.apache.hadoop.hbase.ipc.HBaseRPCErrorHandler;
@ -2901,13 +2902,22 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
isClientCellBlockSupport(context));
}
} catch (IOException e) {
// if we have an exception on scanner next and we are using the callSeq
// we should rollback because the client will retry with the same callSeq
// and get an OutOfOrderScannerNextException if we don't do so.
if (rsh != null && request.hasNextCallSeq()) {
rsh.rollbackNextCallSeq();
// The scanner state might be left in a dirty state, so we will tell the Client to
// fail this RPC and close the scanner while opening up another one from the start of
// row that the client has last seen.
closeScanner(region, scanner, scannerName, context);
// We closed the scanner already. Instead of throwing the IOException, and client
// retrying with the same scannerId only to get USE on the next RPC, we directly throw
// a special exception to save an RPC.
if (VersionInfoUtil.hasMinimumVersion(context.getClientVersionInfo(), 1, 4)) {
// 1.4.0+ clients know how to handle
throw new ScannerResetException("Scanner is closed on the server-side", e);
} else {
// older clients do not know about SRE. Just throw USE, which they will handle
throw new UnknownScannerException("Throwing UnknownScannerException to reset the client"
+ " scanner state for clients older than 1.3.", e);
}
throw e;
} finally {
if (context != null) {
context.setCallBack(rsh.shippedCallback);
@ -2926,29 +2936,8 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
if (!moreResults || closeScanner) {
ttl = 0;
moreResults = false;
if (region != null && region.getCoprocessorHost() != null) {
if (region.getCoprocessorHost().preScannerClose(scanner)) {
return builder.build(); // bypass
}
}
rsh = scanners.remove(scannerName);
if (rsh != null) {
if (context != null) {
context.setCallBack(rsh.closeCallBack);
} else {
rsh.s.close();
}
try {
regionServer.leases.cancelLease(scannerName);
} catch (LeaseException le) {
// No problem, ignore
if (LOG.isTraceEnabled()) {
LOG.trace("Un-able to cancel lease of scanner. It could already be closed.");
}
}
if (region != null && region.getCoprocessorHost() != null) {
region.getCoprocessorHost().postScannerClose(scanner);
}
if (closeScanner(region, scanner, scannerName, context)) {
return builder.build(); // bypass
}
}
@ -2980,6 +2969,35 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
}
}
private boolean closeScanner(Region region, RegionScanner scanner, String scannerName,
RpcCallContext context) throws IOException {
if (region != null && region.getCoprocessorHost() != null) {
if (region.getCoprocessorHost().preScannerClose(scanner)) {
return true; // bypass
}
}
RegionScannerHolder rsh = scanners.remove(scannerName);
if (rsh != null) {
if (context != null) {
context.setCallBack(rsh.closeCallBack);
} else {
rsh.s.close();
}
try {
regionServer.leases.cancelLease(scannerName);
} catch (LeaseException le) {
// No problem, ignore
if (LOG.isTraceEnabled()) {
LOG.trace("Un-able to cancel lease of scanner. It could already be closed.");
}
}
if (region != null && region.getCoprocessorHost() != null) {
region.getCoprocessorHost().postScannerClose(scanner);
}
}
return false;
}
@Override
public CoprocessorServiceResponse execRegionServerService(RpcController controller,
CoprocessorServiceRequest request) throws ServiceException {

View File

@ -1414,12 +1414,8 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
public Table createTable(TableName tableName, byte[][] families,
int numVersions, byte[] startKey, byte[] endKey, int numRegions)
throws IOException{
HTableDescriptor desc = new HTableDescriptor(tableName);
for (byte[] family : families) {
HColumnDescriptor hcd = new HColumnDescriptor(family)
.setMaxVersions(numVersions);
desc.addFamily(hcd);
}
HTableDescriptor desc = createTableDescriptor(tableName, families, numVersions);
getAdmin().createTable(desc, startKey, endKey, numRegions);
// HBaseAdmin only waits for regions to appear in hbase:meta we
// should wait until they are assigned
@ -1782,6 +1778,22 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
MAXVERSIONS, HConstants.FOREVER, HColumnDescriptor.DEFAULT_KEEP_DELETED);
}
public HTableDescriptor createTableDescriptor(final TableName tableName,
byte[] family) {
return createTableDescriptor(tableName, new byte[][] {family}, 1);
}
public HTableDescriptor createTableDescriptor(final TableName tableName,
byte[][] families, int maxVersions) {
HTableDescriptor desc = new HTableDescriptor(tableName);
for (byte[] family : families) {
HColumnDescriptor hcd = new HColumnDescriptor(family)
.setMaxVersions(maxVersions);
desc.addFamily(hcd);
}
return desc;
}
/**
* Create an HRegion that writes to the local tmp dirs
* @param desc
@ -1999,7 +2011,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
put.setDurability(writeToWAL ? Durability.USE_DEFAULT : Durability.SKIP_WAL);
for (int i = 0; i < f.length; i++) {
byte[] value1 = value != null ? value : row;
put.addColumn(f[i], null, value1);
put.addColumn(f[i], f[i], value1);
}
puts.add(put);
}
@ -3541,6 +3553,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
public PortAllocator(Random random) {
this.random = random;
this.portChecker = new AvailablePortChecker() {
@Override
public boolean available(int port) {
try {
ServerSocket sock = new ServerSocket(port);

View File

@ -26,7 +26,6 @@ import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertSame;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
@ -37,10 +36,12 @@ import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.NavigableMap;
import java.util.NavigableSet;
import java.util.UUID;
import java.util.concurrent.Callable;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.atomic.AtomicLong;
import java.util.concurrent.atomic.AtomicReference;
import org.apache.commons.lang.ArrayUtils;
@ -63,8 +64,11 @@ import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.Waiter;
import org.apache.hadoop.hbase.client.metrics.ScanMetrics;
import org.apache.hadoop.hbase.coprocessor.BaseRegionObserver;
import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
import org.apache.hadoop.hbase.coprocessor.MultiRowMutationEndpoint;
import org.apache.hadoop.hbase.coprocessor.ObserverContext;
import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
import org.apache.hadoop.hbase.filter.BinaryComparator;
import org.apache.hadoop.hbase.filter.CompareFilter;
import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
@ -91,10 +95,14 @@ import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.MutationType;
import org.apache.hadoop.hbase.protobuf.generated.MultiRowMutationProtos.MultiRowMutationService;
import org.apache.hadoop.hbase.protobuf.generated.MultiRowMutationProtos.MutateRowsRequest;
import org.apache.hadoop.hbase.regionserver.DelegatingKeyValueScanner;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.regionserver.KeyValueScanner;
import org.apache.hadoop.hbase.regionserver.NoSuchColumnFamilyException;
import org.apache.hadoop.hbase.regionserver.Region;
import org.apache.hadoop.hbase.regionserver.ScanInfo;
import org.apache.hadoop.hbase.regionserver.Store;
import org.apache.hadoop.hbase.regionserver.StoreScanner;
import org.apache.hadoop.hbase.testclassification.ClientTests;
import org.apache.hadoop.hbase.testclassification.LargeTests;
import org.apache.hadoop.hbase.util.Bytes;
@ -536,6 +544,71 @@ public class TestFromClientSide {
assertEquals(rowCount - endKeyCount, countGreater);
}
/**
* This is a coprocessor to inject a test failure so that a store scanner.reseek() call will
* fail with an IOException() on the first call.
*/
public static class ExceptionInReseekRegionObserver extends BaseRegionObserver {
static AtomicLong reqCount = new AtomicLong(0);
class MyStoreScanner extends StoreScanner {
public MyStoreScanner(Store store, ScanInfo scanInfo, Scan scan, NavigableSet<byte[]> columns,
long readPt) throws IOException {
super(store, scanInfo, scan, columns, readPt);
}
@Override
protected List<KeyValueScanner> selectScannersFrom(
List<? extends KeyValueScanner> allScanners) {
List<KeyValueScanner> scanners = super.selectScannersFrom(allScanners);
List<KeyValueScanner> newScanners = new ArrayList<>(scanners.size());
for (KeyValueScanner scanner : scanners) {
newScanners.add(new DelegatingKeyValueScanner(scanner) {
@Override
public boolean reseek(Cell key) throws IOException {
if (reqCount.incrementAndGet() == 1) {
throw new IOException("Injected exception");
}
return super.reseek(key);
}
});
}
return newScanners;
}
}
@Override
public KeyValueScanner preStoreScannerOpen(ObserverContext<RegionCoprocessorEnvironment> c,
Store store, Scan scan, NavigableSet<byte[]> targetCols, KeyValueScanner s,
final long readPt) throws IOException {
return new MyStoreScanner(store, store.getScanInfo(), scan, targetCols, readPt);
}
}
/**
* Tests the case where a Scan can throw an IOException in the middle of the seek / reseek
* leaving the server side RegionScanner to be in dirty state. The client has to ensure that the
* ClientScanner does not get an exception and also sees all the data.
* @throws IOException
* @throws InterruptedException
*/
@Test
public void testClientScannerIsResetWhenScanThrowsIOException()
throws IOException, InterruptedException {
TEST_UTIL.getConfiguration().setBoolean("hbase.client.log.scanner.activity", true);
TableName name = TableName.valueOf("testClientScannerIsResetWhenScanThrowsIOException");
HTableDescriptor htd = TEST_UTIL.createTableDescriptor(name, FAMILY);
htd.addCoprocessor(ExceptionInReseekRegionObserver.class.getName());
TEST_UTIL.getAdmin().createTable(htd);
try (Table t = TEST_UTIL.getConnection().getTable(name)) {
int rowCount = TEST_UTIL.loadTable(t, FAMILY, false);
TEST_UTIL.getAdmin().flush(name);
int actualRowCount = countRows(t, new Scan().addColumn(FAMILY, FAMILY));
assertEquals(rowCount, actualRowCount);
}
assertTrue(ExceptionInReseekRegionObserver.reqCount.get() > 0);
}
/*
* @param key
* @return Scan with RowFilter that does LESS than passed key.

View File

@ -184,7 +184,7 @@ public class TestTableSnapshotScanner {
}
for (int j = 0; j < FAMILIES.length; j++) {
byte[] actual = result.getValue(FAMILIES[j], null);
byte[] actual = result.getValue(FAMILIES[j], FAMILIES[j]);
Assert.assertArrayEquals("Row in snapshot does not match, expected:" + Bytes.toString(row)
+ " ,actual:" + Bytes.toString(actual), row, actual);
}

View File

@ -181,7 +181,7 @@ public abstract class TableSnapshotInputFormatTestBase {
}
for (int j = 0; j < FAMILIES.length; j++) {
byte[] actual = result.getValue(FAMILIES[j], null);
byte[] actual = result.getValue(FAMILIES[j], FAMILIES[j]);
Assert.assertArrayEquals("Row in snapshot does not match, expected:" + Bytes.toString(row)
+ " ,actual:" + Bytes.toString(actual), row, actual);
}

View File

@ -97,6 +97,7 @@ public class TestMultithreadedTableMapper {
* @param context
* @throws IOException
*/
@Override
public void map(ImmutableBytesWritable key, Result value,
Context context)
throws IOException, InterruptedException {
@ -110,7 +111,7 @@ public class TestMultithreadedTableMapper {
Bytes.toString(INPUT_FAMILY) + "'.");
}
// Get the original value and reverse it
String originalValue = Bytes.toString(value.getValue(INPUT_FAMILY, null));
String originalValue = Bytes.toString(value.getValue(INPUT_FAMILY, INPUT_FAMILY));
StringBuilder newValue = new StringBuilder(originalValue);
newValue.reverse();
// Now set the value to be collected

View File

@ -57,6 +57,7 @@ import org.junit.experimental.categories.Category;
public class TestTableMapReduce extends TestTableMapReduceBase {
private static final Log LOG = LogFactory.getLog(TestTableMapReduce.class);
@Override
protected Log getLog() { return LOG; }
/**
@ -72,6 +73,7 @@ public class TestTableMapReduce extends TestTableMapReduceBase {
* @param context
* @throws IOException
*/
@Override
public void map(ImmutableBytesWritable key, Result value,
Context context)
throws IOException, InterruptedException {
@ -86,7 +88,7 @@ public class TestTableMapReduce extends TestTableMapReduceBase {
}
// Get the original value and reverse it
String originalValue = Bytes.toString(value.getValue(INPUT_FAMILY, null));
String originalValue = Bytes.toString(value.getValue(INPUT_FAMILY, INPUT_FAMILY));
StringBuilder newValue = new StringBuilder(originalValue);
newValue.reverse();
// Now set the value to be collected
@ -96,6 +98,7 @@ public class TestTableMapReduce extends TestTableMapReduceBase {
}
}
@Override
protected void runTestOnTable(Table table) throws IOException {
Job job = null;
try {

View File

@ -126,7 +126,7 @@ public abstract class TestTableMapReduceBase {
// Get the original value and reverse it
String originalValue = Bytes.toString(value.getValue(INPUT_FAMILY, null));
String originalValue = Bytes.toString(value.getValue(INPUT_FAMILY, INPUT_FAMILY));
StringBuilder newValue = new StringBuilder(originalValue);
newValue.reverse();

View File

@ -0,0 +1,114 @@
/**
* 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.regionserver;
import java.io.IOException;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.regionserver.KeyValueScanner;
import org.apache.hadoop.hbase.regionserver.Store;
public class DelegatingKeyValueScanner implements KeyValueScanner {
protected KeyValueScanner delegate;
public DelegatingKeyValueScanner(KeyValueScanner delegate) {
this.delegate = delegate;
}
@Override
public void shipped() throws IOException {
delegate.shipped();
}
@Override
public Cell peek() {
return delegate.peek();
}
@Override
public Cell next() throws IOException {
return delegate.next();
}
@Override
public boolean seek(Cell key) throws IOException {
return delegate.seek(key);
}
@Override
public boolean reseek(Cell key) throws IOException {
return delegate.reseek(key);
}
@Override
public long getScannerOrder() {
return delegate.getScannerOrder();
}
@Override
public void close() {
delegate.close();
}
@Override
public boolean shouldUseScanner(Scan scan, Store store, long oldestUnexpiredTS) {
return delegate.shouldUseScanner(scan, store, oldestUnexpiredTS);
}
@Override
public boolean requestSeek(Cell kv, boolean forward, boolean useBloom) throws IOException {
return delegate.requestSeek(kv, forward, useBloom);
}
@Override
public boolean realSeekDone() {
return delegate.realSeekDone();
}
@Override
public void enforceSeek() throws IOException {
delegate.enforceSeek();
}
@Override
public boolean isFileScanner() {
return delegate.isFileScanner();
}
@Override
public boolean backwardSeek(Cell key) throws IOException {
return delegate.backwardSeek(key);
}
@Override
public boolean seekToPreviousRow(Cell key) throws IOException {
return delegate.seekToPreviousRow(key);
}
@Override
public boolean seekToLastRow() throws IOException {
return delegate.seekToLastRow();
}
@Override
public Cell getNextIndexedKey() {
return delegate.getNextIndexedKey();
}
}

View File

@ -0,0 +1,97 @@
/*
* 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.thrift;
import java.util.Locale;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.thrift.server.TThreadedSelectorServer;
import org.apache.thrift.transport.TNonblockingServerTransport;
/**
* A TThreadedSelectorServer.Args that reads hadoop configuration
*/
@InterfaceAudience.Private
public class HThreadedSelectorServerArgs extends TThreadedSelectorServer.Args {
private static final Log LOG = LogFactory.getLog(TThreadedSelectorServer.class);
/**
* Number of selector threads for reading and writing socket
*/
public static final String SELECTOR_THREADS_CONF_KEY =
"hbase.thrift.selector.threads";
/**
* Number fo threads for processing the thrift calls
*/
public static final String WORKER_THREADS_CONF_KEY =
"hbase.thrift.worker.threads";
/**
* Time to wait for server to stop gracefully
*/
public static final String STOP_TIMEOUT_CONF_KEY =
"hbase.thrift.stop.timeout.seconds";
/**
* Maximum number of accepted elements per selector
*/
public static final String ACCEPT_QUEUE_SIZE_PER_THREAD_CONF_KEY =
"hbase.thrift.accept.queue.size.per.selector";
/**
* The strategy for handling new accepted connections.
*/
public static final String ACCEPT_POLICY_CONF_KEY =
"hbase.thrift.accept.policy";
public HThreadedSelectorServerArgs(
TNonblockingServerTransport transport, Configuration conf) {
super(transport);
readConf(conf);
}
private void readConf(Configuration conf) {
int selectorThreads = conf.getInt(
SELECTOR_THREADS_CONF_KEY, getSelectorThreads());
int workerThreads = conf.getInt(
WORKER_THREADS_CONF_KEY, getWorkerThreads());
int stopTimeoutVal = conf.getInt(
STOP_TIMEOUT_CONF_KEY, getStopTimeoutVal());
int acceptQueueSizePerThread = conf.getInt(
ACCEPT_QUEUE_SIZE_PER_THREAD_CONF_KEY, getAcceptQueueSizePerThread());
AcceptPolicy acceptPolicy = AcceptPolicy.valueOf(conf.get(
ACCEPT_POLICY_CONF_KEY, getAcceptPolicy().toString()).toUpperCase(Locale.ROOT));
super.selectorThreads(selectorThreads)
.workerThreads(workerThreads)
.stopTimeoutVal(stopTimeoutVal)
.acceptQueueSizePerThread(acceptQueueSizePerThread)
.acceptPolicy(acceptPolicy);
LOG.info("Read configuration selectorThreads:" + selectorThreads +
" workerThreads:" + workerThreads +
" stopTimeoutVal:" + stopTimeoutVal + "sec" +
" acceptQueueSizePerThread:" + acceptQueueSizePerThread +
" acceptPolicy:" + acceptPolicy);
}
}

View File

@ -0,0 +1,82 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.thrift;
import java.lang.reflect.InvocationHandler;
import java.lang.reflect.InvocationTargetException;
import java.lang.reflect.Method;
import java.lang.reflect.Proxy;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.thrift.generated.Hbase;
/**
* Converts a Hbase.Iface using InvocationHandler so that it reports process
* time of each call to ThriftMetrics.
*/
@InterfaceAudience.Private
public class HbaseHandlerMetricsProxy implements InvocationHandler {
private static final Log LOG = LogFactory.getLog(
HbaseHandlerMetricsProxy.class);
private final Hbase.Iface handler;
private final ThriftMetrics metrics;
public static Hbase.Iface newInstance(Hbase.Iface handler,
ThriftMetrics metrics,
Configuration conf) {
return (Hbase.Iface) Proxy.newProxyInstance(
handler.getClass().getClassLoader(),
new Class[]{Hbase.Iface.class},
new HbaseHandlerMetricsProxy(handler, metrics, conf));
}
private HbaseHandlerMetricsProxy(
Hbase.Iface handler, ThriftMetrics metrics, Configuration conf) {
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);
long processTime = 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();
}
}

View File

@ -0,0 +1,42 @@
/*
* 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.thrift;
public class HttpAuthenticationException extends Exception {
private static final long serialVersionUID = 0;
/**
* @param cause original exception
*/
public HttpAuthenticationException(Throwable cause) {
super(cause);
}
/**
* @param msg exception message
*/
public HttpAuthenticationException(String msg) {
super(msg);
}
/**
* @param msg exception message
* @param cause original exception
*/
public HttpAuthenticationException(String msg, Throwable cause) {
super(msg, cause);
}
}

View File

@ -0,0 +1,372 @@
/*
* 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.thrift;
import java.io.IOException;
import java.util.Arrays;
import java.util.List;
import java.util.Set;
import java.util.concurrent.Callable;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.LinkedBlockingQueue;
import java.util.concurrent.ThreadFactory;
import java.util.concurrent.ThreadPoolExecutor;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicLong;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.thrift.ThriftServerRunner.HBaseHandler;
import org.apache.hadoop.hbase.thrift.generated.TIncrement;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Threads;
import org.apache.hadoop.metrics.util.MBeanUtil;
import org.apache.thrift.TException;
/**
* This class will coalesce increments from a thift server if
* hbase.regionserver.thrift.coalesceIncrement is set to true. Turning this
* config to true will cause the thrift server to queue increments into an
* instance of this class. The thread pool associated with this class will drain
* the coalesced increments as the thread is able. This can cause data loss if the
* thrift server dies or is shut down before everything in the queue is drained.
*
*/
public class IncrementCoalescer implements IncrementCoalescerMBean {
/**
* Used to identify a cell that will be incremented.
*
*/
static class FullyQualifiedRow {
private byte[] table;
private byte[] rowKey;
private byte[] family;
private byte[] qualifier;
public FullyQualifiedRow(byte[] table, byte[] rowKey, byte[] fam, byte[] qual) {
super();
this.table = table;
this.rowKey = rowKey;
this.family = fam;
this.qualifier = qual;
}
public byte[] getTable() {
return table;
}
public void setTable(byte[] table) {
this.table = table;
}
public byte[] getRowKey() {
return rowKey;
}
public void setRowKey(byte[] rowKey) {
this.rowKey = rowKey;
}
public byte[] getFamily() {
return family;
}
public void setFamily(byte[] fam) {
this.family = fam;
}
public byte[] getQualifier() {
return qualifier;
}
public void setQualifier(byte[] qual) {
this.qualifier = qual;
}
@Override
public int hashCode() {
final int prime = 31;
int result = 1;
result = prime * result + Arrays.hashCode(family);
result = prime * result + Arrays.hashCode(qualifier);
result = prime * result + Arrays.hashCode(rowKey);
result = prime * result + Arrays.hashCode(table);
return result;
}
@Override
public boolean equals(Object obj) {
if (this == obj) return true;
if (obj == null) return false;
if (getClass() != obj.getClass()) return false;
FullyQualifiedRow other = (FullyQualifiedRow) obj;
if (!Arrays.equals(family, other.family)) return false;
if (!Arrays.equals(qualifier, other.qualifier)) return false;
if (!Arrays.equals(rowKey, other.rowKey)) return false;
if (!Arrays.equals(table, other.table)) return false;
return true;
}
}
static class DaemonThreadFactory implements ThreadFactory {
static final AtomicInteger poolNumber = new AtomicInteger(1);
final ThreadGroup group;
final AtomicInteger threadNumber = new AtomicInteger(1);
final String namePrefix;
DaemonThreadFactory() {
SecurityManager s = System.getSecurityManager();
group = (s != null) ? s.getThreadGroup() : Thread.currentThread().getThreadGroup();
namePrefix = "ICV-" + poolNumber.getAndIncrement() + "-thread-";
}
public Thread newThread(Runnable r) {
Thread t = new Thread(group, r, namePrefix + threadNumber.getAndIncrement(), 0);
if (!t.isDaemon()) t.setDaemon(true);
if (t.getPriority() != Thread.NORM_PRIORITY) t.setPriority(Thread.NORM_PRIORITY);
return t;
}
}
private final AtomicLong failedIncrements = new AtomicLong();
private final AtomicLong successfulCoalescings = new AtomicLong();
private final AtomicLong totalIncrements = new AtomicLong();
private final ConcurrentMap<FullyQualifiedRow, Long> countersMap =
new ConcurrentHashMap<FullyQualifiedRow, Long>(100000, 0.75f, 1500);
private final ThreadPoolExecutor pool;
private final HBaseHandler handler;
private int maxQueueSize = 500000;
private static final int CORE_POOL_SIZE = 1;
private static final Log LOG = LogFactory.getLog(FullyQualifiedRow.class);
@SuppressWarnings("deprecation")
public IncrementCoalescer(HBaseHandler hand) {
this.handler = hand;
LinkedBlockingQueue<Runnable> queue = new LinkedBlockingQueue<Runnable>();
pool =
new ThreadPoolExecutor(CORE_POOL_SIZE, CORE_POOL_SIZE, 50, TimeUnit.MILLISECONDS, queue,
Threads.newDaemonThreadFactory("IncrementCoalescer"));
MBeanUtil.registerMBean("thrift", "Thrift", this);
}
public boolean queueIncrement(TIncrement inc) throws TException {
if (!canQueue()) {
failedIncrements.incrementAndGet();
return false;
}
return internalQueueTincrement(inc);
}
public boolean queueIncrements(List<TIncrement> incs) throws TException {
if (!canQueue()) {
failedIncrements.incrementAndGet();
return false;
}
for (TIncrement tinc : incs) {
internalQueueTincrement(tinc);
}
return true;
}
private boolean internalQueueTincrement(TIncrement inc) throws TException {
byte[][] famAndQf = KeyValue.parseColumn(inc.getColumn());
if (famAndQf.length != 2) return false;
return internalQueueIncrement(inc.getTable(), inc.getRow(), famAndQf[0], famAndQf[1],
inc.getAmmount());
}
private boolean internalQueueIncrement(byte[] tableName, byte[] rowKey, byte[] fam,
byte[] qual, long ammount) throws TException {
int countersMapSize = countersMap.size();
//Make sure that the number of threads is scaled.
dynamicallySetCoreSize(countersMapSize);
totalIncrements.incrementAndGet();
FullyQualifiedRow key = new FullyQualifiedRow(tableName, rowKey, fam, qual);
long currentAmount = ammount;
// Spin until able to insert the value back without collisions
while (true) {
Long value = countersMap.remove(key);
if (value == null) {
// There was nothing there, create a new value
value = Long.valueOf(currentAmount);
} else {
value += currentAmount;
successfulCoalescings.incrementAndGet();
}
// Try to put the value, only if there was none
Long oldValue = countersMap.putIfAbsent(key, value);
if (oldValue == null) {
// We were able to put it in, we're done
break;
}
// Someone else was able to put a value in, so let's remember our
// current value (plus what we picked up) and retry to add it in
currentAmount = value;
}
// We limit the size of the queue simply because all we need is a
// notification that something needs to be incremented. No need
// for millions of callables that mean the same thing.
if (pool.getQueue().size() <= 1000) {
// queue it up
Callable<Integer> callable = createIncCallable();
pool.submit(callable);
}
return true;
}
public boolean canQueue() {
return countersMap.size() < maxQueueSize;
}
private Callable<Integer> createIncCallable() {
return new Callable<Integer>() {
@Override
public Integer call() throws Exception {
int failures = 0;
Set<FullyQualifiedRow> keys = countersMap.keySet();
for (FullyQualifiedRow row : keys) {
Long counter = countersMap.remove(row);
if (counter == null) {
continue;
}
Table table = null;
try {
table = handler.getTable(row.getTable());
if (failures > 2) {
throw new IOException("Auto-Fail rest of ICVs");
}
table.incrementColumnValue(row.getRowKey(), row.getFamily(), row.getQualifier(),
counter);
} catch (IOException e) {
// log failure of increment
failures++;
LOG.error("FAILED_ICV: " + Bytes.toString(row.getTable()) + ", "
+ Bytes.toStringBinary(row.getRowKey()) + ", "
+ Bytes.toStringBinary(row.getFamily()) + ", "
+ Bytes.toStringBinary(row.getQualifier()) + ", " + counter, e);
} finally{
if(table != null){
table.close();
}
}
}
return failures;
}
};
}
/**
* This method samples the incoming requests and, if selected, will check if
* the corePoolSize should be changed.
* @param countersMapSize
*/
private void dynamicallySetCoreSize(int countersMapSize) {
// Here we are using countersMapSize as a random number, meaning this
// could be a Random object
if (countersMapSize % 10 != 0) {
return;
}
double currentRatio = (double) countersMapSize / (double) maxQueueSize;
int newValue = 1;
if (currentRatio < 0.1) {
// it's 1
} else if (currentRatio < 0.3) {
newValue = 2;
} else if (currentRatio < 0.5) {
newValue = 4;
} else if (currentRatio < 0.7) {
newValue = 8;
} else if (currentRatio < 0.9) {
newValue = 14;
} else {
newValue = 22;
}
if (pool.getCorePoolSize() != newValue) {
pool.setCorePoolSize(newValue);
}
}
// MBean get/set methods
public int getQueueSize() {
return pool.getQueue().size();
}
public int getMaxQueueSize() {
return this.maxQueueSize;
}
public void setMaxQueueSize(int newSize) {
this.maxQueueSize = newSize;
}
public long getPoolCompletedTaskCount() {
return pool.getCompletedTaskCount();
}
public long getPoolTaskCount() {
return pool.getTaskCount();
}
public int getPoolLargestPoolSize() {
return pool.getLargestPoolSize();
}
public int getCorePoolSize() {
return pool.getCorePoolSize();
}
public void setCorePoolSize(int newCoreSize) {
pool.setCorePoolSize(newCoreSize);
}
public int getMaxPoolSize() {
return pool.getMaximumPoolSize();
}
public void setMaxPoolSize(int newMaxSize) {
pool.setMaximumPoolSize(newMaxSize);
}
public long getFailedIncrements() {
return failedIncrements.get();
}
public long getSuccessfulCoalescings() {
return successfulCoalescings.get();
}
public long getTotalIncrements() {
return totalIncrements.get();
}
public long getCountersMapSize() {
return countersMap.size();
}
}

View File

@ -0,0 +1,49 @@
/*
* 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.thrift;
public interface IncrementCoalescerMBean {
int getQueueSize();
int getMaxQueueSize();
void setMaxQueueSize(int newSize);
long getPoolCompletedTaskCount();
long getPoolTaskCount();
int getPoolLargestPoolSize();
int getCorePoolSize();
void setCorePoolSize(int newCoreSize);
int getMaxPoolSize();
void setMaxPoolSize(int newMaxSize);
long getFailedIncrements();
long getSuccessfulCoalescings();
long getTotalIncrements();
long getCountersMapSize();
}

View File

@ -0,0 +1,311 @@
/*
* 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.thrift;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.LinkedBlockingQueue;
import java.util.concurrent.RejectedExecutionException;
import java.util.concurrent.SynchronousQueue;
import java.util.concurrent.ThreadPoolExecutor;
import java.util.concurrent.TimeUnit;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.thrift.CallQueue.Call;
import org.apache.hadoop.hbase.util.Threads;
import org.apache.thrift.TException;
import org.apache.thrift.TProcessor;
import org.apache.thrift.protocol.TProtocol;
import org.apache.thrift.server.TServer;
import org.apache.thrift.server.TThreadPoolServer;
import org.apache.thrift.transport.TServerTransport;
import org.apache.thrift.transport.TSocket;
import org.apache.thrift.transport.TTransport;
import org.apache.thrift.transport.TTransportException;
import com.google.common.util.concurrent.ThreadFactoryBuilder;
/**
* A bounded thread pool server customized for HBase.
*/
@InterfaceAudience.Private
public class TBoundedThreadPoolServer extends TServer {
private static final String QUEUE_FULL_MSG =
"Queue is full, closing connection";
/**
* The "core size" of the thread pool. New threads are created on every
* connection until this many threads are created.
*/
public static final String MIN_WORKER_THREADS_CONF_KEY =
"hbase.thrift.minWorkerThreads";
/**
* This default core pool size should be enough for many test scenarios. We
* want to override this with a much larger number (e.g. at least 200) for a
* large-scale production setup.
*/
public static final int DEFAULT_MIN_WORKER_THREADS = 16;
/**
* The maximum size of the thread pool. When the pending request queue
* overflows, new threads are created until their number reaches this number.
* After that, the server starts dropping connections.
*/
public static final String MAX_WORKER_THREADS_CONF_KEY =
"hbase.thrift.maxWorkerThreads";
public static final int DEFAULT_MAX_WORKER_THREADS = 1000;
/**
* The maximum number of pending connections waiting in the queue. If there
* are no idle threads in the pool, the server queues requests. Only when
* the queue overflows, new threads are added, up to
* hbase.thrift.maxQueuedRequests threads.
*/
public static final String MAX_QUEUED_REQUESTS_CONF_KEY =
"hbase.thrift.maxQueuedRequests";
public static final int DEFAULT_MAX_QUEUED_REQUESTS = 1000;
/**
* Default amount of time in seconds to keep a thread alive. Worker threads
* are stopped after being idle for this long.
*/
public static final String THREAD_KEEP_ALIVE_TIME_SEC_CONF_KEY =
"hbase.thrift.threadKeepAliveTimeSec";
private static final int DEFAULT_THREAD_KEEP_ALIVE_TIME_SEC = 60;
/**
* Time to wait after interrupting all worker threads. This is after a clean
* shutdown has been attempted.
*/
public static final int TIME_TO_WAIT_AFTER_SHUTDOWN_MS = 5000;
private static final Log LOG = LogFactory.getLog(
TBoundedThreadPoolServer.class.getName());
private final CallQueue callQueue;
public static class Args extends TThreadPoolServer.Args {
int maxQueuedRequests;
int threadKeepAliveTimeSec;
public Args(TServerTransport transport, Configuration conf) {
super(transport);
minWorkerThreads = conf.getInt(MIN_WORKER_THREADS_CONF_KEY,
DEFAULT_MIN_WORKER_THREADS);
maxWorkerThreads = conf.getInt(MAX_WORKER_THREADS_CONF_KEY,
DEFAULT_MAX_WORKER_THREADS);
maxQueuedRequests = conf.getInt(MAX_QUEUED_REQUESTS_CONF_KEY,
DEFAULT_MAX_QUEUED_REQUESTS);
threadKeepAliveTimeSec = conf.getInt(THREAD_KEEP_ALIVE_TIME_SEC_CONF_KEY,
DEFAULT_THREAD_KEEP_ALIVE_TIME_SEC);
}
@Override
public String toString() {
return "min worker threads=" + minWorkerThreads
+ ", max worker threads=" + maxWorkerThreads
+ ", max queued requests=" + maxQueuedRequests;
}
}
/** Executor service for handling client connections */
private ThreadPoolExecutor executorService;
/** Flag for stopping the server */
private volatile boolean stopped;
private Args serverOptions;
public TBoundedThreadPoolServer(Args options, ThriftMetrics metrics) {
super(options);
int minWorkerThreads = options.minWorkerThreads;
int maxWorkerThreads = options.maxWorkerThreads;
if (options.maxQueuedRequests > 0) {
this.callQueue = new CallQueue(
new LinkedBlockingQueue<Call>(options.maxQueuedRequests), metrics);
minWorkerThreads = maxWorkerThreads;
} else {
this.callQueue = new CallQueue(new SynchronousQueue<Call>(), metrics);
}
ThreadFactoryBuilder tfb = new ThreadFactoryBuilder();
tfb.setDaemon(true);
tfb.setNameFormat("thrift-worker-%d");
executorService =
new ThreadPoolExecutor(minWorkerThreads,
maxWorkerThreads, options.threadKeepAliveTimeSec,
TimeUnit.SECONDS, this.callQueue, tfb.build());
executorService.allowCoreThreadTimeOut(true);
serverOptions = options;
}
public void serve() {
try {
serverTransport_.listen();
} catch (TTransportException ttx) {
LOG.error("Error occurred during listening.", ttx);
return;
}
Runtime.getRuntime().addShutdownHook(
new Thread(getClass().getSimpleName() + "-shutdown-hook") {
@Override
public void run() {
TBoundedThreadPoolServer.this.stop();
}
});
stopped = false;
while (!stopped && !Thread.interrupted()) {
TTransport client = null;
try {
client = serverTransport_.accept();
} catch (TTransportException ttx) {
if (!stopped) {
LOG.warn("Transport error when accepting message", ttx);
continue;
} else {
// The server has been stopped
break;
}
}
ClientConnnection command = new ClientConnnection(client);
try {
executorService.execute(command);
} catch (RejectedExecutionException rex) {
if (client.getClass() == TSocket.class) {
// We expect the client to be TSocket.
LOG.warn(QUEUE_FULL_MSG + " from " +
((TSocket) client).getSocket().getRemoteSocketAddress());
} else {
LOG.warn(QUEUE_FULL_MSG, rex);
}
client.close();
}
}
shutdownServer();
}
/**
* Loop until {@link ExecutorService#awaitTermination} finally does return
* without an interrupted exception. If we don't do this, then we'll shut
* down prematurely. We want to let the executor service clear its task
* queue, closing client sockets appropriately.
*/
private void shutdownServer() {
executorService.shutdown();
long msLeftToWait =
serverOptions.stopTimeoutUnit.toMillis(serverOptions.stopTimeoutVal);
long timeMillis = System.currentTimeMillis();
LOG.info("Waiting for up to " + msLeftToWait + " ms to finish processing" +
" pending requests");
boolean interrupted = false;
while (msLeftToWait >= 0) {
try {
executorService.awaitTermination(msLeftToWait, TimeUnit.MILLISECONDS);
break;
} catch (InterruptedException ix) {
long timePassed = System.currentTimeMillis() - timeMillis;
msLeftToWait -= timePassed;
timeMillis += timePassed;
interrupted = true;
}
}
LOG.info("Interrupting all worker threads and waiting for "
+ TIME_TO_WAIT_AFTER_SHUTDOWN_MS + " ms longer");
// This will interrupt all the threads, even those running a task.
executorService.shutdownNow();
Threads.sleepWithoutInterrupt(TIME_TO_WAIT_AFTER_SHUTDOWN_MS);
// Preserve the interrupted status.
if (interrupted) {
Thread.currentThread().interrupt();
}
LOG.info("Thrift server shutdown complete");
}
@Override
public void stop() {
stopped = true;
serverTransport_.interrupt();
}
private class ClientConnnection implements Runnable {
private TTransport client;
/**
* Default constructor.
*
* @param client Transport to process
*/
private ClientConnnection(TTransport client) {
this.client = client;
}
/**
* Loops on processing a client forever
*/
public void run() {
TProcessor processor = null;
TTransport inputTransport = null;
TTransport outputTransport = null;
TProtocol inputProtocol = null;
TProtocol outputProtocol = null;
try {
processor = processorFactory_.getProcessor(client);
inputTransport = inputTransportFactory_.getTransport(client);
outputTransport = outputTransportFactory_.getTransport(client);
inputProtocol = inputProtocolFactory_.getProtocol(inputTransport);
outputProtocol = outputProtocolFactory_.getProtocol(outputTransport);
// we check stopped_ first to make sure we're not supposed to be shutting
// down. this is necessary for graceful shutdown.
while (!stopped && processor.process(inputProtocol, outputProtocol)) {}
} catch (TTransportException ttx) {
// Assume the client died and continue silently
} catch (TException tx) {
LOG.error("Thrift error occurred during processing of message.", tx);
} catch (Exception x) {
LOG.error("Error occurred during processing of message.", x);
}
if (inputTransport != null) {
inputTransport.close();
}
if (outputTransport != null) {
outputTransport.close();
}
}
}
}

View File

@ -0,0 +1,226 @@
/**
* 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.thrift;
import java.io.IOException;
import java.security.PrivilegedExceptionAction;
import javax.servlet.ServletException;
import javax.servlet.http.HttpServletRequest;
import javax.servlet.http.HttpServletResponse;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.security.SecurityUtil;
import org.apache.hadoop.hbase.util.Base64;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.authorize.AuthorizationException;
import org.apache.hadoop.security.authorize.ProxyUsers;
import org.apache.thrift.TProcessor;
import org.apache.thrift.protocol.TProtocolFactory;
import org.apache.thrift.server.TServlet;
import org.ietf.jgss.GSSContext;
import org.ietf.jgss.GSSCredential;
import org.ietf.jgss.GSSException;
import org.ietf.jgss.GSSManager;
import org.ietf.jgss.GSSName;
import org.ietf.jgss.Oid;
/**
* Thrift Http Servlet is used for performing Kerberos authentication if security is enabled and
* also used for setting the user specified in "doAs" parameter.
*/
@InterfaceAudience.Private
public class ThriftHttpServlet extends TServlet {
private static final long serialVersionUID = 1L;
private static final Log LOG = LogFactory.getLog(ThriftHttpServlet.class.getName());
private transient final UserGroupInformation realUser;
private transient final Configuration conf;
private final boolean securityEnabled;
private final boolean doAsEnabled;
private transient ThriftServerRunner.HBaseHandler hbaseHandler;
private String outToken;
// HTTP Header related constants.
public static final String WWW_AUTHENTICATE = "WWW-Authenticate";
public static final String AUTHORIZATION = "Authorization";
public static final String NEGOTIATE = "Negotiate";
public ThriftHttpServlet(TProcessor processor, TProtocolFactory protocolFactory,
UserGroupInformation realUser, Configuration conf, ThriftServerRunner.HBaseHandler
hbaseHandler, boolean securityEnabled, boolean doAsEnabled) {
super(processor, protocolFactory);
this.realUser = realUser;
this.conf = conf;
this.hbaseHandler = hbaseHandler;
this.securityEnabled = securityEnabled;
this.doAsEnabled = doAsEnabled;
}
@Override
protected void doPost(HttpServletRequest request, HttpServletResponse response)
throws ServletException, IOException {
String effectiveUser = request.getRemoteUser();
if (securityEnabled) {
try {
// As Thrift HTTP transport doesn't support SPNEGO yet (THRIFT-889),
// Kerberos authentication is being done at servlet level.
effectiveUser = doKerberosAuth(request);
// It is standard for client applications expect this header.
// Please see http://tools.ietf.org/html/rfc4559 for more details.
response.addHeader(WWW_AUTHENTICATE, NEGOTIATE + " " + outToken);
} catch (HttpAuthenticationException e) {
LOG.error("Kerberos Authentication failed", e);
// Send a 401 to the client
response.setStatus(HttpServletResponse.SC_UNAUTHORIZED);
response.addHeader(WWW_AUTHENTICATE, NEGOTIATE);
response.getWriter().println("Authentication Error: " + e.getMessage());
return;
}
}
String doAsUserFromQuery = request.getHeader("doAs");
if(effectiveUser == null) {
effectiveUser = realUser.getShortUserName();
}
if (doAsUserFromQuery != null) {
if (!doAsEnabled) {
throw new ServletException("Support for proxyuser is not configured");
}
// The authenticated remote user is attempting to perform 'doAs' proxy user.
UserGroupInformation remoteUser = UserGroupInformation.createRemoteUser(effectiveUser);
// create and attempt to authorize a proxy user (the client is attempting
// to do proxy user)
UserGroupInformation ugi = UserGroupInformation.createProxyUser(doAsUserFromQuery,
remoteUser);
// validate the proxy user authorization
try {
ProxyUsers.authorize(ugi, request.getRemoteAddr(), conf);
} catch (AuthorizationException e) {
throw new ServletException(e.getMessage());
}
effectiveUser = doAsUserFromQuery;
}
hbaseHandler.setEffectiveUser(effectiveUser);
super.doPost(request, response);
}
/**
* Do the GSS-API kerberos authentication.
* We already have a logged in subject in the form of serviceUGI,
* which GSS-API will extract information from.
*/
private String doKerberosAuth(HttpServletRequest request)
throws HttpAuthenticationException {
HttpKerberosServerAction action = new HttpKerberosServerAction(request, realUser);
try {
String principal = realUser.doAs(action);
outToken = action.outToken;
return principal;
} catch (Exception e) {
LOG.error("Failed to perform authentication");
throw new HttpAuthenticationException(e);
}
}
private static class HttpKerberosServerAction implements PrivilegedExceptionAction<String> {
HttpServletRequest request;
UserGroupInformation serviceUGI;
String outToken = null;
HttpKerberosServerAction(HttpServletRequest request, UserGroupInformation serviceUGI) {
this.request = request;
this.serviceUGI = serviceUGI;
}
@Override
public String run() throws HttpAuthenticationException {
// Get own Kerberos credentials for accepting connection
GSSManager manager = GSSManager.getInstance();
GSSContext gssContext = null;
String serverPrincipal = SecurityUtil.getPrincipalWithoutRealm(serviceUGI.getUserName());
try {
// This Oid for Kerberos GSS-API mechanism.
Oid kerberosMechOid = new Oid("1.2.840.113554.1.2.2");
// Oid for SPNego GSS-API mechanism.
Oid spnegoMechOid = new Oid("1.3.6.1.5.5.2");
// Oid for kerberos principal name
Oid krb5PrincipalOid = new Oid("1.2.840.113554.1.2.2.1");
// GSS name for server
GSSName serverName = manager.createName(serverPrincipal, krb5PrincipalOid);
// GSS credentials for server
GSSCredential serverCreds = manager.createCredential(serverName,
GSSCredential.DEFAULT_LIFETIME,
new Oid[]{kerberosMechOid, spnegoMechOid},
GSSCredential.ACCEPT_ONLY);
// Create a GSS context
gssContext = manager.createContext(serverCreds);
// Get service ticket from the authorization header
String serviceTicketBase64 = getAuthHeader(request);
byte[] inToken = Base64.decode(serviceTicketBase64);
byte[] res = gssContext.acceptSecContext(inToken, 0, inToken.length);
if(res != null) {
outToken = Base64.encodeBytes(res).replace("\n", "");
}
// Authenticate or deny based on its context completion
if (!gssContext.isEstablished()) {
throw new HttpAuthenticationException("Kerberos authentication failed: " +
"unable to establish context with the service ticket " +
"provided by the client.");
}
return SecurityUtil.getUserFromPrincipal(gssContext.getSrcName().toString());
} catch (GSSException e) {
throw new HttpAuthenticationException("Kerberos authentication failed: ", e);
} finally {
if (gssContext != null) {
try {
gssContext.dispose();
} catch (GSSException e) {
LOG.warn("Error while disposing GSS Context", e);
}
}
}
}
/**
* Returns the base64 encoded auth header payload
*
* @throws HttpAuthenticationException if a remote or network exception occurs
*/
private String getAuthHeader(HttpServletRequest request)
throws HttpAuthenticationException {
String authHeader = request.getHeader(AUTHORIZATION);
// Each http request must have an Authorization header
if (authHeader == null || authHeader.isEmpty()) {
throw new HttpAuthenticationException("Authorization header received " +
"from the client is empty.");
}
String authHeaderBase64String;
int beginIndex = (NEGOTIATE + " ").length();
authHeaderBase64String = authHeader.substring(beginIndex);
// Authorization header must have a payload
if (authHeaderBase64String == null || authHeaderBase64String.isEmpty()) {
throw new HttpAuthenticationException("Authorization header received " +
"from the client does not contain any data.");
}
return authHeaderBase64String;
}
}
}

View File

@ -0,0 +1,90 @@
/*
* Copyright 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.thrift;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.CompatibilitySingletonFactory;
/**
* This class is for maintaining the various statistics of thrift server
* and publishing them through the metrics interfaces.
*/
@InterfaceAudience.Private
public class ThriftMetrics {
public enum ThriftServerType {
ONE,
TWO
}
public MetricsThriftServerSource getSource() {
return source;
}
public void setSource(MetricsThriftServerSource source) {
this.source = source;
}
private MetricsThriftServerSource source;
private final long slowResponseTime;
public static final String SLOW_RESPONSE_NANO_SEC =
"hbase.thrift.slow.response.nano.second";
public static final long DEFAULT_SLOW_RESPONSE_NANO_SEC = 10 * 1000 * 1000;
public ThriftMetrics(Configuration conf, ThriftServerType t) {
slowResponseTime = conf.getLong( SLOW_RESPONSE_NANO_SEC, DEFAULT_SLOW_RESPONSE_NANO_SEC);
if (t == ThriftServerType.ONE) {
source = CompatibilitySingletonFactory.getInstance(MetricsThriftServerSourceFactory.class).createThriftOneSource();
} else if (t == ThriftServerType.TWO) {
source = CompatibilitySingletonFactory.getInstance(MetricsThriftServerSourceFactory.class).createThriftTwoSource();
}
}
public void incTimeInQueue(long time) {
source.incTimeInQueue(time);
}
public void setCallQueueLen(int len) {
source.setCallQueueLen(len);
}
public void incNumRowKeysInBatchGet(int diff) {
source.incNumRowKeysInBatchGet(diff);
}
public void incNumRowKeysInBatchMutate(int diff) {
source.incNumRowKeysInBatchMutate(diff);
}
public void incMethodTime(String name, long time) {
source.incMethodTime(name, time);
// inc general processTime
source.incCall(time);
if (time > slowResponseTime) {
source.incSlowCall(time);
}
}
}

View File

@ -0,0 +1,245 @@
/**
* 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.thrift;
import java.util.Arrays;
import java.util.List;
import org.apache.commons.cli.CommandLine;
import org.apache.commons.cli.CommandLineParser;
import org.apache.commons.cli.HelpFormatter;
import org.apache.commons.cli.Options;
import org.apache.commons.cli.PosixParser;
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.HBaseInterfaceAudience;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.http.InfoServer;
import org.apache.hadoop.hbase.thrift.ThriftServerRunner.ImplType;
import org.apache.hadoop.hbase.util.VersionInfo;
import org.apache.hadoop.util.Shell.ExitCodeException;
/**
* ThriftServer- this class starts up a Thrift server which implements the
* Hbase API specified in the Hbase.thrift IDL file. The server runs in an
* independent process.
*/
@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.TOOLS)
public class ThriftServer {
private static final Log LOG = LogFactory.getLog(ThriftServer.class);
private static final String MIN_WORKERS_OPTION = "minWorkers";
private static final String MAX_WORKERS_OPTION = "workers";
private static final String MAX_QUEUE_SIZE_OPTION = "queue";
private static final String KEEP_ALIVE_SEC_OPTION = "keepAliveSec";
static final String BIND_OPTION = "bind";
static final String COMPACT_OPTION = "compact";
static final String FRAMED_OPTION = "framed";
static final String PORT_OPTION = "port";
private static final String DEFAULT_BIND_ADDR = "0.0.0.0";
private static final int DEFAULT_LISTEN_PORT = 9090;
private Configuration conf;
ThriftServerRunner serverRunner;
private InfoServer infoServer;
private static final String READ_TIMEOUT_OPTION = "readTimeout";
//
// Main program and support routines
//
public ThriftServer(Configuration conf) {
this.conf = HBaseConfiguration.create(conf);
}
private static void printUsageAndExit(Options options, int exitCode)
throws ExitCodeException {
HelpFormatter formatter = new HelpFormatter();
formatter.printHelp("Thrift", null, options,
"To start the Thrift server run 'bin/hbase-daemon.sh start thrift'\n" +
"To shutdown the thrift server run 'bin/hbase-daemon.sh stop " +
"thrift' or send a kill signal to the thrift server pid",
true);
throw new ExitCodeException(exitCode, "");
}
/**
* Start up or shuts down the Thrift server, depending on the arguments.
* @param args
*/
void doMain(final String[] args) throws Exception {
processOptions(args);
serverRunner = new ThriftServerRunner(conf);
// Put up info server.
int port = conf.getInt("hbase.thrift.info.port", 9095);
if (port >= 0) {
conf.setLong("startcode", System.currentTimeMillis());
String a = conf.get("hbase.thrift.info.bindAddress", "0.0.0.0");
infoServer = new InfoServer("thrift", a, port, false, conf);
infoServer.setAttribute("hbase.conf", conf);
infoServer.start();
}
serverRunner.run();
}
/**
* Parse the command line options to set parameters the conf.
*/
private void processOptions(final String[] args) throws Exception {
Options options = new Options();
options.addOption("b", BIND_OPTION, true, "Address to bind " +
"the Thrift server to. [default: " + DEFAULT_BIND_ADDR + "]");
options.addOption("p", PORT_OPTION, true, "Port to bind to [default: " +
DEFAULT_LISTEN_PORT + "]");
options.addOption("f", FRAMED_OPTION, false, "Use framed transport");
options.addOption("c", COMPACT_OPTION, false, "Use the compact protocol");
options.addOption("h", "help", false, "Print help information");
options.addOption(null, "infoport", true, "Port for web UI");
options.addOption("m", MIN_WORKERS_OPTION, true,
"The minimum number of worker threads for " +
ImplType.THREAD_POOL.simpleClassName());
options.addOption("w", MAX_WORKERS_OPTION, true,
"The maximum number of worker threads for " +
ImplType.THREAD_POOL.simpleClassName());
options.addOption("q", MAX_QUEUE_SIZE_OPTION, true,
"The maximum number of queued requests in " +
ImplType.THREAD_POOL.simpleClassName());
options.addOption("k", KEEP_ALIVE_SEC_OPTION, true,
"The amount of time in secods to keep a thread alive when idle in " +
ImplType.THREAD_POOL.simpleClassName());
options.addOption("t", READ_TIMEOUT_OPTION, true,
"Amount of time in milliseconds before a server thread will timeout " +
"waiting for client to send data on a connected socket. Currently, " +
"only applies to TBoundedThreadPoolServer");
options.addOptionGroup(ImplType.createOptionGroup());
CommandLineParser parser = new PosixParser();
CommandLine cmd = parser.parse(options, args);
// This is so complicated to please both bin/hbase and bin/hbase-daemon.
// hbase-daemon provides "start" and "stop" arguments
// hbase should print the help if no argument is provided
List<String> commandLine = Arrays.asList(args);
boolean stop = commandLine.contains("stop");
boolean start = commandLine.contains("start");
boolean invalidStartStop = (start && stop) || (!start && !stop);
if (cmd.hasOption("help") || invalidStartStop) {
if (invalidStartStop) {
LOG.error("Exactly one of 'start' and 'stop' has to be specified");
}
printUsageAndExit(options, 1);
}
// Get port to bind to
try {
if (cmd.hasOption(PORT_OPTION)) {
int listenPort = Integer.parseInt(cmd.getOptionValue(PORT_OPTION));
conf.setInt(ThriftServerRunner.PORT_CONF_KEY, listenPort);
}
} catch (NumberFormatException e) {
LOG.error("Could not parse the value provided for the port option", e);
printUsageAndExit(options, -1);
}
// check for user-defined info server port setting, if so override the conf
try {
if (cmd.hasOption("infoport")) {
String val = cmd.getOptionValue("infoport");
conf.setInt("hbase.thrift.info.port", Integer.parseInt(val));
LOG.debug("Web UI port set to " + val);
}
} catch (NumberFormatException e) {
LOG.error("Could not parse the value provided for the infoport option", e);
printUsageAndExit(options, -1);
}
// Make optional changes to the configuration based on command-line options
optionToConf(cmd, MIN_WORKERS_OPTION,
conf, TBoundedThreadPoolServer.MIN_WORKER_THREADS_CONF_KEY);
optionToConf(cmd, MAX_WORKERS_OPTION,
conf, TBoundedThreadPoolServer.MAX_WORKER_THREADS_CONF_KEY);
optionToConf(cmd, MAX_QUEUE_SIZE_OPTION,
conf, TBoundedThreadPoolServer.MAX_QUEUED_REQUESTS_CONF_KEY);
optionToConf(cmd, KEEP_ALIVE_SEC_OPTION,
conf, TBoundedThreadPoolServer.THREAD_KEEP_ALIVE_TIME_SEC_CONF_KEY);
optionToConf(cmd, READ_TIMEOUT_OPTION, conf,
ThriftServerRunner.THRIFT_SERVER_SOCKET_READ_TIMEOUT_KEY);
// Set general thrift server options
boolean compact = cmd.hasOption(COMPACT_OPTION) ||
conf.getBoolean(ThriftServerRunner.COMPACT_CONF_KEY, false);
conf.setBoolean(ThriftServerRunner.COMPACT_CONF_KEY, compact);
boolean framed = cmd.hasOption(FRAMED_OPTION) ||
conf.getBoolean(ThriftServerRunner.FRAMED_CONF_KEY, false);
conf.setBoolean(ThriftServerRunner.FRAMED_CONF_KEY, framed);
if (cmd.hasOption(BIND_OPTION)) {
conf.set(ThriftServerRunner.BIND_CONF_KEY, cmd.getOptionValue(BIND_OPTION));
}
ImplType.setServerImpl(cmd, conf);
}
public void stop() {
if (this.infoServer != null) {
LOG.info("Stopping infoServer");
try {
this.infoServer.stop();
} catch (Exception ex) {
ex.printStackTrace();
}
}
serverRunner.shutdown();
}
private static void optionToConf(CommandLine cmd, String option,
Configuration conf, String destConfKey) {
if (cmd.hasOption(option)) {
String value = cmd.getOptionValue(option);
LOG.info("Set configuration key:" + destConfKey + " value:" + value);
conf.set(destConfKey, value);
}
}
/**
* @param args
* @throws Exception
*/
public static void main(String [] args) throws Exception {
VersionInfo.logVersion();
try {
new ThriftServer(HBaseConfiguration.create()).doMain(args);
} catch (ExitCodeException ex) {
System.exit(ex.getExitCode());
}
}
}

File diff suppressed because it is too large Load Diff

View File

@ -0,0 +1,235 @@
/**
* 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.thrift;
import static org.apache.hadoop.hbase.util.Bytes.getBytes;
import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.List;
import java.util.Locale;
import java.util.TreeMap;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.client.Append;
import org.apache.hadoop.hbase.client.Increment;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.io.compress.Compression;
import org.apache.hadoop.hbase.regionserver.BloomType;
import org.apache.hadoop.hbase.thrift.generated.ColumnDescriptor;
import org.apache.hadoop.hbase.thrift.generated.IllegalArgument;
import org.apache.hadoop.hbase.thrift.generated.TAppend;
import org.apache.hadoop.hbase.thrift.generated.TCell;
import org.apache.hadoop.hbase.thrift.generated.TColumn;
import org.apache.hadoop.hbase.thrift.generated.TIncrement;
import org.apache.hadoop.hbase.thrift.generated.TRowResult;
import org.apache.hadoop.hbase.util.Bytes;
@InterfaceAudience.Private
public class ThriftUtilities {
/**
* This utility method creates a new Hbase HColumnDescriptor object based on a
* Thrift ColumnDescriptor "struct".
*
* @param in
* Thrift ColumnDescriptor object
* @return HColumnDescriptor
* @throws IllegalArgument
*/
static public HColumnDescriptor colDescFromThrift(ColumnDescriptor in)
throws IllegalArgument {
Compression.Algorithm comp =
Compression.getCompressionAlgorithmByName(in.compression.toLowerCase(Locale.ROOT));
BloomType bt =
BloomType.valueOf(in.bloomFilterType);
if (in.name == null || !in.name.hasRemaining()) {
throw new IllegalArgument("column name is empty");
}
byte [] parsedName = KeyValue.parseColumn(Bytes.getBytes(in.name))[0];
HColumnDescriptor col = new HColumnDescriptor(parsedName)
.setMaxVersions(in.maxVersions)
.setCompressionType(comp)
.setInMemory(in.inMemory)
.setBlockCacheEnabled(in.blockCacheEnabled)
.setTimeToLive(in.timeToLive > 0 ? in.timeToLive : Integer.MAX_VALUE)
.setBloomFilterType(bt);
return col;
}
/**
* This utility method creates a new Thrift ColumnDescriptor "struct" based on
* an Hbase HColumnDescriptor object.
*
* @param in
* Hbase HColumnDescriptor object
* @return Thrift ColumnDescriptor
*/
static public ColumnDescriptor colDescFromHbase(HColumnDescriptor in) {
ColumnDescriptor col = new ColumnDescriptor();
col.name = ByteBuffer.wrap(Bytes.add(in.getName(), KeyValue.COLUMN_FAMILY_DELIM_ARRAY));
col.maxVersions = in.getMaxVersions();
col.compression = in.getCompressionType().toString();
col.inMemory = in.isInMemory();
col.blockCacheEnabled = in.isBlockCacheEnabled();
col.bloomFilterType = in.getBloomFilterType().toString();
col.timeToLive = in.getTimeToLive();
return col;
}
/**
* This utility method creates a list of Thrift TCell "struct" based on
* an Hbase Cell object. The empty list is returned if the input is null.
*
* @param in
* Hbase Cell object
* @return Thrift TCell array
*/
static public List<TCell> cellFromHBase(Cell in) {
List<TCell> list = new ArrayList<TCell>(1);
if (in != null) {
list.add(new TCell(ByteBuffer.wrap(CellUtil.cloneValue(in)), in.getTimestamp()));
}
return list;
}
/**
* This utility method creates a list of Thrift TCell "struct" based on
* an Hbase Cell array. The empty list is returned if the input is null.
* @param in Hbase Cell array
* @return Thrift TCell array
*/
static public List<TCell> cellFromHBase(Cell[] in) {
List<TCell> list = null;
if (in != null) {
list = new ArrayList<TCell>(in.length);
for (int i = 0; i < in.length; i++) {
list.add(new TCell(ByteBuffer.wrap(CellUtil.cloneValue(in[i])), in[i].getTimestamp()));
}
} else {
list = new ArrayList<TCell>(0);
}
return list;
}
/**
* This utility method creates a list of Thrift TRowResult "struct" based on
* an Hbase RowResult object. The empty list is returned if the input is
* null.
*
* @param in
* Hbase RowResult object
* @param sortColumns
* This boolean dictates if row data is returned in a sorted order
* sortColumns = True will set TRowResult's sortedColumns member
* which is an ArrayList of TColumn struct
* sortColumns = False will set TRowResult's columns member which is
* a map of columnName and TCell struct
* @return Thrift TRowResult array
*/
static public List<TRowResult> rowResultFromHBase(Result[] in, boolean sortColumns) {
List<TRowResult> results = new ArrayList<TRowResult>();
for ( Result result_ : in) {
if(result_ == null || result_.isEmpty()) {
continue;
}
TRowResult result = new TRowResult();
result.row = ByteBuffer.wrap(result_.getRow());
if (sortColumns) {
result.sortedColumns = new ArrayList<TColumn>();
for (Cell kv : result_.rawCells()) {
result.sortedColumns.add(new TColumn(
ByteBuffer.wrap(KeyValue.makeColumn(CellUtil.cloneFamily(kv),
CellUtil.cloneQualifier(kv))),
new TCell(ByteBuffer.wrap(CellUtil.cloneValue(kv)), kv.getTimestamp())));
}
} else {
result.columns = new TreeMap<ByteBuffer, TCell>();
for (Cell kv : result_.rawCells()) {
result.columns.put(
ByteBuffer.wrap(KeyValue.makeColumn(CellUtil.cloneFamily(kv),
CellUtil.cloneQualifier(kv))),
new TCell(ByteBuffer.wrap(CellUtil.cloneValue(kv)), kv.getTimestamp()));
}
}
results.add(result);
}
return results;
}
/**
* This utility method creates a list of Thrift TRowResult "struct" based on
* an array of Hbase RowResult objects. The empty list is returned if the input is
* null.
*
* @param in
* Array of Hbase RowResult objects
* @return Thrift TRowResult array
*/
static public List<TRowResult> rowResultFromHBase(Result[] in) {
return rowResultFromHBase(in, false);
}
static public List<TRowResult> rowResultFromHBase(Result in) {
Result [] result = { in };
return rowResultFromHBase(result);
}
/**
* From a {@link TIncrement} create an {@link Increment}.
* @param tincrement the Thrift version of an increment
* @return an increment that the {@link TIncrement} represented.
*/
public static Increment incrementFromThrift(TIncrement tincrement) {
Increment inc = new Increment(tincrement.getRow());
byte[][] famAndQf = KeyValue.parseColumn(tincrement.getColumn());
if (famAndQf.length != 2) return null;
inc.addColumn(famAndQf[0], famAndQf[1], tincrement.getAmmount());
return inc;
}
/**
* From a {@link TAppend} create an {@link Append}.
* @param tappend the Thrift version of an append.
* @return an increment that the {@link TAppend} represented.
*/
public static Append appendFromThrift(TAppend tappend) {
Append append = new Append(tappend.getRow());
List<ByteBuffer> columns = tappend.getColumns();
List<ByteBuffer> values = tappend.getValues();
if (columns.size() != values.size()) {
throw new IllegalArgumentException(
"Sizes of columns and values in tappend object are not matching");
}
int length = columns.size();
for (int i = 0; i < length; i++) {
byte[][] famAndQf = KeyValue.parseColumn(getBytes(columns.get(i)));
append.add(famAndQf[0], famAndQf[1], getBytes(values.get(i)));
}
return append;
}
}

View File

@ -0,0 +1,402 @@
/**
* Autogenerated by Thrift Compiler (0.9.3)
*
* DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
* @generated
*/
package org.apache.hadoop.hbase.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 javax.annotation.Generated;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
/**
* An AlreadyExists exceptions signals that a table with the specified
* name already exists
*/
@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2016-05-25")
public class AlreadyExists extends TException implements org.apache.thrift.TBase<AlreadyExists, AlreadyExists._Fields>, java.io.Serializable, Cloneable, Comparable<AlreadyExists> {
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);
private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
static {
schemes.put(StandardScheme.class, new AlreadyExistsStandardSchemeFactory());
schemes.put(TupleScheme.class, new AlreadyExistsTupleSchemeFactory());
}
public String message; // required
/** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
public enum _Fields implements org.apache.thrift.TFieldIdEnum {
MESSAGE((short)1, "message");
private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
static {
for (_Fields field : EnumSet.allOf(_Fields.class)) {
byName.put(field.getFieldName(), field);
}
}
/**
* Find the _Fields constant that matches fieldId, or null if its not found.
*/
public static _Fields findByThriftId(int fieldId) {
switch(fieldId) {
case 1: // MESSAGE
return MESSAGE;
default:
return null;
}
}
/**
* Find the _Fields constant that matches fieldId, throwing an exception
* if it is not found.
*/
public static _Fields findByThriftIdOrThrow(int fieldId) {
_Fields fields = findByThriftId(fieldId);
if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
return fields;
}
/**
* Find the _Fields constant that matches name, or null if its not found.
*/
public static _Fields findByName(String name) {
return byName.get(name);
}
private final short _thriftId;
private final String _fieldName;
_Fields(short thriftId, String fieldName) {
_thriftId = thriftId;
_fieldName = fieldName;
}
public short getThriftFieldId() {
return _thriftId;
}
public String getFieldName() {
return _fieldName;
}
}
// isset id assignments
public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
static {
Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
tmpMap.put(_Fields.MESSAGE, new org.apache.thrift.meta_data.FieldMetaData("message", org.apache.thrift.TFieldRequirementType.DEFAULT,
new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
metaDataMap = Collections.unmodifiableMap(tmpMap);
org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(AlreadyExists.class, metaDataMap);
}
public AlreadyExists() {
}
public AlreadyExists(
String message)
{
this();
this.message = message;
}
/**
* Performs a deep copy on <i>other</i>.
*/
public AlreadyExists(AlreadyExists other) {
if (other.isSetMessage()) {
this.message = other.message;
}
}
public AlreadyExists deepCopy() {
return new AlreadyExists(this);
}
@Override
public void clear() {
this.message = null;
}
public String getMessage() {
return this.message;
}
public AlreadyExists 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 AlreadyExists)
return this.equals((AlreadyExists)that);
return false;
}
public boolean equals(AlreadyExists 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() {
List<Object> list = new ArrayList<Object>();
boolean present_message = true && (isSetMessage());
list.add(present_message);
if (present_message)
list.add(message);
return list.hashCode();
}
@Override
public int compareTo(AlreadyExists 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("AlreadyExists(");
boolean first = true;
sb.append("message:");
if (this.message == null) {
sb.append("null");
} else {
sb.append(this.message);
}
first = false;
sb.append(")");
return sb.toString();
}
public void validate() throws org.apache.thrift.TException {
// check for required fields
// check for sub-struct validity
}
private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
try {
write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
} catch (org.apache.thrift.TException te) {
throw new java.io.IOException(te);
}
}
private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
try {
read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
} catch (org.apache.thrift.TException te) {
throw new java.io.IOException(te);
}
}
private static class AlreadyExistsStandardSchemeFactory implements SchemeFactory {
public AlreadyExistsStandardScheme getScheme() {
return new AlreadyExistsStandardScheme();
}
}
private static class AlreadyExistsStandardScheme extends StandardScheme<AlreadyExists> {
public void read(org.apache.thrift.protocol.TProtocol iprot, AlreadyExists 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, AlreadyExists struct) throws org.apache.thrift.TException {
struct.validate();
oprot.writeStructBegin(STRUCT_DESC);
if (struct.message != null) {
oprot.writeFieldBegin(MESSAGE_FIELD_DESC);
oprot.writeString(struct.message);
oprot.writeFieldEnd();
}
oprot.writeFieldStop();
oprot.writeStructEnd();
}
}
private static class AlreadyExistsTupleSchemeFactory implements SchemeFactory {
public AlreadyExistsTupleScheme getScheme() {
return new AlreadyExistsTupleScheme();
}
}
private static class AlreadyExistsTupleScheme extends TupleScheme<AlreadyExists> {
@Override
public void write(org.apache.thrift.protocol.TProtocol prot, AlreadyExists 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, AlreadyExists 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);
}
}
}
}

View File

@ -0,0 +1,570 @@
/**
* Autogenerated by Thrift Compiler (0.9.3)
*
* DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
* @generated
*/
package org.apache.hadoop.hbase.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 javax.annotation.Generated;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
/**
* A BatchMutation object is used to apply a number of Mutations to a single row.
*/
@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2016-05-25")
public class BatchMutation implements org.apache.thrift.TBase<BatchMutation, BatchMutation._Fields>, java.io.Serializable, Cloneable, Comparable<BatchMutation> {
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);
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)2);
private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
static {
schemes.put(StandardScheme.class, new BatchMutationStandardSchemeFactory());
schemes.put(TupleScheme.class, new BatchMutationTupleSchemeFactory());
}
public ByteBuffer row; // required
public List<Mutation> 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 {
ROW((short)1, "row"),
MUTATIONS((short)2, "mutations");
private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
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: // 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.ROW, new org.apache.thrift.meta_data.FieldMetaData("row", org.apache.thrift.TFieldRequirementType.DEFAULT,
new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , "Text")));
tmpMap.put(_Fields.MUTATIONS, new org.apache.thrift.meta_data.FieldMetaData("mutations", org.apache.thrift.TFieldRequirementType.DEFAULT,
new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST,
new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, Mutation.class))));
metaDataMap = Collections.unmodifiableMap(tmpMap);
org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(BatchMutation.class, metaDataMap);
}
public BatchMutation() {
}
public BatchMutation(
ByteBuffer row,
List<Mutation> mutations)
{
this();
this.row = org.apache.thrift.TBaseHelper.copyBinary(row);
this.mutations = mutations;
}
/**
* Performs a deep copy on <i>other</i>.
*/
public BatchMutation(BatchMutation other) {
if (other.isSetRow()) {
this.row = other.row;
}
if (other.isSetMutations()) {
List<Mutation> __this__mutations = new ArrayList<Mutation>(other.mutations.size());
for (Mutation other_element : other.mutations) {
__this__mutations.add(new Mutation(other_element));
}
this.mutations = __this__mutations;
}
}
public BatchMutation deepCopy() {
return new BatchMutation(this);
}
@Override
public void clear() {
this.row = null;
this.mutations = null;
}
public byte[] getRow() {
setRow(org.apache.thrift.TBaseHelper.rightSize(row));
return row == null ? null : row.array();
}
public ByteBuffer bufferForRow() {
return org.apache.thrift.TBaseHelper.copyBinary(row);
}
public BatchMutation setRow(byte[] row) {
this.row = row == null ? (ByteBuffer)null : ByteBuffer.wrap(Arrays.copyOf(row, row.length));
return this;
}
public BatchMutation setRow(ByteBuffer row) {
this.row = org.apache.thrift.TBaseHelper.copyBinary(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 int getMutationsSize() {
return (this.mutations == null) ? 0 : this.mutations.size();
}
public java.util.Iterator<Mutation> getMutationsIterator() {
return (this.mutations == null) ? null : this.mutations.iterator();
}
public void addToMutations(Mutation elem) {
if (this.mutations == null) {
this.mutations = new ArrayList<Mutation>();
}
this.mutations.add(elem);
}
public List<Mutation> getMutations() {
return this.mutations;
}
public BatchMutation setMutations(List<Mutation> 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 ROW:
if (value == null) {
unsetRow();
} else {
setRow((ByteBuffer)value);
}
break;
case MUTATIONS:
if (value == null) {
unsetMutations();
} else {
setMutations((List<Mutation>)value);
}
break;
}
}
public Object getFieldValue(_Fields field) {
switch (field) {
case ROW:
return getRow();
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 ROW:
return isSetRow();
case MUTATIONS:
return isSetMutations();
}
throw new IllegalStateException();
}
@Override
public boolean equals(Object that) {
if (that == null)
return false;
if (that instanceof BatchMutation)
return this.equals((BatchMutation)that);
return false;
}
public boolean equals(BatchMutation 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_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() {
List<Object> list = new ArrayList<Object>();
boolean present_row = true && (isSetRow());
list.add(present_row);
if (present_row)
list.add(row);
boolean present_mutations = true && (isSetMutations());
list.add(present_mutations);
if (present_mutations)
list.add(mutations);
return list.hashCode();
}
@Override
public int compareTo(BatchMutation 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(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("BatchMutation(");
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("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
// 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 BatchMutationStandardSchemeFactory implements SchemeFactory {
public BatchMutationStandardScheme getScheme() {
return new BatchMutationStandardScheme();
}
}
private static class BatchMutationStandardScheme extends StandardScheme<BatchMutation> {
public void read(org.apache.thrift.protocol.TProtocol iprot, BatchMutation 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: // MUTATIONS
if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
{
org.apache.thrift.protocol.TList _list0 = iprot.readListBegin();
struct.mutations = new ArrayList<Mutation>(_list0.size);
Mutation _elem1;
for (int _i2 = 0; _i2 < _list0.size; ++_i2)
{
_elem1 = new Mutation();
_elem1.read(iprot);
struct.mutations.add(_elem1);
}
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, BatchMutation 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.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 (Mutation _iter3 : struct.mutations)
{
_iter3.write(oprot);
}
oprot.writeListEnd();
}
oprot.writeFieldEnd();
}
oprot.writeFieldStop();
oprot.writeStructEnd();
}
}
private static class BatchMutationTupleSchemeFactory implements SchemeFactory {
public BatchMutationTupleScheme getScheme() {
return new BatchMutationTupleScheme();
}
}
private static class BatchMutationTupleScheme extends TupleScheme<BatchMutation> {
@Override
public void write(org.apache.thrift.protocol.TProtocol prot, BatchMutation struct) throws org.apache.thrift.TException {
TTupleProtocol oprot = (TTupleProtocol) prot;
BitSet optionals = new BitSet();
if (struct.isSetRow()) {
optionals.set(0);
}
if (struct.isSetMutations()) {
optionals.set(1);
}
oprot.writeBitSet(optionals, 2);
if (struct.isSetRow()) {
oprot.writeBinary(struct.row);
}
if (struct.isSetMutations()) {
{
oprot.writeI32(struct.mutations.size());
for (Mutation _iter4 : struct.mutations)
{
_iter4.write(oprot);
}
}
}
}
@Override
public void read(org.apache.thrift.protocol.TProtocol prot, BatchMutation struct) throws org.apache.thrift.TException {
TTupleProtocol iprot = (TTupleProtocol) prot;
BitSet incoming = iprot.readBitSet(2);
if (incoming.get(0)) {
struct.row = iprot.readBinary();
struct.setRowIsSet(true);
}
if (incoming.get(1)) {
{
org.apache.thrift.protocol.TList _list5 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
struct.mutations = new ArrayList<Mutation>(_list5.size);
Mutation _elem6;
for (int _i7 = 0; _i7 < _list5.size; ++_i7)
{
_elem6 = new Mutation();
_elem6.read(iprot);
struct.mutations.add(_elem6);
}
}
struct.setMutationsIsSet(true);
}
}
}
}

File diff suppressed because it is too large Load Diff

View File

@ -0,0 +1,403 @@
/**
* Autogenerated by Thrift Compiler (0.9.3)
*
* DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
* @generated
*/
package org.apache.hadoop.hbase.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 javax.annotation.Generated;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
/**
* An IOError exception signals that an error occurred communicating
* to the Hbase master or an Hbase region server. Also used to return
* more general Hbase error conditions.
*/
@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2016-05-25")
public class IOError extends TException implements org.apache.thrift.TBase<IOError, IOError._Fields>, java.io.Serializable, Cloneable, Comparable<IOError> {
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);
private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
static {
schemes.put(StandardScheme.class, new IOErrorStandardSchemeFactory());
schemes.put(TupleScheme.class, new IOErrorTupleSchemeFactory());
}
public String message; // required
/** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
public enum _Fields implements org.apache.thrift.TFieldIdEnum {
MESSAGE((short)1, "message");
private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
static {
for (_Fields field : EnumSet.allOf(_Fields.class)) {
byName.put(field.getFieldName(), field);
}
}
/**
* Find the _Fields constant that matches fieldId, or null if its not found.
*/
public static _Fields findByThriftId(int fieldId) {
switch(fieldId) {
case 1: // MESSAGE
return MESSAGE;
default:
return null;
}
}
/**
* Find the _Fields constant that matches fieldId, throwing an exception
* if it is not found.
*/
public static _Fields findByThriftIdOrThrow(int fieldId) {
_Fields fields = findByThriftId(fieldId);
if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
return fields;
}
/**
* Find the _Fields constant that matches name, or null if its not found.
*/
public static _Fields findByName(String name) {
return byName.get(name);
}
private final short _thriftId;
private final String _fieldName;
_Fields(short thriftId, String fieldName) {
_thriftId = thriftId;
_fieldName = fieldName;
}
public short getThriftFieldId() {
return _thriftId;
}
public String getFieldName() {
return _fieldName;
}
}
// isset id assignments
public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
static {
Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
tmpMap.put(_Fields.MESSAGE, new org.apache.thrift.meta_data.FieldMetaData("message", org.apache.thrift.TFieldRequirementType.DEFAULT,
new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
metaDataMap = Collections.unmodifiableMap(tmpMap);
org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(IOError.class, metaDataMap);
}
public IOError() {
}
public IOError(
String message)
{
this();
this.message = message;
}
/**
* Performs a deep copy on <i>other</i>.
*/
public IOError(IOError other) {
if (other.isSetMessage()) {
this.message = other.message;
}
}
public IOError deepCopy() {
return new IOError(this);
}
@Override
public void clear() {
this.message = null;
}
public String getMessage() {
return this.message;
}
public IOError 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 IOError)
return this.equals((IOError)that);
return false;
}
public boolean equals(IOError 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() {
List<Object> list = new ArrayList<Object>();
boolean present_message = true && (isSetMessage());
list.add(present_message);
if (present_message)
list.add(message);
return list.hashCode();
}
@Override
public int compareTo(IOError 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("IOError(");
boolean first = true;
sb.append("message:");
if (this.message == null) {
sb.append("null");
} else {
sb.append(this.message);
}
first = false;
sb.append(")");
return sb.toString();
}
public void validate() throws org.apache.thrift.TException {
// check for required fields
// check for sub-struct validity
}
private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
try {
write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
} catch (org.apache.thrift.TException te) {
throw new java.io.IOException(te);
}
}
private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
try {
read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
} catch (org.apache.thrift.TException te) {
throw new java.io.IOException(te);
}
}
private static class IOErrorStandardSchemeFactory implements SchemeFactory {
public IOErrorStandardScheme getScheme() {
return new IOErrorStandardScheme();
}
}
private static class IOErrorStandardScheme extends StandardScheme<IOError> {
public void read(org.apache.thrift.protocol.TProtocol iprot, IOError 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, IOError struct) throws org.apache.thrift.TException {
struct.validate();
oprot.writeStructBegin(STRUCT_DESC);
if (struct.message != null) {
oprot.writeFieldBegin(MESSAGE_FIELD_DESC);
oprot.writeString(struct.message);
oprot.writeFieldEnd();
}
oprot.writeFieldStop();
oprot.writeStructEnd();
}
}
private static class IOErrorTupleSchemeFactory implements SchemeFactory {
public IOErrorTupleScheme getScheme() {
return new IOErrorTupleScheme();
}
}
private static class IOErrorTupleScheme extends TupleScheme<IOError> {
@Override
public void write(org.apache.thrift.protocol.TProtocol prot, IOError 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, IOError 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);
}
}
}
}

View File

@ -0,0 +1,402 @@
/**
* Autogenerated by Thrift Compiler (0.9.3)
*
* DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
* @generated
*/
package org.apache.hadoop.hbase.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 javax.annotation.Generated;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
/**
* An IllegalArgument exception indicates an illegal or invalid
* argument was passed into a procedure.
*/
@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2016-05-25")
public class IllegalArgument extends TException implements org.apache.thrift.TBase<IllegalArgument, IllegalArgument._Fields>, java.io.Serializable, Cloneable, Comparable<IllegalArgument> {
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);
private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
static {
schemes.put(StandardScheme.class, new IllegalArgumentStandardSchemeFactory());
schemes.put(TupleScheme.class, new IllegalArgumentTupleSchemeFactory());
}
public String message; // required
/** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
public enum _Fields implements org.apache.thrift.TFieldIdEnum {
MESSAGE((short)1, "message");
private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
static {
for (_Fields field : EnumSet.allOf(_Fields.class)) {
byName.put(field.getFieldName(), field);
}
}
/**
* Find the _Fields constant that matches fieldId, or null if its not found.
*/
public static _Fields findByThriftId(int fieldId) {
switch(fieldId) {
case 1: // MESSAGE
return MESSAGE;
default:
return null;
}
}
/**
* Find the _Fields constant that matches fieldId, throwing an exception
* if it is not found.
*/
public static _Fields findByThriftIdOrThrow(int fieldId) {
_Fields fields = findByThriftId(fieldId);
if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
return fields;
}
/**
* Find the _Fields constant that matches name, or null if its not found.
*/
public static _Fields findByName(String name) {
return byName.get(name);
}
private final short _thriftId;
private final String _fieldName;
_Fields(short thriftId, String fieldName) {
_thriftId = thriftId;
_fieldName = fieldName;
}
public short getThriftFieldId() {
return _thriftId;
}
public String getFieldName() {
return _fieldName;
}
}
// isset id assignments
public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
static {
Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
tmpMap.put(_Fields.MESSAGE, new org.apache.thrift.meta_data.FieldMetaData("message", org.apache.thrift.TFieldRequirementType.DEFAULT,
new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
metaDataMap = Collections.unmodifiableMap(tmpMap);
org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(IllegalArgument.class, metaDataMap);
}
public IllegalArgument() {
}
public IllegalArgument(
String message)
{
this();
this.message = message;
}
/**
* Performs a deep copy on <i>other</i>.
*/
public IllegalArgument(IllegalArgument other) {
if (other.isSetMessage()) {
this.message = other.message;
}
}
public IllegalArgument deepCopy() {
return new IllegalArgument(this);
}
@Override
public void clear() {
this.message = null;
}
public String getMessage() {
return this.message;
}
public IllegalArgument 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 IllegalArgument)
return this.equals((IllegalArgument)that);
return false;
}
public boolean equals(IllegalArgument 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() {
List<Object> list = new ArrayList<Object>();
boolean present_message = true && (isSetMessage());
list.add(present_message);
if (present_message)
list.add(message);
return list.hashCode();
}
@Override
public int compareTo(IllegalArgument 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("IllegalArgument(");
boolean first = true;
sb.append("message:");
if (this.message == null) {
sb.append("null");
} else {
sb.append(this.message);
}
first = false;
sb.append(")");
return sb.toString();
}
public void validate() throws org.apache.thrift.TException {
// check for required fields
// check for sub-struct validity
}
private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
try {
write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
} catch (org.apache.thrift.TException te) {
throw new java.io.IOException(te);
}
}
private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
try {
read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
} catch (org.apache.thrift.TException te) {
throw new java.io.IOException(te);
}
}
private static class IllegalArgumentStandardSchemeFactory implements SchemeFactory {
public IllegalArgumentStandardScheme getScheme() {
return new IllegalArgumentStandardScheme();
}
}
private static class IllegalArgumentStandardScheme extends StandardScheme<IllegalArgument> {
public void read(org.apache.thrift.protocol.TProtocol iprot, IllegalArgument 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, IllegalArgument struct) throws org.apache.thrift.TException {
struct.validate();
oprot.writeStructBegin(STRUCT_DESC);
if (struct.message != null) {
oprot.writeFieldBegin(MESSAGE_FIELD_DESC);
oprot.writeString(struct.message);
oprot.writeFieldEnd();
}
oprot.writeFieldStop();
oprot.writeStructEnd();
}
}
private static class IllegalArgumentTupleSchemeFactory implements SchemeFactory {
public IllegalArgumentTupleScheme getScheme() {
return new IllegalArgumentTupleScheme();
}
}
private static class IllegalArgumentTupleScheme extends TupleScheme<IllegalArgument> {
@Override
public void write(org.apache.thrift.protocol.TProtocol prot, IllegalArgument 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, IllegalArgument 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);
}
}
}
}

View File

@ -0,0 +1,732 @@
/**
* Autogenerated by Thrift Compiler (0.9.3)
*
* DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
* @generated
*/
package org.apache.hadoop.hbase.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 javax.annotation.Generated;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
/**
* A Mutation object is used to either update or delete a column-value.
*/
@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2016-05-25")
public class Mutation implements org.apache.thrift.TBase<Mutation, Mutation._Fields>, java.io.Serializable, Cloneable, Comparable<Mutation> {
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);
private static final org.apache.thrift.protocol.TField COLUMN_FIELD_DESC = new org.apache.thrift.protocol.TField("column", org.apache.thrift.protocol.TType.STRING, (short)2);
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)3);
private static final org.apache.thrift.protocol.TField WRITE_TO_WAL_FIELD_DESC = new org.apache.thrift.protocol.TField("writeToWAL", org.apache.thrift.protocol.TType.BOOL, (short)4);
private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
static {
schemes.put(StandardScheme.class, new MutationStandardSchemeFactory());
schemes.put(TupleScheme.class, new MutationTupleSchemeFactory());
}
public boolean isDelete; // required
public ByteBuffer column; // required
public ByteBuffer value; // required
public boolean writeToWAL; // 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 {
IS_DELETE((short)1, "isDelete"),
COLUMN((short)2, "column"),
VALUE((short)3, "value"),
WRITE_TO_WAL((short)4, "writeToWAL");
private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
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: // IS_DELETE
return IS_DELETE;
case 2: // COLUMN
return COLUMN;
case 3: // VALUE
return VALUE;
case 4: // WRITE_TO_WAL
return WRITE_TO_WAL;
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 __ISDELETE_ISSET_ID = 0;
private static final int __WRITETOWAL_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.IS_DELETE, new org.apache.thrift.meta_data.FieldMetaData("isDelete", org.apache.thrift.TFieldRequirementType.DEFAULT,
new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL)));
tmpMap.put(_Fields.COLUMN, new org.apache.thrift.meta_data.FieldMetaData("column", org.apache.thrift.TFieldRequirementType.DEFAULT,
new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , "Text")));
tmpMap.put(_Fields.VALUE, new org.apache.thrift.meta_data.FieldMetaData("value", org.apache.thrift.TFieldRequirementType.DEFAULT,
new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , "Text")));
tmpMap.put(_Fields.WRITE_TO_WAL, new org.apache.thrift.meta_data.FieldMetaData("writeToWAL", org.apache.thrift.TFieldRequirementType.DEFAULT,
new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL)));
metaDataMap = Collections.unmodifiableMap(tmpMap);
org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(Mutation.class, metaDataMap);
}
public Mutation() {
this.isDelete = false;
this.writeToWAL = true;
}
public Mutation(
boolean isDelete,
ByteBuffer column,
ByteBuffer value,
boolean writeToWAL)
{
this();
this.isDelete = isDelete;
setIsDeleteIsSet(true);
this.column = org.apache.thrift.TBaseHelper.copyBinary(column);
this.value = org.apache.thrift.TBaseHelper.copyBinary(value);
this.writeToWAL = writeToWAL;
setWriteToWALIsSet(true);
}
/**
* Performs a deep copy on <i>other</i>.
*/
public Mutation(Mutation other) {
__isset_bitfield = other.__isset_bitfield;
this.isDelete = other.isDelete;
if (other.isSetColumn()) {
this.column = other.column;
}
if (other.isSetValue()) {
this.value = other.value;
}
this.writeToWAL = other.writeToWAL;
}
public Mutation deepCopy() {
return new Mutation(this);
}
@Override
public void clear() {
this.isDelete = false;
this.column = null;
this.value = null;
this.writeToWAL = true;
}
public boolean isIsDelete() {
return this.isDelete;
}
public Mutation setIsDelete(boolean isDelete) {
this.isDelete = isDelete;
setIsDeleteIsSet(true);
return this;
}
public void unsetIsDelete() {
__isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __ISDELETE_ISSET_ID);
}
/** Returns true if field isDelete is set (has been assigned a value) and false otherwise */
public boolean isSetIsDelete() {
return EncodingUtils.testBit(__isset_bitfield, __ISDELETE_ISSET_ID);
}
public void setIsDeleteIsSet(boolean value) {
__isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __ISDELETE_ISSET_ID, value);
}
public byte[] getColumn() {
setColumn(org.apache.thrift.TBaseHelper.rightSize(column));
return column == null ? null : column.array();
}
public ByteBuffer bufferForColumn() {
return org.apache.thrift.TBaseHelper.copyBinary(column);
}
public Mutation setColumn(byte[] column) {
this.column = column == null ? (ByteBuffer)null : ByteBuffer.wrap(Arrays.copyOf(column, column.length));
return this;
}
public Mutation setColumn(ByteBuffer column) {
this.column = org.apache.thrift.TBaseHelper.copyBinary(column);
return this;
}
public void unsetColumn() {
this.column = null;
}
/** Returns true if field column is set (has been assigned a value) and false otherwise */
public boolean isSetColumn() {
return this.column != null;
}
public void setColumnIsSet(boolean value) {
if (!value) {
this.column = null;
}
}
public byte[] getValue() {
setValue(org.apache.thrift.TBaseHelper.rightSize(value));
return value == null ? null : value.array();
}
public ByteBuffer bufferForValue() {
return org.apache.thrift.TBaseHelper.copyBinary(value);
}
public Mutation setValue(byte[] value) {
this.value = value == null ? (ByteBuffer)null : ByteBuffer.wrap(Arrays.copyOf(value, value.length));
return this;
}
public Mutation setValue(ByteBuffer value) {
this.value = org.apache.thrift.TBaseHelper.copyBinary(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 boolean isWriteToWAL() {
return this.writeToWAL;
}
public Mutation setWriteToWAL(boolean writeToWAL) {
this.writeToWAL = writeToWAL;
setWriteToWALIsSet(true);
return this;
}
public void unsetWriteToWAL() {
__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 EncodingUtils.testBit(__isset_bitfield, __WRITETOWAL_ISSET_ID);
}
public void setWriteToWALIsSet(boolean value) {
__isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __WRITETOWAL_ISSET_ID, value);
}
public void setFieldValue(_Fields field, Object value) {
switch (field) {
case IS_DELETE:
if (value == null) {
unsetIsDelete();
} else {
setIsDelete((Boolean)value);
}
break;
case COLUMN:
if (value == null) {
unsetColumn();
} else {
setColumn((ByteBuffer)value);
}
break;
case VALUE:
if (value == null) {
unsetValue();
} else {
setValue((ByteBuffer)value);
}
break;
case WRITE_TO_WAL:
if (value == null) {
unsetWriteToWAL();
} else {
setWriteToWAL((Boolean)value);
}
break;
}
}
public Object getFieldValue(_Fields field) {
switch (field) {
case IS_DELETE:
return isIsDelete();
case COLUMN:
return getColumn();
case VALUE:
return getValue();
case WRITE_TO_WAL:
return isWriteToWAL();
}
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 IS_DELETE:
return isSetIsDelete();
case COLUMN:
return isSetColumn();
case VALUE:
return isSetValue();
case WRITE_TO_WAL:
return isSetWriteToWAL();
}
throw new IllegalStateException();
}
@Override
public boolean equals(Object that) {
if (that == null)
return false;
if (that instanceof Mutation)
return this.equals((Mutation)that);
return false;
}
public boolean equals(Mutation that) {
if (that == null)
return false;
boolean this_present_isDelete = true;
boolean that_present_isDelete = true;
if (this_present_isDelete || that_present_isDelete) {
if (!(this_present_isDelete && that_present_isDelete))
return false;
if (this.isDelete != that.isDelete)
return false;
}
boolean this_present_column = true && this.isSetColumn();
boolean that_present_column = true && that.isSetColumn();
if (this_present_column || that_present_column) {
if (!(this_present_column && that_present_column))
return false;
if (!this.column.equals(that.column))
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_writeToWAL = true;
boolean that_present_writeToWAL = true;
if (this_present_writeToWAL || that_present_writeToWAL) {
if (!(this_present_writeToWAL && that_present_writeToWAL))
return false;
if (this.writeToWAL != that.writeToWAL)
return false;
}
return true;
}
@Override
public int hashCode() {
List<Object> list = new ArrayList<Object>();
boolean present_isDelete = true;
list.add(present_isDelete);
if (present_isDelete)
list.add(isDelete);
boolean present_column = true && (isSetColumn());
list.add(present_column);
if (present_column)
list.add(column);
boolean present_value = true && (isSetValue());
list.add(present_value);
if (present_value)
list.add(value);
boolean present_writeToWAL = true;
list.add(present_writeToWAL);
if (present_writeToWAL)
list.add(writeToWAL);
return list.hashCode();
}
@Override
public int compareTo(Mutation other) {
if (!getClass().equals(other.getClass())) {
return getClass().getName().compareTo(other.getClass().getName());
}
int lastComparison = 0;
lastComparison = Boolean.valueOf(isSetIsDelete()).compareTo(other.isSetIsDelete());
if (lastComparison != 0) {
return lastComparison;
}
if (isSetIsDelete()) {
lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.isDelete, other.isDelete);
if (lastComparison != 0) {
return lastComparison;
}
}
lastComparison = Boolean.valueOf(isSetColumn()).compareTo(other.isSetColumn());
if (lastComparison != 0) {
return lastComparison;
}
if (isSetColumn()) {
lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.column, other.column);
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(isSetWriteToWAL()).compareTo(other.isSetWriteToWAL());
if (lastComparison != 0) {
return lastComparison;
}
if (isSetWriteToWAL()) {
lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.writeToWAL, other.writeToWAL);
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("Mutation(");
boolean first = true;
sb.append("isDelete:");
sb.append(this.isDelete);
first = false;
if (!first) sb.append(", ");
sb.append("column:");
if (this.column == null) {
sb.append("null");
} else {
org.apache.thrift.TBaseHelper.toString(this.column, 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 (!first) sb.append(", ");
sb.append("writeToWAL:");
sb.append(this.writeToWAL);
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 MutationStandardSchemeFactory implements SchemeFactory {
public MutationStandardScheme getScheme() {
return new MutationStandardScheme();
}
}
private static class MutationStandardScheme extends StandardScheme<Mutation> {
public void read(org.apache.thrift.protocol.TProtocol iprot, Mutation 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: // IS_DELETE
if (schemeField.type == org.apache.thrift.protocol.TType.BOOL) {
struct.isDelete = iprot.readBool();
struct.setIsDeleteIsSet(true);
} else {
org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
}
break;
case 2: // COLUMN
if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
struct.column = iprot.readBinary();
struct.setColumnIsSet(true);
} else {
org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
}
break;
case 3: // 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 4: // WRITE_TO_WAL
if (schemeField.type == org.apache.thrift.protocol.TType.BOOL) {
struct.writeToWAL = iprot.readBool();
struct.setWriteToWALIsSet(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, Mutation struct) throws org.apache.thrift.TException {
struct.validate();
oprot.writeStructBegin(STRUCT_DESC);
oprot.writeFieldBegin(IS_DELETE_FIELD_DESC);
oprot.writeBool(struct.isDelete);
oprot.writeFieldEnd();
if (struct.column != null) {
oprot.writeFieldBegin(COLUMN_FIELD_DESC);
oprot.writeBinary(struct.column);
oprot.writeFieldEnd();
}
if (struct.value != null) {
oprot.writeFieldBegin(VALUE_FIELD_DESC);
oprot.writeBinary(struct.value);
oprot.writeFieldEnd();
}
oprot.writeFieldBegin(WRITE_TO_WAL_FIELD_DESC);
oprot.writeBool(struct.writeToWAL);
oprot.writeFieldEnd();
oprot.writeFieldStop();
oprot.writeStructEnd();
}
}
private static class MutationTupleSchemeFactory implements SchemeFactory {
public MutationTupleScheme getScheme() {
return new MutationTupleScheme();
}
}
private static class MutationTupleScheme extends TupleScheme<Mutation> {
@Override
public void write(org.apache.thrift.protocol.TProtocol prot, Mutation struct) throws org.apache.thrift.TException {
TTupleProtocol oprot = (TTupleProtocol) prot;
BitSet optionals = new BitSet();
if (struct.isSetIsDelete()) {
optionals.set(0);
}
if (struct.isSetColumn()) {
optionals.set(1);
}
if (struct.isSetValue()) {
optionals.set(2);
}
if (struct.isSetWriteToWAL()) {
optionals.set(3);
}
oprot.writeBitSet(optionals, 4);
if (struct.isSetIsDelete()) {
oprot.writeBool(struct.isDelete);
}
if (struct.isSetColumn()) {
oprot.writeBinary(struct.column);
}
if (struct.isSetValue()) {
oprot.writeBinary(struct.value);
}
if (struct.isSetWriteToWAL()) {
oprot.writeBool(struct.writeToWAL);
}
}
@Override
public void read(org.apache.thrift.protocol.TProtocol prot, Mutation struct) throws org.apache.thrift.TException {
TTupleProtocol iprot = (TTupleProtocol) prot;
BitSet incoming = iprot.readBitSet(4);
if (incoming.get(0)) {
struct.isDelete = iprot.readBool();
struct.setIsDeleteIsSet(true);
}
if (incoming.get(1)) {
struct.column = iprot.readBinary();
struct.setColumnIsSet(true);
}
if (incoming.get(2)) {
struct.value = iprot.readBinary();
struct.setValueIsSet(true);
}
if (incoming.get(3)) {
struct.writeToWAL = iprot.readBool();
struct.setWriteToWALIsSet(true);
}
}
}
}

View File

@ -0,0 +1,840 @@
/**
* Autogenerated by Thrift Compiler (0.9.3)
*
* DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
* @generated
*/
package org.apache.hadoop.hbase.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 javax.annotation.Generated;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
/**
* An Append object is used to specify the parameters for performing the append operation.
*/
@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2016-05-25")
public class TAppend implements org.apache.thrift.TBase<TAppend, TAppend._Fields>, java.io.Serializable, Cloneable, Comparable<TAppend> {
private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TAppend");
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);
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)2);
private static final org.apache.thrift.protocol.TField COLUMNS_FIELD_DESC = new org.apache.thrift.protocol.TField("columns", org.apache.thrift.protocol.TType.LIST, (short)3);
private static final org.apache.thrift.protocol.TField VALUES_FIELD_DESC = new org.apache.thrift.protocol.TField("values", org.apache.thrift.protocol.TType.LIST, (short)4);
private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
static {
schemes.put(StandardScheme.class, new TAppendStandardSchemeFactory());
schemes.put(TupleScheme.class, new TAppendTupleSchemeFactory());
}
public ByteBuffer table; // required
public ByteBuffer row; // required
public List<ByteBuffer> columns; // required
public List<ByteBuffer> values; // 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((short)1, "table"),
ROW((short)2, "row"),
COLUMNS((short)3, "columns"),
VALUES((short)4, "values");
private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
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
return TABLE;
case 2: // ROW
return ROW;
case 3: // COLUMNS
return COLUMNS;
case 4: // VALUES
return VALUES;
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.TABLE, new org.apache.thrift.meta_data.FieldMetaData("table", org.apache.thrift.TFieldRequirementType.DEFAULT,
new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , "Text")));
tmpMap.put(_Fields.ROW, new org.apache.thrift.meta_data.FieldMetaData("row", org.apache.thrift.TFieldRequirementType.DEFAULT,
new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , "Text")));
tmpMap.put(_Fields.COLUMNS, new org.apache.thrift.meta_data.FieldMetaData("columns", org.apache.thrift.TFieldRequirementType.DEFAULT,
new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST,
new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , "Text"))));
tmpMap.put(_Fields.VALUES, new org.apache.thrift.meta_data.FieldMetaData("values", org.apache.thrift.TFieldRequirementType.DEFAULT,
new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST,
new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , "Text"))));
metaDataMap = Collections.unmodifiableMap(tmpMap);
org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TAppend.class, metaDataMap);
}
public TAppend() {
}
public TAppend(
ByteBuffer table,
ByteBuffer row,
List<ByteBuffer> columns,
List<ByteBuffer> values)
{
this();
this.table = org.apache.thrift.TBaseHelper.copyBinary(table);
this.row = org.apache.thrift.TBaseHelper.copyBinary(row);
this.columns = columns;
this.values = values;
}
/**
* Performs a deep copy on <i>other</i>.
*/
public TAppend(TAppend other) {
if (other.isSetTable()) {
this.table = other.table;
}
if (other.isSetRow()) {
this.row = other.row;
}
if (other.isSetColumns()) {
List<ByteBuffer> __this__columns = new ArrayList<ByteBuffer>(other.columns.size());
for (ByteBuffer other_element : other.columns) {
__this__columns.add(other_element);
}
this.columns = __this__columns;
}
if (other.isSetValues()) {
List<ByteBuffer> __this__values = new ArrayList<ByteBuffer>(other.values.size());
for (ByteBuffer other_element : other.values) {
__this__values.add(other_element);
}
this.values = __this__values;
}
}
public TAppend deepCopy() {
return new TAppend(this);
}
@Override
public void clear() {
this.table = null;
this.row = null;
this.columns = null;
this.values = null;
}
public byte[] getTable() {
setTable(org.apache.thrift.TBaseHelper.rightSize(table));
return table == null ? null : table.array();
}
public ByteBuffer bufferForTable() {
return org.apache.thrift.TBaseHelper.copyBinary(table);
}
public TAppend setTable(byte[] table) {
this.table = table == null ? (ByteBuffer)null : ByteBuffer.wrap(Arrays.copyOf(table, table.length));
return this;
}
public TAppend setTable(ByteBuffer table) {
this.table = org.apache.thrift.TBaseHelper.copyBinary(table);
return this;
}
public void unsetTable() {
this.table = null;
}
/** Returns true if field table is set (has been assigned a value) and false otherwise */
public boolean isSetTable() {
return this.table != null;
}
public void setTableIsSet(boolean value) {
if (!value) {
this.table = null;
}
}
public byte[] getRow() {
setRow(org.apache.thrift.TBaseHelper.rightSize(row));
return row == null ? null : row.array();
}
public ByteBuffer bufferForRow() {
return org.apache.thrift.TBaseHelper.copyBinary(row);
}
public TAppend setRow(byte[] row) {
this.row = row == null ? (ByteBuffer)null : ByteBuffer.wrap(Arrays.copyOf(row, row.length));
return this;
}
public TAppend setRow(ByteBuffer row) {
this.row = org.apache.thrift.TBaseHelper.copyBinary(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 int getColumnsSize() {
return (this.columns == null) ? 0 : this.columns.size();
}
public java.util.Iterator<ByteBuffer> getColumnsIterator() {
return (this.columns == null) ? null : this.columns.iterator();
}
public void addToColumns(ByteBuffer elem) {
if (this.columns == null) {
this.columns = new ArrayList<ByteBuffer>();
}
this.columns.add(elem);
}
public List<ByteBuffer> getColumns() {
return this.columns;
}
public TAppend setColumns(List<ByteBuffer> columns) {
this.columns = columns;
return this;
}
public void unsetColumns() {
this.columns = null;
}
/** Returns true if field columns is set (has been assigned a value) and false otherwise */
public boolean isSetColumns() {
return this.columns != null;
}
public void setColumnsIsSet(boolean value) {
if (!value) {
this.columns = null;
}
}
public int getValuesSize() {
return (this.values == null) ? 0 : this.values.size();
}
public java.util.Iterator<ByteBuffer> getValuesIterator() {
return (this.values == null) ? null : this.values.iterator();
}
public void addToValues(ByteBuffer elem) {
if (this.values == null) {
this.values = new ArrayList<ByteBuffer>();
}
this.values.add(elem);
}
public List<ByteBuffer> getValues() {
return this.values;
}
public TAppend setValues(List<ByteBuffer> values) {
this.values = values;
return this;
}
public void unsetValues() {
this.values = null;
}
/** Returns true if field values is set (has been assigned a value) and false otherwise */
public boolean isSetValues() {
return this.values != null;
}
public void setValuesIsSet(boolean value) {
if (!value) {
this.values = null;
}
}
public void setFieldValue(_Fields field, Object value) {
switch (field) {
case TABLE:
if (value == null) {
unsetTable();
} else {
setTable((ByteBuffer)value);
}
break;
case ROW:
if (value == null) {
unsetRow();
} else {
setRow((ByteBuffer)value);
}
break;
case COLUMNS:
if (value == null) {
unsetColumns();
} else {
setColumns((List<ByteBuffer>)value);
}
break;
case VALUES:
if (value == null) {
unsetValues();
} else {
setValues((List<ByteBuffer>)value);
}
break;
}
}
public Object getFieldValue(_Fields field) {
switch (field) {
case TABLE:
return getTable();
case ROW:
return getRow();
case COLUMNS:
return getColumns();
case VALUES:
return getValues();
}
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:
return isSetTable();
case ROW:
return isSetRow();
case COLUMNS:
return isSetColumns();
case VALUES:
return isSetValues();
}
throw new IllegalStateException();
}
@Override
public boolean equals(Object that) {
if (that == null)
return false;
if (that instanceof TAppend)
return this.equals((TAppend)that);
return false;
}
public boolean equals(TAppend that) {
if (that == null)
return false;
boolean this_present_table = true && this.isSetTable();
boolean that_present_table = true && that.isSetTable();
if (this_present_table || that_present_table) {
if (!(this_present_table && that_present_table))
return false;
if (!this.table.equals(that.table))
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_columns = true && this.isSetColumns();
boolean that_present_columns = true && that.isSetColumns();
if (this_present_columns || that_present_columns) {
if (!(this_present_columns && that_present_columns))
return false;
if (!this.columns.equals(that.columns))
return false;
}
boolean this_present_values = true && this.isSetValues();
boolean that_present_values = true && that.isSetValues();
if (this_present_values || that_present_values) {
if (!(this_present_values && that_present_values))
return false;
if (!this.values.equals(that.values))
return false;
}
return true;
}
@Override
public int hashCode() {
List<Object> list = new ArrayList<Object>();
boolean present_table = true && (isSetTable());
list.add(present_table);
if (present_table)
list.add(table);
boolean present_row = true && (isSetRow());
list.add(present_row);
if (present_row)
list.add(row);
boolean present_columns = true && (isSetColumns());
list.add(present_columns);
if (present_columns)
list.add(columns);
boolean present_values = true && (isSetValues());
list.add(present_values);
if (present_values)
list.add(values);
return list.hashCode();
}
@Override
public int compareTo(TAppend other) {
if (!getClass().equals(other.getClass())) {
return getClass().getName().compareTo(other.getClass().getName());
}
int lastComparison = 0;
lastComparison = Boolean.valueOf(isSetTable()).compareTo(other.isSetTable());
if (lastComparison != 0) {
return lastComparison;
}
if (isSetTable()) {
lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.table, other.table);
if (lastComparison != 0) {
return lastComparison;
}
}
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(isSetColumns()).compareTo(other.isSetColumns());
if (lastComparison != 0) {
return lastComparison;
}
if (isSetColumns()) {
lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.columns, other.columns);
if (lastComparison != 0) {
return lastComparison;
}
}
lastComparison = Boolean.valueOf(isSetValues()).compareTo(other.isSetValues());
if (lastComparison != 0) {
return lastComparison;
}
if (isSetValues()) {
lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.values, other.values);
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("TAppend(");
boolean first = true;
sb.append("table:");
if (this.table == null) {
sb.append("null");
} else {
org.apache.thrift.TBaseHelper.toString(this.table, sb);
}
first = false;
if (!first) sb.append(", ");
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("columns:");
if (this.columns == null) {
sb.append("null");
} else {
org.apache.thrift.TBaseHelper.toString(this.columns, sb);
}
first = false;
if (!first) sb.append(", ");
sb.append("values:");
if (this.values == null) {
sb.append("null");
} else {
org.apache.thrift.TBaseHelper.toString(this.values, sb);
}
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 TAppendStandardSchemeFactory implements SchemeFactory {
public TAppendStandardScheme getScheme() {
return new TAppendStandardScheme();
}
}
private static class TAppendStandardScheme extends StandardScheme<TAppend> {
public void read(org.apache.thrift.protocol.TProtocol iprot, TAppend 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
if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
struct.table = iprot.readBinary();
struct.setTableIsSet(true);
} else {
org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
}
break;
case 2: // 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 3: // COLUMNS
if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
{
org.apache.thrift.protocol.TList _list34 = iprot.readListBegin();
struct.columns = new ArrayList<ByteBuffer>(_list34.size);
ByteBuffer _elem35;
for (int _i36 = 0; _i36 < _list34.size; ++_i36)
{
_elem35 = iprot.readBinary();
struct.columns.add(_elem35);
}
iprot.readListEnd();
}
struct.setColumnsIsSet(true);
} else {
org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
}
break;
case 4: // VALUES
if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
{
org.apache.thrift.protocol.TList _list37 = iprot.readListBegin();
struct.values = new ArrayList<ByteBuffer>(_list37.size);
ByteBuffer _elem38;
for (int _i39 = 0; _i39 < _list37.size; ++_i39)
{
_elem38 = iprot.readBinary();
struct.values.add(_elem38);
}
iprot.readListEnd();
}
struct.setValuesIsSet(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, TAppend struct) throws org.apache.thrift.TException {
struct.validate();
oprot.writeStructBegin(STRUCT_DESC);
if (struct.table != null) {
oprot.writeFieldBegin(TABLE_FIELD_DESC);
oprot.writeBinary(struct.table);
oprot.writeFieldEnd();
}
if (struct.row != null) {
oprot.writeFieldBegin(ROW_FIELD_DESC);
oprot.writeBinary(struct.row);
oprot.writeFieldEnd();
}
if (struct.columns != null) {
oprot.writeFieldBegin(COLUMNS_FIELD_DESC);
{
oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.columns.size()));
for (ByteBuffer _iter40 : struct.columns)
{
oprot.writeBinary(_iter40);
}
oprot.writeListEnd();
}
oprot.writeFieldEnd();
}
if (struct.values != null) {
oprot.writeFieldBegin(VALUES_FIELD_DESC);
{
oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.values.size()));
for (ByteBuffer _iter41 : struct.values)
{
oprot.writeBinary(_iter41);
}
oprot.writeListEnd();
}
oprot.writeFieldEnd();
}
oprot.writeFieldStop();
oprot.writeStructEnd();
}
}
private static class TAppendTupleSchemeFactory implements SchemeFactory {
public TAppendTupleScheme getScheme() {
return new TAppendTupleScheme();
}
}
private static class TAppendTupleScheme extends TupleScheme<TAppend> {
@Override
public void write(org.apache.thrift.protocol.TProtocol prot, TAppend struct) throws org.apache.thrift.TException {
TTupleProtocol oprot = (TTupleProtocol) prot;
BitSet optionals = new BitSet();
if (struct.isSetTable()) {
optionals.set(0);
}
if (struct.isSetRow()) {
optionals.set(1);
}
if (struct.isSetColumns()) {
optionals.set(2);
}
if (struct.isSetValues()) {
optionals.set(3);
}
oprot.writeBitSet(optionals, 4);
if (struct.isSetTable()) {
oprot.writeBinary(struct.table);
}
if (struct.isSetRow()) {
oprot.writeBinary(struct.row);
}
if (struct.isSetColumns()) {
{
oprot.writeI32(struct.columns.size());
for (ByteBuffer _iter42 : struct.columns)
{
oprot.writeBinary(_iter42);
}
}
}
if (struct.isSetValues()) {
{
oprot.writeI32(struct.values.size());
for (ByteBuffer _iter43 : struct.values)
{
oprot.writeBinary(_iter43);
}
}
}
}
@Override
public void read(org.apache.thrift.protocol.TProtocol prot, TAppend struct) throws org.apache.thrift.TException {
TTupleProtocol iprot = (TTupleProtocol) prot;
BitSet incoming = iprot.readBitSet(4);
if (incoming.get(0)) {
struct.table = iprot.readBinary();
struct.setTableIsSet(true);
}
if (incoming.get(1)) {
struct.row = iprot.readBinary();
struct.setRowIsSet(true);
}
if (incoming.get(2)) {
{
org.apache.thrift.protocol.TList _list44 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
struct.columns = new ArrayList<ByteBuffer>(_list44.size);
ByteBuffer _elem45;
for (int _i46 = 0; _i46 < _list44.size; ++_i46)
{
_elem45 = iprot.readBinary();
struct.columns.add(_elem45);
}
}
struct.setColumnsIsSet(true);
}
if (incoming.get(3)) {
{
org.apache.thrift.protocol.TList _list47 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
struct.values = new ArrayList<ByteBuffer>(_list47.size);
ByteBuffer _elem48;
for (int _i49 = 0; _i49 < _list47.size; ++_i49)
{
_elem48 = iprot.readBinary();
struct.values.add(_elem48);
}
}
struct.setValuesIsSet(true);
}
}
}
}

View File

@ -0,0 +1,517 @@
/**
* Autogenerated by Thrift Compiler (0.9.3)
*
* DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
* @generated
*/
package org.apache.hadoop.hbase.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 javax.annotation.Generated;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
/**
* TCell - Used to transport a cell value (byte[]) and the timestamp it was
* stored with together as a result for get and getRow methods. This promotes
* the timestamp of a cell to a first-class value, making it easy to take
* note of temporal data. Cell is used all the way from HStore up to HTable.
*/
@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2016-05-25")
public class TCell implements org.apache.thrift.TBase<TCell, TCell._Fields>, java.io.Serializable, Cloneable, Comparable<TCell> {
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);
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)2);
private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
static {
schemes.put(StandardScheme.class, new TCellStandardSchemeFactory());
schemes.put(TupleScheme.class, new TCellTupleSchemeFactory());
}
public ByteBuffer value; // required
public long timestamp; // 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 {
VALUE((short)1, "value"),
TIMESTAMP((short)2, "timestamp");
private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
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: // VALUE
return VALUE;
case 2: // TIMESTAMP
return TIMESTAMP;
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 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.VALUE, new org.apache.thrift.meta_data.FieldMetaData("value", org.apache.thrift.TFieldRequirementType.DEFAULT,
new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , "Bytes")));
tmpMap.put(_Fields.TIMESTAMP, new org.apache.thrift.meta_data.FieldMetaData("timestamp", 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(TCell.class, metaDataMap);
}
public TCell() {
}
public TCell(
ByteBuffer value,
long timestamp)
{
this();
this.value = org.apache.thrift.TBaseHelper.copyBinary(value);
this.timestamp = timestamp;
setTimestampIsSet(true);
}
/**
* Performs a deep copy on <i>other</i>.
*/
public TCell(TCell other) {
__isset_bitfield = other.__isset_bitfield;
if (other.isSetValue()) {
this.value = other.value;
}
this.timestamp = other.timestamp;
}
public TCell deepCopy() {
return new TCell(this);
}
@Override
public void clear() {
this.value = null;
setTimestampIsSet(false);
this.timestamp = 0;
}
public byte[] getValue() {
setValue(org.apache.thrift.TBaseHelper.rightSize(value));
return value == null ? null : value.array();
}
public ByteBuffer bufferForValue() {
return org.apache.thrift.TBaseHelper.copyBinary(value);
}
public TCell setValue(byte[] value) {
this.value = value == null ? (ByteBuffer)null : ByteBuffer.wrap(Arrays.copyOf(value, value.length));
return this;
}
public TCell setValue(ByteBuffer value) {
this.value = org.apache.thrift.TBaseHelper.copyBinary(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 TCell 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 void setFieldValue(_Fields field, Object value) {
switch (field) {
case VALUE:
if (value == null) {
unsetValue();
} else {
setValue((ByteBuffer)value);
}
break;
case TIMESTAMP:
if (value == null) {
unsetTimestamp();
} else {
setTimestamp((Long)value);
}
break;
}
}
public Object getFieldValue(_Fields field) {
switch (field) {
case VALUE:
return getValue();
case TIMESTAMP:
return getTimestamp();
}
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 VALUE:
return isSetValue();
case TIMESTAMP:
return isSetTimestamp();
}
throw new IllegalStateException();
}
@Override
public boolean equals(Object that) {
if (that == null)
return false;
if (that instanceof TCell)
return this.equals((TCell)that);
return false;
}
public boolean equals(TCell that) {
if (that == null)
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;
boolean that_present_timestamp = true;
if (this_present_timestamp || that_present_timestamp) {
if (!(this_present_timestamp && that_present_timestamp))
return false;
if (this.timestamp != that.timestamp)
return false;
}
return true;
}
@Override
public int hashCode() {
List<Object> list = new ArrayList<Object>();
boolean present_value = true && (isSetValue());
list.add(present_value);
if (present_value)
list.add(value);
boolean present_timestamp = true;
list.add(present_timestamp);
if (present_timestamp)
list.add(timestamp);
return list.hashCode();
}
@Override
public int compareTo(TCell other) {
if (!getClass().equals(other.getClass())) {
return getClass().getName().compareTo(other.getClass().getName());
}
int lastComparison = 0;
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;
}
}
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("TCell(");
boolean first = true;
sb.append("value:");
if (this.value == null) {
sb.append("null");
} else {
org.apache.thrift.TBaseHelper.toString(this.value, sb);
}
first = false;
if (!first) sb.append(", ");
sb.append("timestamp:");
sb.append(this.timestamp);
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 TCellStandardSchemeFactory implements SchemeFactory {
public TCellStandardScheme getScheme() {
return new TCellStandardScheme();
}
}
private static class TCellStandardScheme extends StandardScheme<TCell> {
public void read(org.apache.thrift.protocol.TProtocol iprot, TCell 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: // 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 2: // 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;
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, TCell struct) throws org.apache.thrift.TException {
struct.validate();
oprot.writeStructBegin(STRUCT_DESC);
if (struct.value != null) {
oprot.writeFieldBegin(VALUE_FIELD_DESC);
oprot.writeBinary(struct.value);
oprot.writeFieldEnd();
}
oprot.writeFieldBegin(TIMESTAMP_FIELD_DESC);
oprot.writeI64(struct.timestamp);
oprot.writeFieldEnd();
oprot.writeFieldStop();
oprot.writeStructEnd();
}
}
private static class TCellTupleSchemeFactory implements SchemeFactory {
public TCellTupleScheme getScheme() {
return new TCellTupleScheme();
}
}
private static class TCellTupleScheme extends TupleScheme<TCell> {
@Override
public void write(org.apache.thrift.protocol.TProtocol prot, TCell struct) throws org.apache.thrift.TException {
TTupleProtocol oprot = (TTupleProtocol) prot;
BitSet optionals = new BitSet();
if (struct.isSetValue()) {
optionals.set(0);
}
if (struct.isSetTimestamp()) {
optionals.set(1);
}
oprot.writeBitSet(optionals, 2);
if (struct.isSetValue()) {
oprot.writeBinary(struct.value);
}
if (struct.isSetTimestamp()) {
oprot.writeI64(struct.timestamp);
}
}
@Override
public void read(org.apache.thrift.protocol.TProtocol prot, TCell struct) throws org.apache.thrift.TException {
TTupleProtocol iprot = (TTupleProtocol) prot;
BitSet incoming = iprot.readBitSet(2);
if (incoming.get(0)) {
struct.value = iprot.readBinary();
struct.setValueIsSet(true);
}
if (incoming.get(1)) {
struct.timestamp = iprot.readI64();
struct.setTimestampIsSet(true);
}
}
}
}

View File

@ -0,0 +1,521 @@
/**
* Autogenerated by Thrift Compiler (0.9.3)
*
* DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
* @generated
*/
package org.apache.hadoop.hbase.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 javax.annotation.Generated;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
/**
* Holds column name and the cell.
*/
@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2016-05-25")
public class TColumn implements org.apache.thrift.TBase<TColumn, TColumn._Fields>, java.io.Serializable, Cloneable, Comparable<TColumn> {
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);
private static final org.apache.thrift.protocol.TField CELL_FIELD_DESC = new org.apache.thrift.protocol.TField("cell", org.apache.thrift.protocol.TType.STRUCT, (short)2);
private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
static {
schemes.put(StandardScheme.class, new TColumnStandardSchemeFactory());
schemes.put(TupleScheme.class, new TColumnTupleSchemeFactory());
}
public ByteBuffer columnName; // required
public TCell cell; // 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 {
COLUMN_NAME((short)1, "columnName"),
CELL((short)2, "cell");
private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
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: // COLUMN_NAME
return COLUMN_NAME;
case 2: // CELL
return CELL;
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.COLUMN_NAME, new org.apache.thrift.meta_data.FieldMetaData("columnName", org.apache.thrift.TFieldRequirementType.DEFAULT,
new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , "Text")));
tmpMap.put(_Fields.CELL, new org.apache.thrift.meta_data.FieldMetaData("cell", org.apache.thrift.TFieldRequirementType.DEFAULT,
new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TCell.class)));
metaDataMap = Collections.unmodifiableMap(tmpMap);
org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TColumn.class, metaDataMap);
}
public TColumn() {
}
public TColumn(
ByteBuffer columnName,
TCell cell)
{
this();
this.columnName = org.apache.thrift.TBaseHelper.copyBinary(columnName);
this.cell = cell;
}
/**
* Performs a deep copy on <i>other</i>.
*/
public TColumn(TColumn other) {
if (other.isSetColumnName()) {
this.columnName = other.columnName;
}
if (other.isSetCell()) {
this.cell = new TCell(other.cell);
}
}
public TColumn deepCopy() {
return new TColumn(this);
}
@Override
public void clear() {
this.columnName = null;
this.cell = null;
}
public byte[] getColumnName() {
setColumnName(org.apache.thrift.TBaseHelper.rightSize(columnName));
return columnName == null ? null : columnName.array();
}
public ByteBuffer bufferForColumnName() {
return org.apache.thrift.TBaseHelper.copyBinary(columnName);
}
public TColumn setColumnName(byte[] columnName) {
this.columnName = columnName == null ? (ByteBuffer)null : ByteBuffer.wrap(Arrays.copyOf(columnName, columnName.length));
return this;
}
public TColumn setColumnName(ByteBuffer columnName) {
this.columnName = org.apache.thrift.TBaseHelper.copyBinary(columnName);
return this;
}
public void unsetColumnName() {
this.columnName = null;
}
/** Returns true if field columnName is set (has been assigned a value) and false otherwise */
public boolean isSetColumnName() {
return this.columnName != null;
}
public void setColumnNameIsSet(boolean value) {
if (!value) {
this.columnName = null;
}
}
public TCell getCell() {
return this.cell;
}
public TColumn setCell(TCell cell) {
this.cell = cell;
return this;
}
public void unsetCell() {
this.cell = null;
}
/** Returns true if field cell is set (has been assigned a value) and false otherwise */
public boolean isSetCell() {
return this.cell != null;
}
public void setCellIsSet(boolean value) {
if (!value) {
this.cell = null;
}
}
public void setFieldValue(_Fields field, Object value) {
switch (field) {
case COLUMN_NAME:
if (value == null) {
unsetColumnName();
} else {
setColumnName((ByteBuffer)value);
}
break;
case CELL:
if (value == null) {
unsetCell();
} else {
setCell((TCell)value);
}
break;
}
}
public Object getFieldValue(_Fields field) {
switch (field) {
case COLUMN_NAME:
return getColumnName();
case CELL:
return getCell();
}
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 COLUMN_NAME:
return isSetColumnName();
case CELL:
return isSetCell();
}
throw new IllegalStateException();
}
@Override
public boolean equals(Object that) {
if (that == null)
return false;
if (that instanceof TColumn)
return this.equals((TColumn)that);
return false;
}
public boolean equals(TColumn that) {
if (that == null)
return false;
boolean this_present_columnName = true && this.isSetColumnName();
boolean that_present_columnName = true && that.isSetColumnName();
if (this_present_columnName || that_present_columnName) {
if (!(this_present_columnName && that_present_columnName))
return false;
if (!this.columnName.equals(that.columnName))
return false;
}
boolean this_present_cell = true && this.isSetCell();
boolean that_present_cell = true && that.isSetCell();
if (this_present_cell || that_present_cell) {
if (!(this_present_cell && that_present_cell))
return false;
if (!this.cell.equals(that.cell))
return false;
}
return true;
}
@Override
public int hashCode() {
List<Object> list = new ArrayList<Object>();
boolean present_columnName = true && (isSetColumnName());
list.add(present_columnName);
if (present_columnName)
list.add(columnName);
boolean present_cell = true && (isSetCell());
list.add(present_cell);
if (present_cell)
list.add(cell);
return list.hashCode();
}
@Override
public int compareTo(TColumn other) {
if (!getClass().equals(other.getClass())) {
return getClass().getName().compareTo(other.getClass().getName());
}
int lastComparison = 0;
lastComparison = Boolean.valueOf(isSetColumnName()).compareTo(other.isSetColumnName());
if (lastComparison != 0) {
return lastComparison;
}
if (isSetColumnName()) {
lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.columnName, other.columnName);
if (lastComparison != 0) {
return lastComparison;
}
}
lastComparison = Boolean.valueOf(isSetCell()).compareTo(other.isSetCell());
if (lastComparison != 0) {
return lastComparison;
}
if (isSetCell()) {
lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.cell, other.cell);
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("TColumn(");
boolean first = true;
sb.append("columnName:");
if (this.columnName == null) {
sb.append("null");
} else {
org.apache.thrift.TBaseHelper.toString(this.columnName, sb);
}
first = false;
if (!first) sb.append(", ");
sb.append("cell:");
if (this.cell == null) {
sb.append("null");
} else {
sb.append(this.cell);
}
first = false;
sb.append(")");
return sb.toString();
}
public void validate() throws org.apache.thrift.TException {
// check for required fields
// check for sub-struct validity
if (cell != null) {
cell.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 TColumnStandardSchemeFactory implements SchemeFactory {
public TColumnStandardScheme getScheme() {
return new TColumnStandardScheme();
}
}
private static class TColumnStandardScheme extends StandardScheme<TColumn> {
public void read(org.apache.thrift.protocol.TProtocol iprot, TColumn 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: // COLUMN_NAME
if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
struct.columnName = iprot.readBinary();
struct.setColumnNameIsSet(true);
} else {
org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
}
break;
case 2: // CELL
if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
struct.cell = new TCell();
struct.cell.read(iprot);
struct.setCellIsSet(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, TColumn struct) throws org.apache.thrift.TException {
struct.validate();
oprot.writeStructBegin(STRUCT_DESC);
if (struct.columnName != null) {
oprot.writeFieldBegin(COLUMN_NAME_FIELD_DESC);
oprot.writeBinary(struct.columnName);
oprot.writeFieldEnd();
}
if (struct.cell != null) {
oprot.writeFieldBegin(CELL_FIELD_DESC);
struct.cell.write(oprot);
oprot.writeFieldEnd();
}
oprot.writeFieldStop();
oprot.writeStructEnd();
}
}
private static class TColumnTupleSchemeFactory implements SchemeFactory {
public TColumnTupleScheme getScheme() {
return new TColumnTupleScheme();
}
}
private static class TColumnTupleScheme extends TupleScheme<TColumn> {
@Override
public void write(org.apache.thrift.protocol.TProtocol prot, TColumn struct) throws org.apache.thrift.TException {
TTupleProtocol oprot = (TTupleProtocol) prot;
BitSet optionals = new BitSet();
if (struct.isSetColumnName()) {
optionals.set(0);
}
if (struct.isSetCell()) {
optionals.set(1);
}
oprot.writeBitSet(optionals, 2);
if (struct.isSetColumnName()) {
oprot.writeBinary(struct.columnName);
}
if (struct.isSetCell()) {
struct.cell.write(oprot);
}
}
@Override
public void read(org.apache.thrift.protocol.TProtocol prot, TColumn struct) throws org.apache.thrift.TException {
TTupleProtocol iprot = (TTupleProtocol) prot;
BitSet incoming = iprot.readBitSet(2);
if (incoming.get(0)) {
struct.columnName = iprot.readBinary();
struct.setColumnNameIsSet(true);
}
if (incoming.get(1)) {
struct.cell = new TCell();
struct.cell.read(iprot);
struct.setCellIsSet(true);
}
}
}
}

View File

@ -0,0 +1,745 @@
/**
* Autogenerated by Thrift Compiler (0.9.3)
*
* DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
* @generated
*/
package org.apache.hadoop.hbase.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 javax.annotation.Generated;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
/**
* For increments that are not incrementColumnValue
* equivalents.
*/
@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2016-05-25")
public class TIncrement implements org.apache.thrift.TBase<TIncrement, TIncrement._Fields>, java.io.Serializable, Cloneable, Comparable<TIncrement> {
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);
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)2);
private static final org.apache.thrift.protocol.TField COLUMN_FIELD_DESC = new org.apache.thrift.protocol.TField("column", org.apache.thrift.protocol.TType.STRING, (short)3);
private static final org.apache.thrift.protocol.TField AMMOUNT_FIELD_DESC = new org.apache.thrift.protocol.TField("ammount", org.apache.thrift.protocol.TType.I64, (short)4);
private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
static {
schemes.put(StandardScheme.class, new TIncrementStandardSchemeFactory());
schemes.put(TupleScheme.class, new TIncrementTupleSchemeFactory());
}
public ByteBuffer table; // required
public ByteBuffer row; // required
public ByteBuffer column; // required
public long ammount; // 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((short)1, "table"),
ROW((short)2, "row"),
COLUMN((short)3, "column"),
AMMOUNT((short)4, "ammount");
private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
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
return TABLE;
case 2: // ROW
return ROW;
case 3: // COLUMN
return COLUMN;
case 4: // AMMOUNT
return AMMOUNT;
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 __AMMOUNT_ISSET_ID = 0;
private byte __isset_bitfield = 0;
public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
static {
Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
tmpMap.put(_Fields.TABLE, new org.apache.thrift.meta_data.FieldMetaData("table", org.apache.thrift.TFieldRequirementType.DEFAULT,
new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , "Text")));
tmpMap.put(_Fields.ROW, new org.apache.thrift.meta_data.FieldMetaData("row", org.apache.thrift.TFieldRequirementType.DEFAULT,
new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , "Text")));
tmpMap.put(_Fields.COLUMN, new org.apache.thrift.meta_data.FieldMetaData("column", org.apache.thrift.TFieldRequirementType.DEFAULT,
new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , "Text")));
tmpMap.put(_Fields.AMMOUNT, new org.apache.thrift.meta_data.FieldMetaData("ammount", 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(TIncrement.class, metaDataMap);
}
public TIncrement() {
}
public TIncrement(
ByteBuffer table,
ByteBuffer row,
ByteBuffer column,
long ammount)
{
this();
this.table = org.apache.thrift.TBaseHelper.copyBinary(table);
this.row = org.apache.thrift.TBaseHelper.copyBinary(row);
this.column = org.apache.thrift.TBaseHelper.copyBinary(column);
this.ammount = ammount;
setAmmountIsSet(true);
}
/**
* Performs a deep copy on <i>other</i>.
*/
public TIncrement(TIncrement other) {
__isset_bitfield = other.__isset_bitfield;
if (other.isSetTable()) {
this.table = other.table;
}
if (other.isSetRow()) {
this.row = other.row;
}
if (other.isSetColumn()) {
this.column = other.column;
}
this.ammount = other.ammount;
}
public TIncrement deepCopy() {
return new TIncrement(this);
}
@Override
public void clear() {
this.table = null;
this.row = null;
this.column = null;
setAmmountIsSet(false);
this.ammount = 0;
}
public byte[] getTable() {
setTable(org.apache.thrift.TBaseHelper.rightSize(table));
return table == null ? null : table.array();
}
public ByteBuffer bufferForTable() {
return org.apache.thrift.TBaseHelper.copyBinary(table);
}
public TIncrement setTable(byte[] table) {
this.table = table == null ? (ByteBuffer)null : ByteBuffer.wrap(Arrays.copyOf(table, table.length));
return this;
}
public TIncrement setTable(ByteBuffer table) {
this.table = org.apache.thrift.TBaseHelper.copyBinary(table);
return this;
}
public void unsetTable() {
this.table = null;
}
/** Returns true if field table is set (has been assigned a value) and false otherwise */
public boolean isSetTable() {
return this.table != null;
}
public void setTableIsSet(boolean value) {
if (!value) {
this.table = null;
}
}
public byte[] getRow() {
setRow(org.apache.thrift.TBaseHelper.rightSize(row));
return row == null ? null : row.array();
}
public ByteBuffer bufferForRow() {
return org.apache.thrift.TBaseHelper.copyBinary(row);
}
public TIncrement setRow(byte[] row) {
this.row = row == null ? (ByteBuffer)null : ByteBuffer.wrap(Arrays.copyOf(row, row.length));
return this;
}
public TIncrement setRow(ByteBuffer row) {
this.row = org.apache.thrift.TBaseHelper.copyBinary(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[] getColumn() {
setColumn(org.apache.thrift.TBaseHelper.rightSize(column));
return column == null ? null : column.array();
}
public ByteBuffer bufferForColumn() {
return org.apache.thrift.TBaseHelper.copyBinary(column);
}
public TIncrement setColumn(byte[] column) {
this.column = column == null ? (ByteBuffer)null : ByteBuffer.wrap(Arrays.copyOf(column, column.length));
return this;
}
public TIncrement setColumn(ByteBuffer column) {
this.column = org.apache.thrift.TBaseHelper.copyBinary(column);
return this;
}
public void unsetColumn() {
this.column = null;
}
/** Returns true if field column is set (has been assigned a value) and false otherwise */
public boolean isSetColumn() {
return this.column != null;
}
public void setColumnIsSet(boolean value) {
if (!value) {
this.column = null;
}
}
public long getAmmount() {
return this.ammount;
}
public TIncrement setAmmount(long ammount) {
this.ammount = ammount;
setAmmountIsSet(true);
return this;
}
public void unsetAmmount() {
__isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __AMMOUNT_ISSET_ID);
}
/** Returns true if field ammount is set (has been assigned a value) and false otherwise */
public boolean isSetAmmount() {
return EncodingUtils.testBit(__isset_bitfield, __AMMOUNT_ISSET_ID);
}
public void setAmmountIsSet(boolean value) {
__isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __AMMOUNT_ISSET_ID, value);
}
public void setFieldValue(_Fields field, Object value) {
switch (field) {
case TABLE:
if (value == null) {
unsetTable();
} else {
setTable((ByteBuffer)value);
}
break;
case ROW:
if (value == null) {
unsetRow();
} else {
setRow((ByteBuffer)value);
}
break;
case COLUMN:
if (value == null) {
unsetColumn();
} else {
setColumn((ByteBuffer)value);
}
break;
case AMMOUNT:
if (value == null) {
unsetAmmount();
} else {
setAmmount((Long)value);
}
break;
}
}
public Object getFieldValue(_Fields field) {
switch (field) {
case TABLE:
return getTable();
case ROW:
return getRow();
case COLUMN:
return getColumn();
case AMMOUNT:
return getAmmount();
}
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:
return isSetTable();
case ROW:
return isSetRow();
case COLUMN:
return isSetColumn();
case AMMOUNT:
return isSetAmmount();
}
throw new IllegalStateException();
}
@Override
public boolean equals(Object that) {
if (that == null)
return false;
if (that instanceof TIncrement)
return this.equals((TIncrement)that);
return false;
}
public boolean equals(TIncrement that) {
if (that == null)
return false;
boolean this_present_table = true && this.isSetTable();
boolean that_present_table = true && that.isSetTable();
if (this_present_table || that_present_table) {
if (!(this_present_table && that_present_table))
return false;
if (!this.table.equals(that.table))
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_column = true && this.isSetColumn();
boolean that_present_column = true && that.isSetColumn();
if (this_present_column || that_present_column) {
if (!(this_present_column && that_present_column))
return false;
if (!this.column.equals(that.column))
return false;
}
boolean this_present_ammount = true;
boolean that_present_ammount = true;
if (this_present_ammount || that_present_ammount) {
if (!(this_present_ammount && that_present_ammount))
return false;
if (this.ammount != that.ammount)
return false;
}
return true;
}
@Override
public int hashCode() {
List<Object> list = new ArrayList<Object>();
boolean present_table = true && (isSetTable());
list.add(present_table);
if (present_table)
list.add(table);
boolean present_row = true && (isSetRow());
list.add(present_row);
if (present_row)
list.add(row);
boolean present_column = true && (isSetColumn());
list.add(present_column);
if (present_column)
list.add(column);
boolean present_ammount = true;
list.add(present_ammount);
if (present_ammount)
list.add(ammount);
return list.hashCode();
}
@Override
public int compareTo(TIncrement other) {
if (!getClass().equals(other.getClass())) {
return getClass().getName().compareTo(other.getClass().getName());
}
int lastComparison = 0;
lastComparison = Boolean.valueOf(isSetTable()).compareTo(other.isSetTable());
if (lastComparison != 0) {
return lastComparison;
}
if (isSetTable()) {
lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.table, other.table);
if (lastComparison != 0) {
return lastComparison;
}
}
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(isSetColumn()).compareTo(other.isSetColumn());
if (lastComparison != 0) {
return lastComparison;
}
if (isSetColumn()) {
lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.column, other.column);
if (lastComparison != 0) {
return lastComparison;
}
}
lastComparison = Boolean.valueOf(isSetAmmount()).compareTo(other.isSetAmmount());
if (lastComparison != 0) {
return lastComparison;
}
if (isSetAmmount()) {
lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.ammount, other.ammount);
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("TIncrement(");
boolean first = true;
sb.append("table:");
if (this.table == null) {
sb.append("null");
} else {
org.apache.thrift.TBaseHelper.toString(this.table, sb);
}
first = false;
if (!first) sb.append(", ");
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("column:");
if (this.column == null) {
sb.append("null");
} else {
org.apache.thrift.TBaseHelper.toString(this.column, sb);
}
first = false;
if (!first) sb.append(", ");
sb.append("ammount:");
sb.append(this.ammount);
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 TIncrementStandardSchemeFactory implements SchemeFactory {
public TIncrementStandardScheme getScheme() {
return new TIncrementStandardScheme();
}
}
private static class TIncrementStandardScheme extends StandardScheme<TIncrement> {
public void read(org.apache.thrift.protocol.TProtocol iprot, TIncrement 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
if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
struct.table = iprot.readBinary();
struct.setTableIsSet(true);
} else {
org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
}
break;
case 2: // 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 3: // COLUMN
if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
struct.column = iprot.readBinary();
struct.setColumnIsSet(true);
} else {
org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
}
break;
case 4: // AMMOUNT
if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
struct.ammount = iprot.readI64();
struct.setAmmountIsSet(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, TIncrement struct) throws org.apache.thrift.TException {
struct.validate();
oprot.writeStructBegin(STRUCT_DESC);
if (struct.table != null) {
oprot.writeFieldBegin(TABLE_FIELD_DESC);
oprot.writeBinary(struct.table);
oprot.writeFieldEnd();
}
if (struct.row != null) {
oprot.writeFieldBegin(ROW_FIELD_DESC);
oprot.writeBinary(struct.row);
oprot.writeFieldEnd();
}
if (struct.column != null) {
oprot.writeFieldBegin(COLUMN_FIELD_DESC);
oprot.writeBinary(struct.column);
oprot.writeFieldEnd();
}
oprot.writeFieldBegin(AMMOUNT_FIELD_DESC);
oprot.writeI64(struct.ammount);
oprot.writeFieldEnd();
oprot.writeFieldStop();
oprot.writeStructEnd();
}
}
private static class TIncrementTupleSchemeFactory implements SchemeFactory {
public TIncrementTupleScheme getScheme() {
return new TIncrementTupleScheme();
}
}
private static class TIncrementTupleScheme extends TupleScheme<TIncrement> {
@Override
public void write(org.apache.thrift.protocol.TProtocol prot, TIncrement struct) throws org.apache.thrift.TException {
TTupleProtocol oprot = (TTupleProtocol) prot;
BitSet optionals = new BitSet();
if (struct.isSetTable()) {
optionals.set(0);
}
if (struct.isSetRow()) {
optionals.set(1);
}
if (struct.isSetColumn()) {
optionals.set(2);
}
if (struct.isSetAmmount()) {
optionals.set(3);
}
oprot.writeBitSet(optionals, 4);
if (struct.isSetTable()) {
oprot.writeBinary(struct.table);
}
if (struct.isSetRow()) {
oprot.writeBinary(struct.row);
}
if (struct.isSetColumn()) {
oprot.writeBinary(struct.column);
}
if (struct.isSetAmmount()) {
oprot.writeI64(struct.ammount);
}
}
@Override
public void read(org.apache.thrift.protocol.TProtocol prot, TIncrement struct) throws org.apache.thrift.TException {
TTupleProtocol iprot = (TTupleProtocol) prot;
BitSet incoming = iprot.readBitSet(4);
if (incoming.get(0)) {
struct.table = iprot.readBinary();
struct.setTableIsSet(true);
}
if (incoming.get(1)) {
struct.row = iprot.readBinary();
struct.setRowIsSet(true);
}
if (incoming.get(2)) {
struct.column = iprot.readBinary();
struct.setColumnIsSet(true);
}
if (incoming.get(3)) {
struct.ammount = iprot.readI64();
struct.setAmmountIsSet(true);
}
}
}
}

View File

@ -0,0 +1,745 @@
/**
* Autogenerated by Thrift Compiler (0.9.3)
*
* DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
* @generated
*/
package org.apache.hadoop.hbase.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 javax.annotation.Generated;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
/**
* Holds row name and then a map of columns to cells.
*/
@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2016-05-25")
public class TRowResult implements org.apache.thrift.TBase<TRowResult, TRowResult._Fields>, java.io.Serializable, Cloneable, Comparable<TRowResult> {
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);
private static final org.apache.thrift.protocol.TField COLUMNS_FIELD_DESC = new org.apache.thrift.protocol.TField("columns", org.apache.thrift.protocol.TType.MAP, (short)2);
private static final org.apache.thrift.protocol.TField SORTED_COLUMNS_FIELD_DESC = new org.apache.thrift.protocol.TField("sortedColumns", org.apache.thrift.protocol.TType.LIST, (short)3);
private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
static {
schemes.put(StandardScheme.class, new TRowResultStandardSchemeFactory());
schemes.put(TupleScheme.class, new TRowResultTupleSchemeFactory());
}
public ByteBuffer row; // required
public Map<ByteBuffer,TCell> columns; // optional
public List<TColumn> sortedColumns; // 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 {
ROW((short)1, "row"),
COLUMNS((short)2, "columns"),
SORTED_COLUMNS((short)3, "sortedColumns");
private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
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: // COLUMNS
return COLUMNS;
case 3: // SORTED_COLUMNS
return SORTED_COLUMNS;
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 _Fields optionals[] = {_Fields.COLUMNS,_Fields.SORTED_COLUMNS};
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.DEFAULT,
new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , "Text")));
tmpMap.put(_Fields.COLUMNS, new org.apache.thrift.meta_data.FieldMetaData("columns", org.apache.thrift.TFieldRequirementType.OPTIONAL,
new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP,
new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , "Text"),
new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TCell.class))));
tmpMap.put(_Fields.SORTED_COLUMNS, new org.apache.thrift.meta_data.FieldMetaData("sortedColumns", org.apache.thrift.TFieldRequirementType.OPTIONAL,
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, TColumn.class))));
metaDataMap = Collections.unmodifiableMap(tmpMap);
org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TRowResult.class, metaDataMap);
}
public TRowResult() {
}
public TRowResult(
ByteBuffer row)
{
this();
this.row = org.apache.thrift.TBaseHelper.copyBinary(row);
}
/**
* Performs a deep copy on <i>other</i>.
*/
public TRowResult(TRowResult other) {
if (other.isSetRow()) {
this.row = other.row;
}
if (other.isSetColumns()) {
Map<ByteBuffer,TCell> __this__columns = new HashMap<ByteBuffer,TCell>(other.columns.size());
for (Map.Entry<ByteBuffer, TCell> other_element : other.columns.entrySet()) {
ByteBuffer other_element_key = other_element.getKey();
TCell other_element_value = other_element.getValue();
ByteBuffer __this__columns_copy_key = other_element_key;
TCell __this__columns_copy_value = new TCell(other_element_value);
__this__columns.put(__this__columns_copy_key, __this__columns_copy_value);
}
this.columns = __this__columns;
}
if (other.isSetSortedColumns()) {
List<TColumn> __this__sortedColumns = new ArrayList<TColumn>(other.sortedColumns.size());
for (TColumn other_element : other.sortedColumns) {
__this__sortedColumns.add(new TColumn(other_element));
}
this.sortedColumns = __this__sortedColumns;
}
}
public TRowResult deepCopy() {
return new TRowResult(this);
}
@Override
public void clear() {
this.row = null;
this.columns = null;
this.sortedColumns = null;
}
public byte[] getRow() {
setRow(org.apache.thrift.TBaseHelper.rightSize(row));
return row == null ? null : row.array();
}
public ByteBuffer bufferForRow() {
return org.apache.thrift.TBaseHelper.copyBinary(row);
}
public TRowResult setRow(byte[] row) {
this.row = row == null ? (ByteBuffer)null : ByteBuffer.wrap(Arrays.copyOf(row, row.length));
return this;
}
public TRowResult setRow(ByteBuffer row) {
this.row = org.apache.thrift.TBaseHelper.copyBinary(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 int getColumnsSize() {
return (this.columns == null) ? 0 : this.columns.size();
}
public void putToColumns(ByteBuffer key, TCell val) {
if (this.columns == null) {
this.columns = new HashMap<ByteBuffer,TCell>();
}
this.columns.put(key, val);
}
public Map<ByteBuffer,TCell> getColumns() {
return this.columns;
}
public TRowResult setColumns(Map<ByteBuffer,TCell> columns) {
this.columns = columns;
return this;
}
public void unsetColumns() {
this.columns = null;
}
/** Returns true if field columns is set (has been assigned a value) and false otherwise */
public boolean isSetColumns() {
return this.columns != null;
}
public void setColumnsIsSet(boolean value) {
if (!value) {
this.columns = null;
}
}
public int getSortedColumnsSize() {
return (this.sortedColumns == null) ? 0 : this.sortedColumns.size();
}
public java.util.Iterator<TColumn> getSortedColumnsIterator() {
return (this.sortedColumns == null) ? null : this.sortedColumns.iterator();
}
public void addToSortedColumns(TColumn elem) {
if (this.sortedColumns == null) {
this.sortedColumns = new ArrayList<TColumn>();
}
this.sortedColumns.add(elem);
}
public List<TColumn> getSortedColumns() {
return this.sortedColumns;
}
public TRowResult setSortedColumns(List<TColumn> sortedColumns) {
this.sortedColumns = sortedColumns;
return this;
}
public void unsetSortedColumns() {
this.sortedColumns = null;
}
/** Returns true if field sortedColumns is set (has been assigned a value) and false otherwise */
public boolean isSetSortedColumns() {
return this.sortedColumns != null;
}
public void setSortedColumnsIsSet(boolean value) {
if (!value) {
this.sortedColumns = null;
}
}
public void setFieldValue(_Fields field, Object value) {
switch (field) {
case ROW:
if (value == null) {
unsetRow();
} else {
setRow((ByteBuffer)value);
}
break;
case COLUMNS:
if (value == null) {
unsetColumns();
} else {
setColumns((Map<ByteBuffer,TCell>)value);
}
break;
case SORTED_COLUMNS:
if (value == null) {
unsetSortedColumns();
} else {
setSortedColumns((List<TColumn>)value);
}
break;
}
}
public Object getFieldValue(_Fields field) {
switch (field) {
case ROW:
return getRow();
case COLUMNS:
return getColumns();
case SORTED_COLUMNS:
return getSortedColumns();
}
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 COLUMNS:
return isSetColumns();
case SORTED_COLUMNS:
return isSetSortedColumns();
}
throw new IllegalStateException();
}
@Override
public boolean equals(Object that) {
if (that == null)
return false;
if (that instanceof TRowResult)
return this.equals((TRowResult)that);
return false;
}
public boolean equals(TRowResult 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_columns = true && this.isSetColumns();
boolean that_present_columns = true && that.isSetColumns();
if (this_present_columns || that_present_columns) {
if (!(this_present_columns && that_present_columns))
return false;
if (!this.columns.equals(that.columns))
return false;
}
boolean this_present_sortedColumns = true && this.isSetSortedColumns();
boolean that_present_sortedColumns = true && that.isSetSortedColumns();
if (this_present_sortedColumns || that_present_sortedColumns) {
if (!(this_present_sortedColumns && that_present_sortedColumns))
return false;
if (!this.sortedColumns.equals(that.sortedColumns))
return false;
}
return true;
}
@Override
public int hashCode() {
List<Object> list = new ArrayList<Object>();
boolean present_row = true && (isSetRow());
list.add(present_row);
if (present_row)
list.add(row);
boolean present_columns = true && (isSetColumns());
list.add(present_columns);
if (present_columns)
list.add(columns);
boolean present_sortedColumns = true && (isSetSortedColumns());
list.add(present_sortedColumns);
if (present_sortedColumns)
list.add(sortedColumns);
return list.hashCode();
}
@Override
public int compareTo(TRowResult 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(isSetColumns()).compareTo(other.isSetColumns());
if (lastComparison != 0) {
return lastComparison;
}
if (isSetColumns()) {
lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.columns, other.columns);
if (lastComparison != 0) {
return lastComparison;
}
}
lastComparison = Boolean.valueOf(isSetSortedColumns()).compareTo(other.isSetSortedColumns());
if (lastComparison != 0) {
return lastComparison;
}
if (isSetSortedColumns()) {
lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.sortedColumns, other.sortedColumns);
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("TRowResult(");
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 (isSetColumns()) {
if (!first) sb.append(", ");
sb.append("columns:");
if (this.columns == null) {
sb.append("null");
} else {
sb.append(this.columns);
}
first = false;
}
if (isSetSortedColumns()) {
if (!first) sb.append(", ");
sb.append("sortedColumns:");
if (this.sortedColumns == null) {
sb.append("null");
} else {
sb.append(this.sortedColumns);
}
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 TRowResultStandardSchemeFactory implements SchemeFactory {
public TRowResultStandardScheme getScheme() {
return new TRowResultStandardScheme();
}
}
private static class TRowResultStandardScheme extends StandardScheme<TRowResult> {
public void read(org.apache.thrift.protocol.TProtocol iprot, TRowResult 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: // COLUMNS
if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
{
org.apache.thrift.protocol.TMap _map8 = iprot.readMapBegin();
struct.columns = new HashMap<ByteBuffer,TCell>(2*_map8.size);
ByteBuffer _key9;
TCell _val10;
for (int _i11 = 0; _i11 < _map8.size; ++_i11)
{
_key9 = iprot.readBinary();
_val10 = new TCell();
_val10.read(iprot);
struct.columns.put(_key9, _val10);
}
iprot.readMapEnd();
}
struct.setColumnsIsSet(true);
} else {
org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
}
break;
case 3: // SORTED_COLUMNS
if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
{
org.apache.thrift.protocol.TList _list12 = iprot.readListBegin();
struct.sortedColumns = new ArrayList<TColumn>(_list12.size);
TColumn _elem13;
for (int _i14 = 0; _i14 < _list12.size; ++_i14)
{
_elem13 = new TColumn();
_elem13.read(iprot);
struct.sortedColumns.add(_elem13);
}
iprot.readListEnd();
}
struct.setSortedColumnsIsSet(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, TRowResult 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.columns != null) {
if (struct.isSetColumns()) {
oprot.writeFieldBegin(COLUMNS_FIELD_DESC);
{
oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, struct.columns.size()));
for (Map.Entry<ByteBuffer, TCell> _iter15 : struct.columns.entrySet())
{
oprot.writeBinary(_iter15.getKey());
_iter15.getValue().write(oprot);
}
oprot.writeMapEnd();
}
oprot.writeFieldEnd();
}
}
if (struct.sortedColumns != null) {
if (struct.isSetSortedColumns()) {
oprot.writeFieldBegin(SORTED_COLUMNS_FIELD_DESC);
{
oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.sortedColumns.size()));
for (TColumn _iter16 : struct.sortedColumns)
{
_iter16.write(oprot);
}
oprot.writeListEnd();
}
oprot.writeFieldEnd();
}
}
oprot.writeFieldStop();
oprot.writeStructEnd();
}
}
private static class TRowResultTupleSchemeFactory implements SchemeFactory {
public TRowResultTupleScheme getScheme() {
return new TRowResultTupleScheme();
}
}
private static class TRowResultTupleScheme extends TupleScheme<TRowResult> {
@Override
public void write(org.apache.thrift.protocol.TProtocol prot, TRowResult struct) throws org.apache.thrift.TException {
TTupleProtocol oprot = (TTupleProtocol) prot;
BitSet optionals = new BitSet();
if (struct.isSetRow()) {
optionals.set(0);
}
if (struct.isSetColumns()) {
optionals.set(1);
}
if (struct.isSetSortedColumns()) {
optionals.set(2);
}
oprot.writeBitSet(optionals, 3);
if (struct.isSetRow()) {
oprot.writeBinary(struct.row);
}
if (struct.isSetColumns()) {
{
oprot.writeI32(struct.columns.size());
for (Map.Entry<ByteBuffer, TCell> _iter17 : struct.columns.entrySet())
{
oprot.writeBinary(_iter17.getKey());
_iter17.getValue().write(oprot);
}
}
}
if (struct.isSetSortedColumns()) {
{
oprot.writeI32(struct.sortedColumns.size());
for (TColumn _iter18 : struct.sortedColumns)
{
_iter18.write(oprot);
}
}
}
}
@Override
public void read(org.apache.thrift.protocol.TProtocol prot, TRowResult struct) throws org.apache.thrift.TException {
TTupleProtocol iprot = (TTupleProtocol) prot;
BitSet incoming = iprot.readBitSet(3);
if (incoming.get(0)) {
struct.row = iprot.readBinary();
struct.setRowIsSet(true);
}
if (incoming.get(1)) {
{
org.apache.thrift.protocol.TMap _map19 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
struct.columns = new HashMap<ByteBuffer,TCell>(2*_map19.size);
ByteBuffer _key20;
TCell _val21;
for (int _i22 = 0; _i22 < _map19.size; ++_i22)
{
_key20 = iprot.readBinary();
_val21 = new TCell();
_val21.read(iprot);
struct.columns.put(_key20, _val21);
}
}
struct.setColumnsIsSet(true);
}
if (incoming.get(2)) {
{
org.apache.thrift.protocol.TList _list23 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
struct.sortedColumns = new ArrayList<TColumn>(_list23.size);
TColumn _elem24;
for (int _i25 = 0; _i25 < _list23.size; ++_i25)
{
_elem24 = new TColumn();
_elem24.read(iprot);
struct.sortedColumns.add(_elem24);
}
}
struct.setSortedColumnsIsSet(true);
}
}
}
}

File diff suppressed because it is too large Load Diff

View File

@ -0,0 +1,487 @@
/**
*
* 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.thrift2;
import static org.apache.hadoop.hbase.thrift2.ThriftUtilities.appendFromThrift;
import static org.apache.hadoop.hbase.thrift2.ThriftUtilities.compareOpFromThrift;
import static org.apache.hadoop.hbase.thrift2.ThriftUtilities.deleteFromThrift;
import static org.apache.hadoop.hbase.thrift2.ThriftUtilities.deletesFromThrift;
import static org.apache.hadoop.hbase.thrift2.ThriftUtilities.getFromThrift;
import static org.apache.hadoop.hbase.thrift2.ThriftUtilities.getsFromThrift;
import static org.apache.hadoop.hbase.thrift2.ThriftUtilities.incrementFromThrift;
import static org.apache.hadoop.hbase.thrift2.ThriftUtilities.putFromThrift;
import static org.apache.hadoop.hbase.thrift2.ThriftUtilities.putsFromThrift;
import static org.apache.hadoop.hbase.thrift2.ThriftUtilities.resultFromHBase;
import static org.apache.hadoop.hbase.thrift2.ThriftUtilities.resultsFromHBase;
import static org.apache.hadoop.hbase.thrift2.ThriftUtilities.rowMutationsFromThrift;
import static org.apache.hadoop.hbase.thrift2.ThriftUtilities.scanFromThrift;
import static org.apache.thrift.TBaseHelper.byteBufferToByteArray;
import java.io.IOException;
import java.lang.reflect.InvocationHandler;
import java.lang.reflect.InvocationTargetException;
import java.lang.reflect.Method;
import java.lang.reflect.Proxy;
import java.nio.ByteBuffer;
import java.util.Collections;
import java.util.List;
import java.util.Map;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.atomic.AtomicInteger;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.client.RegionLocator;
import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.security.UserProvider;
import org.apache.hadoop.hbase.thrift.ThriftMetrics;
import org.apache.hadoop.hbase.thrift2.generated.TAppend;
import org.apache.hadoop.hbase.thrift2.generated.TCompareOp;
import org.apache.hadoop.hbase.thrift2.generated.TDelete;
import org.apache.hadoop.hbase.thrift2.generated.TGet;
import org.apache.hadoop.hbase.thrift2.generated.THBaseService;
import org.apache.hadoop.hbase.thrift2.generated.THRegionLocation;
import org.apache.hadoop.hbase.thrift2.generated.TIOError;
import org.apache.hadoop.hbase.thrift2.generated.TIllegalArgument;
import org.apache.hadoop.hbase.thrift2.generated.TIncrement;
import org.apache.hadoop.hbase.thrift2.generated.TPut;
import org.apache.hadoop.hbase.thrift2.generated.TResult;
import org.apache.hadoop.hbase.thrift2.generated.TRowMutations;
import org.apache.hadoop.hbase.thrift2.generated.TScan;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.ConnectionCache;
import org.apache.thrift.TException;
/**
* This class is a glue object that connects Thrift RPC calls to the HBase client API primarily
* defined in the Table interface.
*/
@InterfaceAudience.Private
@SuppressWarnings("deprecation")
public class ThriftHBaseServiceHandler implements THBaseService.Iface {
// TODO: Size of pool configuraple
private static final Log LOG = LogFactory.getLog(ThriftHBaseServiceHandler.class);
// nextScannerId and scannerMap are used to manage scanner state
// TODO: Cleanup thread for Scanners, Scanner id wrap
private final AtomicInteger nextScannerId = new AtomicInteger(0);
private final Map<Integer, ResultScanner> scannerMap =
new ConcurrentHashMap<Integer, ResultScanner>();
private final ConnectionCache connectionCache;
static final String CLEANUP_INTERVAL = "hbase.thrift.connection.cleanup-interval";
static final String MAX_IDLETIME = "hbase.thrift.connection.max-idletime";
public static THBaseService.Iface newInstance(
THBaseService.Iface handler, ThriftMetrics metrics) {
return (THBaseService.Iface) Proxy.newProxyInstance(handler.getClass().getClassLoader(),
new Class[] { THBaseService.Iface.class }, new THBaseServiceMetricsProxy(handler, metrics));
}
private static final 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(final Configuration conf,
final UserProvider userProvider) throws IOException {
int cleanInterval = conf.getInt(CLEANUP_INTERVAL, 10 * 1000);
int maxIdleTime = conf.getInt(MAX_IDLETIME, 10 * 60 * 1000);
connectionCache = new ConnectionCache(
conf, userProvider, cleanInterval, maxIdleTime);
}
private Table getTable(ByteBuffer tableName) {
try {
return connectionCache.getTable(Bytes.toString(byteBufferToByteArray(tableName)));
} catch (IOException ie) {
throw new RuntimeException(ie);
}
}
private RegionLocator getLocator(ByteBuffer tableName) {
try {
return connectionCache.getRegionLocator(byteBufferToByteArray(tableName));
} catch (IOException ie) {
throw new RuntimeException(ie);
}
}
private void closeTable(Table table) throws TIOError {
try {
table.close();
} catch (IOException e) {
throw getTIOError(e);
}
}
private TIOError getTIOError(IOException e) {
TIOError err = new TIOError();
err.setMessage(e.getMessage());
return err;
}
/**
* Assigns a unique ID to the scanner and adds the mapping to an internal HashMap.
* @param scanner to add
* @return Id for this Scanner
*/
private int addScanner(ResultScanner scanner) {
int id = nextScannerId.getAndIncrement();
scannerMap.put(id, scanner);
return id;
}
/**
* Returns the Scanner associated with the specified Id.
* @param id of the Scanner to get
* @return a Scanner, or null if the Id is invalid
*/
private ResultScanner getScanner(int id) {
return scannerMap.get(id);
}
void setEffectiveUser(String effectiveUser) {
connectionCache.setEffectiveUser(effectiveUser);
}
/**
* Removes the scanner associated with the specified ID from the internal HashMap.
* @param id of the Scanner to remove
* @return the removed Scanner, or <code>null</code> if the Id is invalid
*/
protected ResultScanner removeScanner(int id) {
return scannerMap.remove(id);
}
@Override
public boolean exists(ByteBuffer table, TGet get) throws TIOError, TException {
Table htable = getTable(table);
try {
return htable.exists(getFromThrift(get));
} catch (IOException e) {
throw getTIOError(e);
} finally {
closeTable(htable);
}
}
@Override
public TResult get(ByteBuffer table, TGet get) throws TIOError, TException {
Table htable = getTable(table);
try {
return resultFromHBase(htable.get(getFromThrift(get)));
} catch (IOException e) {
throw getTIOError(e);
} finally {
closeTable(htable);
}
}
@Override
public List<TResult> getMultiple(ByteBuffer table, List<TGet> gets) throws TIOError, TException {
Table htable = getTable(table);
try {
return resultsFromHBase(htable.get(getsFromThrift(gets)));
} catch (IOException e) {
throw getTIOError(e);
} finally {
closeTable(htable);
}
}
@Override
public void put(ByteBuffer table, TPut put) throws TIOError, TException {
Table htable = getTable(table);
try {
htable.put(putFromThrift(put));
} catch (IOException e) {
throw getTIOError(e);
} finally {
closeTable(htable);
}
}
@Override
public boolean checkAndPut(ByteBuffer table, ByteBuffer row, ByteBuffer family,
ByteBuffer qualifier, ByteBuffer value, TPut put) throws TIOError, TException {
Table htable = getTable(table);
try {
return htable.checkAndPut(byteBufferToByteArray(row), byteBufferToByteArray(family),
byteBufferToByteArray(qualifier), (value == null) ? null : byteBufferToByteArray(value),
putFromThrift(put));
} catch (IOException e) {
throw getTIOError(e);
} finally {
closeTable(htable);
}
}
@Override
public void putMultiple(ByteBuffer table, List<TPut> puts) throws TIOError, TException {
Table htable = getTable(table);
try {
htable.put(putsFromThrift(puts));
} catch (IOException e) {
throw getTIOError(e);
} finally {
closeTable(htable);
}
}
@Override
public void deleteSingle(ByteBuffer table, TDelete deleteSingle) throws TIOError, TException {
Table htable = getTable(table);
try {
htable.delete(deleteFromThrift(deleteSingle));
} catch (IOException e) {
throw getTIOError(e);
} finally {
closeTable(htable);
}
}
@Override
public List<TDelete> deleteMultiple(ByteBuffer table, List<TDelete> deletes) throws TIOError,
TException {
Table htable = getTable(table);
try {
htable.delete(deletesFromThrift(deletes));
} catch (IOException e) {
throw getTIOError(e);
} finally {
closeTable(htable);
}
return Collections.emptyList();
}
@Override
public boolean checkAndMutate(ByteBuffer table, ByteBuffer row, ByteBuffer family,
ByteBuffer qualifier, TCompareOp compareOp, ByteBuffer value, TRowMutations rowMutations)
throws TIOError, TException {
try (final Table htable = getTable(table)) {
return htable.checkAndMutate(byteBufferToByteArray(row), byteBufferToByteArray(family),
byteBufferToByteArray(qualifier), compareOpFromThrift(compareOp),
byteBufferToByteArray(value), rowMutationsFromThrift(rowMutations));
} catch (IOException e) {
throw getTIOError(e);
}
}
@Override
public boolean checkAndDelete(ByteBuffer table, ByteBuffer row, ByteBuffer family,
ByteBuffer qualifier, ByteBuffer value, TDelete deleteSingle) throws TIOError, TException {
Table htable = getTable(table);
try {
if (value == null) {
return htable.checkAndDelete(byteBufferToByteArray(row), byteBufferToByteArray(family),
byteBufferToByteArray(qualifier), null, deleteFromThrift(deleteSingle));
} else {
return htable.checkAndDelete(byteBufferToByteArray(row), byteBufferToByteArray(family),
byteBufferToByteArray(qualifier), byteBufferToByteArray(value),
deleteFromThrift(deleteSingle));
}
} catch (IOException e) {
throw getTIOError(e);
} finally {
closeTable(htable);
}
}
@Override
public TResult increment(ByteBuffer table, TIncrement increment) throws TIOError, TException {
Table htable = getTable(table);
try {
return resultFromHBase(htable.increment(incrementFromThrift(increment)));
} catch (IOException e) {
throw getTIOError(e);
} finally {
closeTable(htable);
}
}
@Override
public TResult append(ByteBuffer table, TAppend append) throws TIOError, TException {
Table htable = getTable(table);
try {
return resultFromHBase(htable.append(appendFromThrift(append)));
} catch (IOException e) {
throw getTIOError(e);
} finally {
closeTable(htable);
}
}
@Override
public int openScanner(ByteBuffer table, TScan scan) throws TIOError, TException {
Table htable = getTable(table);
ResultScanner resultScanner = null;
try {
resultScanner = htable.getScanner(scanFromThrift(scan));
} catch (IOException e) {
throw getTIOError(e);
} finally {
closeTable(htable);
}
return addScanner(resultScanner);
}
@Override
public List<TResult> getScannerRows(int scannerId, int numRows) throws TIOError,
TIllegalArgument, TException {
ResultScanner scanner = getScanner(scannerId);
if (scanner == null) {
TIllegalArgument ex = new TIllegalArgument();
ex.setMessage("Invalid scanner Id");
throw ex;
}
try {
connectionCache.updateConnectionAccessTime();
return resultsFromHBase(scanner.next(numRows));
} catch (IOException e) {
throw getTIOError(e);
}
}
@Override
public List<TResult> getScannerResults(ByteBuffer table, TScan scan, int numRows)
throws TIOError, TException {
Table htable = getTable(table);
List<TResult> results = null;
ResultScanner scanner = null;
try {
scanner = htable.getScanner(scanFromThrift(scan));
results = resultsFromHBase(scanner.next(numRows));
} catch (IOException e) {
throw getTIOError(e);
} finally {
if (scanner != null) {
scanner.close();
}
closeTable(htable);
}
return results;
}
@Override
public void closeScanner(int scannerId) throws TIOError, TIllegalArgument, TException {
LOG.debug("scannerClose: id=" + scannerId);
ResultScanner scanner = getScanner(scannerId);
if (scanner == null) {
String message = "scanner ID is invalid";
LOG.warn(message);
TIllegalArgument ex = new TIllegalArgument();
ex.setMessage("Invalid scanner Id");
throw ex;
}
scanner.close();
removeScanner(scannerId);
}
@Override
public void mutateRow(ByteBuffer table, TRowMutations rowMutations) throws TIOError, TException {
Table htable = getTable(table);
try {
htable.mutateRow(rowMutationsFromThrift(rowMutations));
} catch (IOException e) {
throw getTIOError(e);
} finally {
closeTable(htable);
}
}
@Override
public List<THRegionLocation> getAllRegionLocations(ByteBuffer table)
throws TIOError, TException {
RegionLocator locator = null;
try {
locator = getLocator(table);
return ThriftUtilities.regionLocationsFromHBase(locator.getAllRegionLocations());
} catch (IOException e) {
throw getTIOError(e);
} finally {
if (locator != null) {
try {
locator.close();
} catch (IOException e) {
LOG.warn("Couldn't close the locator.", e);
}
}
}
}
@Override
public THRegionLocation getRegionLocation(ByteBuffer table, ByteBuffer row, boolean reload)
throws TIOError, TException {
RegionLocator locator = null;
try {
locator = getLocator(table);
byte[] rowBytes = byteBufferToByteArray(row);
HRegionLocation hrl = locator.getRegionLocation(rowBytes, reload);
return ThriftUtilities.regionLocationFromHBase(hrl);
} catch (IOException e) {
throw getTIOError(e);
} finally {
if (locator != null) {
try {
locator.close();
} catch (IOException e) {
LOG.warn("Couldn't close the locator.", e);
}
}
}
}
}

View File

@ -0,0 +1,555 @@
/**
*
* 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.thrift2;
import java.io.IOException;
import java.net.InetAddress;
import java.net.InetSocketAddress;
import java.net.UnknownHostException;
import java.security.PrivilegedAction;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.LinkedBlockingQueue;
import java.util.concurrent.ThreadPoolExecutor;
import java.util.concurrent.TimeUnit;
import javax.security.auth.callback.Callback;
import javax.security.auth.callback.UnsupportedCallbackException;
import javax.security.sasl.AuthorizeCallback;
import javax.security.sasl.Sasl;
import javax.security.sasl.SaslServer;
import org.apache.commons.cli.CommandLine;
import org.apache.commons.cli.CommandLineParser;
import org.apache.commons.cli.HelpFormatter;
import org.apache.commons.cli.Option;
import org.apache.commons.cli.OptionGroup;
import org.apache.commons.cli.Options;
import org.apache.commons.cli.ParseException;
import org.apache.commons.cli.PosixParser;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.conf.Configured;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
import org.apache.hadoop.hbase.filter.ParseFilter;
import org.apache.hadoop.hbase.http.InfoServer;
import org.apache.hadoop.hbase.security.SaslUtil;
import org.apache.hadoop.hbase.security.SecurityUtil;
import org.apache.hadoop.hbase.security.UserProvider;
import org.apache.hadoop.hbase.thrift.CallQueue;
import org.apache.hadoop.hbase.thrift.CallQueue.Call;
import org.apache.hadoop.hbase.thrift.ThriftMetrics;
import org.apache.hadoop.hbase.thrift2.generated.THBaseService;
import org.apache.hadoop.hbase.util.DNS;
import org.apache.hadoop.hbase.util.JvmPauseMonitor;
import org.apache.hadoop.hbase.util.Strings;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.SaslRpcServer.SaslGssCallbackHandler;
import org.apache.hadoop.util.Tool;
import org.apache.hadoop.util.ToolRunner;
import org.apache.thrift.TException;
import org.apache.thrift.TProcessor;
import org.apache.thrift.protocol.TBinaryProtocol;
import org.apache.thrift.protocol.TCompactProtocol;
import org.apache.thrift.protocol.TProtocol;
import org.apache.thrift.protocol.TProtocolFactory;
import org.apache.thrift.server.THsHaServer;
import org.apache.thrift.server.TNonblockingServer;
import org.apache.thrift.server.TServer;
import org.apache.thrift.server.TThreadPoolServer;
import org.apache.thrift.transport.TFramedTransport;
import org.apache.thrift.transport.TNonblockingServerSocket;
import org.apache.thrift.transport.TNonblockingServerTransport;
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 com.google.common.util.concurrent.ThreadFactoryBuilder;
/**
* ThriftServer - this class starts up a Thrift server which implements the HBase API specified in the
* HbaseClient.thrift IDL file.
*/
@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.TOOLS)
@SuppressWarnings({ "rawtypes", "unchecked" })
public class ThriftServer extends Configured implements Tool {
private static final Log log = LogFactory.getLog(ThriftServer.class);
/**
* Thrift quality of protection configuration key. Valid values can be:
* privacy: authentication, integrity and confidentiality checking
* integrity: authentication and integrity checking
* authentication: authentication only
*
* This is used to authenticate the callers and support impersonation.
* The thrift server and the HBase cluster must run in secure mode.
*/
static final String THRIFT_QOP_KEY = "hbase.thrift.security.qop";
static final String BACKLOG_CONF_KEY = "hbase.regionserver.thrift.backlog";
public static final int DEFAULT_LISTEN_PORT = 9090;
private static final String READ_TIMEOUT_OPTION = "readTimeout";
/**
* Amount of time in milliseconds before a server thread will timeout
* waiting for client to send data on a connected socket. Currently,
* applies only to TBoundedThreadPoolServer
*/
public static final String THRIFT_SERVER_SOCKET_READ_TIMEOUT_KEY =
"hbase.thrift.server.socket.read.timeout";
public static final int THRIFT_SERVER_SOCKET_READ_TIMEOUT_DEFAULT = 60000;
public ThriftServer() {
}
private static void printUsage() {
HelpFormatter formatter = new HelpFormatter();
formatter.printHelp("Thrift", null, getOptions(),
"To start the Thrift server run 'bin/hbase-daemon.sh start thrift2'\n" +
"To shutdown the thrift server run 'bin/hbase-daemon.sh stop thrift2' or" +
" send a kill signal to the thrift server pid",
true);
}
private static Options getOptions() {
Options options = new Options();
options.addOption("b", "bind", true,
"Address to bind the Thrift server to. [default: 0.0.0.0]");
options.addOption("p", "port", true, "Port to bind to [default: " + DEFAULT_LISTEN_PORT + "]");
options.addOption("f", "framed", false, "Use framed transport");
options.addOption("c", "compact", false, "Use the compact protocol");
options.addOption("w", "workers", true, "How many worker threads to use.");
options.addOption("q", "callQueueSize", true,
"Max size of request queue (unbounded by default)");
options.addOption("h", "help", false, "Print help information");
options.addOption(null, "infoport", true, "Port for web UI");
options.addOption("t", READ_TIMEOUT_OPTION, true,
"Amount of time in milliseconds before a server thread will timeout " +
"waiting for client to send data on a connected socket. Currently, " +
"only applies to TBoundedThreadPoolServer");
OptionGroup servers = new OptionGroup();
servers.addOption(
new Option("nonblocking", false, "Use the TNonblockingServer. This implies the framed transport."));
servers.addOption(new Option("hsha", false, "Use the THsHaServer. This implies the framed transport."));
servers.addOption(new Option("threadpool", false, "Use the TThreadPoolServer. This is the default."));
options.addOptionGroup(servers);
return options;
}
private static CommandLine parseArguments(Configuration conf, Options options, String[] args)
throws ParseException, IOException {
CommandLineParser parser = new PosixParser();
return parser.parse(options, args);
}
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(
SaslUtil.QualityOfProtection qop, String name, String host,
boolean framed, int frameSize) {
if (framed) {
if (qop != null) {
throw new RuntimeException("Thrift server authentication"
+ " doesn't work with framed transport yet");
}
log.debug("Using framed transport");
return new TFramedTransport.Factory(frameSize);
} else if (qop == null) {
return new TTransportFactory();
} else {
Map<String, String> saslProperties = new HashMap<String, String>();
saslProperties.put(Sasl.QOP, qop.getSaslQop());
TSaslServerTransport.Factory saslFactory = new TSaslServerTransport.Factory();
saslFactory.addServerDefinition("GSSAPI", name, host, saslProperties,
new SaslGssCallbackHandler() {
@Override
public void handle(Callback[] callbacks)
throws UnsupportedCallbackException {
AuthorizeCallback ac = null;
for (Callback callback : callbacks) {
if (callback instanceof AuthorizeCallback) {
ac = (AuthorizeCallback) callback;
} else {
throw new UnsupportedCallbackException(callback,
"Unrecognized SASL GSSAPI Callback");
}
}
if (ac != null) {
String authid = ac.getAuthenticationID();
String authzid = ac.getAuthorizationID();
if (!authid.equals(authzid)) {
ac.setAuthorized(false);
} else {
ac.setAuthorized(true);
String userName = SecurityUtil.getUserFromPrincipal(authzid);
log.info("Effective user: " + userName);
ac.setAuthorizedID(userName);
}
}
}
});
return saslFactory;
}
}
/*
* 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 getTNonBlockingServer(TProtocolFactory protocolFactory, TProcessor processor,
TTransportFactory transportFactory, InetSocketAddress inetSocketAddress) throws TTransportException {
TNonblockingServerTransport serverTransport = new TNonblockingServerSocket(inetSocketAddress);
log.info("starting HBase Nonblocking Thrift server on " + inetSocketAddress.toString());
TNonblockingServer.Args serverArgs = new TNonblockingServer.Args(serverTransport);
serverArgs.processor(processor);
serverArgs.transportFactory(transportFactory);
serverArgs.protocolFactory(protocolFactory);
return new TNonblockingServer(serverArgs);
}
private static TServer getTHsHaServer(TProtocolFactory protocolFactory,
TProcessor processor, TTransportFactory transportFactory,
int workerThreads, int maxCallQueueSize,
InetSocketAddress inetSocketAddress, ThriftMetrics metrics)
throws TTransportException {
TNonblockingServerTransport serverTransport = new TNonblockingServerSocket(inetSocketAddress);
log.info("starting HBase HsHA Thrift server on " + inetSocketAddress.toString());
THsHaServer.Args serverArgs = new THsHaServer.Args(serverTransport);
if (workerThreads > 0) {
// Could support the min & max threads, avoiding to preserve existing functionality.
serverArgs.minWorkerThreads(workerThreads).maxWorkerThreads(workerThreads);
}
ExecutorService executorService = createExecutor(
workerThreads, maxCallQueueSize, metrics);
serverArgs.executorService(executorService);
serverArgs.processor(processor);
serverArgs.transportFactory(transportFactory);
serverArgs.protocolFactory(protocolFactory);
return new THsHaServer(serverArgs);
}
private static ExecutorService createExecutor(
int workerThreads, int maxCallQueueSize, ThriftMetrics metrics) {
CallQueue callQueue;
if (maxCallQueueSize > 0) {
callQueue = new CallQueue(new LinkedBlockingQueue<Call>(maxCallQueueSize), metrics);
} else {
callQueue = new CallQueue(new LinkedBlockingQueue<Call>(), metrics);
}
ThreadFactoryBuilder tfb = new ThreadFactoryBuilder();
tfb.setDaemon(true);
tfb.setNameFormat("thrift2-worker-%d");
ThreadPoolExecutor pool = new ThreadPoolExecutor(workerThreads, workerThreads,
Long.MAX_VALUE, TimeUnit.SECONDS, callQueue, tfb.build());
pool.prestartAllCoreThreads();
return pool;
}
private static TServer getTThreadPoolServer(TProtocolFactory protocolFactory,
TProcessor processor,
TTransportFactory transportFactory,
int workerThreads,
InetSocketAddress inetSocketAddress,
int backlog,
int clientTimeout)
throws TTransportException {
TServerTransport serverTransport = new TServerSocket(
new TServerSocket.ServerSocketTransportArgs().
bindAddr(inetSocketAddress).backlog(backlog).
clientTimeout(clientTimeout));
log.info("starting HBase ThreadPool Thrift server on " + inetSocketAddress.toString());
TThreadPoolServer.Args serverArgs = new TThreadPoolServer.Args(serverTransport);
serverArgs.processor(processor);
serverArgs.transportFactory(transportFactory);
serverArgs.protocolFactory(protocolFactory);
if (workerThreads > 0) {
serverArgs.maxWorkerThreads(workerThreads);
}
return new TThreadPoolServer(serverArgs);
}
/**
* Adds the option to pre-load filters at startup.
*
* @param conf The current configuration instance.
*/
protected static void registerFilters(Configuration conf) {
String[] filters = conf.getStrings("hbase.thrift.filters");
if(filters != null) {
for(String filterClass: filters) {
String[] filterPart = filterClass.split(":");
if(filterPart.length != 2) {
log.warn("Invalid filter specification " + filterClass + " - skipping");
} else {
ParseFilter.registerFilter(filterPart[0], filterPart[1]);
}
}
}
}
/**
* Start up the Thrift2 server.
*/
public static void main(String[] args) throws Exception {
final Configuration conf = HBaseConfiguration.create();
// for now, only time we return is on an argument error.
final int status = ToolRunner.run(conf, new ThriftServer(), args);
System.exit(status);
}
@Override
public int run(String[] args) throws Exception {
final Configuration conf = getConf();
TServer server = null;
Options options = getOptions();
CommandLine cmd = parseArguments(conf, options, args);
int workerThreads = 0;
int maxCallQueueSize = -1; // use unbounded queue by default
/**
* This is to please both bin/hbase and bin/hbase-daemon. hbase-daemon provides "start" and "stop" arguments hbase
* should print the help if no argument is provided
*/
List<?> argList = cmd.getArgList();
if (cmd.hasOption("help") || !argList.contains("start") || argList.contains("stop")) {
printUsage();
return 1;
}
// Get address to bind
String bindAddress;
if (cmd.hasOption("bind")) {
bindAddress = cmd.getOptionValue("bind");
conf.set("hbase.thrift.info.bindAddress", bindAddress);
} else {
bindAddress = conf.get("hbase.thrift.info.bindAddress");
}
// Get read timeout
int readTimeout = THRIFT_SERVER_SOCKET_READ_TIMEOUT_DEFAULT;
if (cmd.hasOption(READ_TIMEOUT_OPTION)) {
try {
readTimeout = Integer.parseInt(cmd.getOptionValue(READ_TIMEOUT_OPTION));
} catch (NumberFormatException e) {
throw new RuntimeException("Could not parse the value provided for the timeout option", e);
}
} else {
readTimeout = conf.getInt(THRIFT_SERVER_SOCKET_READ_TIMEOUT_KEY,
THRIFT_SERVER_SOCKET_READ_TIMEOUT_DEFAULT);
}
// Get port to bind to
int listenPort = 0;
try {
if (cmd.hasOption("port")) {
listenPort = Integer.parseInt(cmd.getOptionValue("port"));
} else {
listenPort = conf.getInt("hbase.regionserver.thrift.port", DEFAULT_LISTEN_PORT);
}
} catch (NumberFormatException e) {
throw new RuntimeException("Could not parse the value provided for the port option", e);
}
// Thrift's implementation uses '0' as a placeholder for 'use the default.'
int backlog = conf.getInt(BACKLOG_CONF_KEY, 0);
// Local hostname and user name,
// used only if QOP is configured.
String host = null;
String name = null;
UserProvider userProvider = UserProvider.instantiate(conf);
// login the server principal (if using secure Hadoop)
boolean securityEnabled = userProvider.isHadoopSecurityEnabled()
&& userProvider.isHBaseSecurityEnabled();
if (securityEnabled) {
host = Strings.domainNamePointerToHostName(DNS.getDefaultHost(
conf.get("hbase.thrift.dns.interface", "default"),
conf.get("hbase.thrift.dns.nameserver", "default")));
userProvider.login("hbase.thrift.keytab.file",
"hbase.thrift.kerberos.principal", host);
}
UserGroupInformation realUser = userProvider.getCurrent().getUGI();
String stringQop = conf.get(THRIFT_QOP_KEY);
SaslUtil.QualityOfProtection qop = null;
if (stringQop != null) {
qop = SaslUtil.getQop(stringQop);
if (!securityEnabled) {
throw new IOException("Thrift server must"
+ " run in secure mode to support authentication");
}
// Extract the name from the principal
name = SecurityUtil.getUserFromPrincipal(
conf.get("hbase.thrift.kerberos.principal"));
}
boolean nonblocking = cmd.hasOption("nonblocking");
boolean hsha = cmd.hasOption("hsha");
ThriftMetrics metrics = new ThriftMetrics(conf, ThriftMetrics.ThriftServerType.TWO);
final JvmPauseMonitor pauseMonitor = new JvmPauseMonitor(conf, metrics.getSource());
String implType = "threadpool";
if (nonblocking) {
implType = "nonblocking";
} else if (hsha) {
implType = "hsha";
}
conf.set("hbase.regionserver.thrift.server.type", implType);
conf.setInt("hbase.regionserver.thrift.port", listenPort);
registerFilters(conf);
// Construct correct ProtocolFactory
boolean compact = cmd.hasOption("compact") ||
conf.getBoolean("hbase.regionserver.thrift.compact", false);
TProtocolFactory protocolFactory = getTProtocolFactory(compact);
final ThriftHBaseServiceHandler hbaseHandler =
new ThriftHBaseServiceHandler(conf, userProvider);
THBaseService.Iface handler =
ThriftHBaseServiceHandler.newInstance(hbaseHandler, metrics);
final THBaseService.Processor p = new THBaseService.Processor(handler);
conf.setBoolean("hbase.regionserver.thrift.compact", compact);
TProcessor processor = p;
boolean framed = cmd.hasOption("framed") ||
conf.getBoolean("hbase.regionserver.thrift.framed", false) || nonblocking || hsha;
TTransportFactory transportFactory = getTTransportFactory(qop, name, host, framed,
conf.getInt("hbase.regionserver.thrift.framed.max_frame_size_in_mb", 2) * 1024 * 1024);
InetSocketAddress inetSocketAddress = bindToPort(bindAddress, listenPort);
conf.setBoolean("hbase.regionserver.thrift.framed", framed);
if (qop != null) {
// Create a processor wrapper, to get the caller
processor = new TProcessor() {
@Override
public boolean process(TProtocol inProt,
TProtocol outProt) throws TException {
TSaslServerTransport saslServerTransport =
(TSaslServerTransport)inProt.getTransport();
SaslServer saslServer = saslServerTransport.getSaslServer();
String principal = saslServer.getAuthorizationID();
hbaseHandler.setEffectiveUser(principal);
return p.process(inProt, outProt);
}
};
}
if (cmd.hasOption("w")) {
workerThreads = Integer.parseInt(cmd.getOptionValue("w"));
}
if (cmd.hasOption("q")) {
maxCallQueueSize = Integer.parseInt(cmd.getOptionValue("q"));
}
// check for user-defined info server port setting, if so override the conf
try {
if (cmd.hasOption("infoport")) {
String val = cmd.getOptionValue("infoport");
conf.setInt("hbase.thrift.info.port", Integer.parseInt(val));
log.debug("Web UI port set to " + val);
}
} catch (NumberFormatException e) {
log.error("Could not parse the value provided for the infoport option", e);
printUsage();
System.exit(1);
}
// Put up info server.
int port = conf.getInt("hbase.thrift.info.port", 9095);
if (port >= 0) {
conf.setLong("startcode", System.currentTimeMillis());
String a = conf.get("hbase.thrift.info.bindAddress", "0.0.0.0");
InfoServer infoServer = new InfoServer("thrift", a, port, false, conf);
infoServer.setAttribute("hbase.conf", conf);
infoServer.start();
}
if (nonblocking) {
server = getTNonBlockingServer(protocolFactory,
processor,
transportFactory,
inetSocketAddress);
} else if (hsha) {
server = getTHsHaServer(protocolFactory,
processor,
transportFactory,
workerThreads,
maxCallQueueSize,
inetSocketAddress,
metrics);
} else {
server = getTThreadPoolServer(protocolFactory,
processor,
transportFactory,
workerThreads,
inetSocketAddress,
backlog,
readTimeout);
}
final TServer tserver = server;
realUser.doAs(
new PrivilegedAction<Object>() {
@Override
public Object run() {
pauseMonitor.start();
try {
tserver.serve();
return null;
} finally {
pauseMonitor.stop();
}
}
});
// when tserver.stop eventually happens we'll get here.
return 0;
}
}

View File

@ -0,0 +1,552 @@
/**
*
* 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.thrift2;
import static org.apache.hadoop.hbase.util.Bytes.getBytes;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.List;
import java.util.Map;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.client.Append;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Durability;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.Increment;
import org.apache.hadoop.hbase.client.OperationWithAttributes;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.RowMutations;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
import org.apache.hadoop.hbase.filter.ParseFilter;
import org.apache.hadoop.hbase.security.visibility.Authorizations;
import org.apache.hadoop.hbase.security.visibility.CellVisibility;
import org.apache.hadoop.hbase.thrift2.generated.TAppend;
import org.apache.hadoop.hbase.thrift2.generated.TColumn;
import org.apache.hadoop.hbase.thrift2.generated.TColumnIncrement;
import org.apache.hadoop.hbase.thrift2.generated.TColumnValue;
import org.apache.hadoop.hbase.thrift2.generated.TCompareOp;
import org.apache.hadoop.hbase.thrift2.generated.TDelete;
import org.apache.hadoop.hbase.thrift2.generated.TDeleteType;
import org.apache.hadoop.hbase.thrift2.generated.TDurability;
import org.apache.hadoop.hbase.thrift2.generated.TGet;
import org.apache.hadoop.hbase.thrift2.generated.THRegionInfo;
import org.apache.hadoop.hbase.thrift2.generated.THRegionLocation;
import org.apache.hadoop.hbase.thrift2.generated.TIncrement;
import org.apache.hadoop.hbase.thrift2.generated.TMutation;
import org.apache.hadoop.hbase.thrift2.generated.TPut;
import org.apache.hadoop.hbase.thrift2.generated.TResult;
import org.apache.hadoop.hbase.thrift2.generated.TRowMutations;
import org.apache.hadoop.hbase.thrift2.generated.TScan;
import org.apache.hadoop.hbase.thrift2.generated.TServerName;
import org.apache.hadoop.hbase.thrift2.generated.TTimeRange;
import org.apache.hadoop.hbase.util.Bytes;
@InterfaceAudience.Private
public class ThriftUtilities {
private ThriftUtilities() {
throw new UnsupportedOperationException("Can't initialize class");
}
/**
* Creates a {@link Get} (HBase) from a {@link TGet} (Thrift).
*
* This ignores any timestamps set on {@link TColumn} objects.
*
* @param in the <code>TGet</code> to convert
*
* @return <code>Get</code> object
*
* @throws IOException if an invalid time range or max version parameter is given
*/
public static Get getFromThrift(TGet in) throws IOException {
Get out = new Get(in.getRow());
// Timestamp overwrites time range if both are set
if (in.isSetTimestamp()) {
out.setTimeStamp(in.getTimestamp());
} else if (in.isSetTimeRange()) {
out.setTimeRange(in.getTimeRange().getMinStamp(), in.getTimeRange().getMaxStamp());
}
if (in.isSetMaxVersions()) {
out.setMaxVersions(in.getMaxVersions());
}
if (in.isSetFilterString()) {
ParseFilter parseFilter = new ParseFilter();
out.setFilter(parseFilter.parseFilterString(in.getFilterString()));
}
if (in.isSetAttributes()) {
addAttributes(out,in.getAttributes());
}
if (in.isSetAuthorizations()) {
out.setAuthorizations(new Authorizations(in.getAuthorizations().getLabels()));
}
if (!in.isSetColumns()) {
return out;
}
for (TColumn column : in.getColumns()) {
if (column.isSetQualifier()) {
out.addColumn(column.getFamily(), column.getQualifier());
} else {
out.addFamily(column.getFamily());
}
}
return out;
}
/**
* Converts multiple {@link TGet}s (Thrift) into a list of {@link Get}s (HBase).
*
* @param in list of <code>TGet</code>s to convert
*
* @return list of <code>Get</code> objects
*
* @throws IOException if an invalid time range or max version parameter is given
* @see #getFromThrift(TGet)
*/
public static List<Get> getsFromThrift(List<TGet> in) throws IOException {
List<Get> out = new ArrayList<Get>(in.size());
for (TGet get : in) {
out.add(getFromThrift(get));
}
return out;
}
/**
* Creates a {@link TResult} (Thrift) from a {@link Result} (HBase).
*
* @param in the <code>Result</code> to convert
*
* @return converted result, returns an empty result if the input is <code>null</code>
*/
public static TResult resultFromHBase(Result in) {
Cell[] raw = in.rawCells();
TResult out = new TResult();
byte[] row = in.getRow();
if (row != null) {
out.setRow(in.getRow());
}
List<TColumnValue> columnValues = new ArrayList<TColumnValue>();
for (Cell kv : raw) {
TColumnValue col = new TColumnValue();
col.setFamily(CellUtil.cloneFamily(kv));
col.setQualifier(CellUtil.cloneQualifier(kv));
col.setTimestamp(kv.getTimestamp());
col.setValue(CellUtil.cloneValue(kv));
if (kv.getTagsLength() > 0) {
col.setTags(CellUtil.getTagArray(kv));
}
columnValues.add(col);
}
out.setColumnValues(columnValues);
return out;
}
/**
* Converts multiple {@link Result}s (HBase) into a list of {@link TResult}s (Thrift).
*
* @param in array of <code>Result</code>s to convert
*
* @return list of converted <code>TResult</code>s
*
* @see #resultFromHBase(Result)
*/
public static List<TResult> resultsFromHBase(Result[] in) {
List<TResult> out = new ArrayList<TResult>(in.length);
for (Result result : in) {
out.add(resultFromHBase(result));
}
return out;
}
/**
* Creates a {@link Put} (HBase) from a {@link TPut} (Thrift)
*
* @param in the <code>TPut</code> to convert
*
* @return converted <code>Put</code>
*/
public static Put putFromThrift(TPut in) {
Put out;
if (in.isSetTimestamp()) {
out = new Put(in.getRow(), in.getTimestamp());
} else {
out = new Put(in.getRow());
}
if (in.isSetDurability()) {
out.setDurability(durabilityFromThrift(in.getDurability()));
}
for (TColumnValue columnValue : in.getColumnValues()) {
if (columnValue.isSetTimestamp()) {
out.addImmutable(
columnValue.getFamily(), columnValue.getQualifier(), columnValue.getTimestamp(),
columnValue.getValue());
} else {
out.addImmutable(
columnValue.getFamily(), columnValue.getQualifier(), columnValue.getValue());
}
}
if (in.isSetAttributes()) {
addAttributes(out,in.getAttributes());
}
if (in.getCellVisibility() != null) {
out.setCellVisibility(new CellVisibility(in.getCellVisibility().getExpression()));
}
return out;
}
/**
* Converts multiple {@link TPut}s (Thrift) into a list of {@link Put}s (HBase).
*
* @param in list of <code>TPut</code>s to convert
*
* @return list of converted <code>Put</code>s
*
* @see #putFromThrift(TPut)
*/
public static List<Put> putsFromThrift(List<TPut> in) {
List<Put> out = new ArrayList<Put>(in.size());
for (TPut put : in) {
out.add(putFromThrift(put));
}
return out;
}
/**
* Creates a {@link Delete} (HBase) from a {@link TDelete} (Thrift).
*
* @param in the <code>TDelete</code> to convert
*
* @return converted <code>Delete</code>
*/
public static Delete deleteFromThrift(TDelete in) {
Delete out;
if (in.isSetColumns()) {
out = new Delete(in.getRow());
for (TColumn column : in.getColumns()) {
if (column.isSetQualifier()) {
if (column.isSetTimestamp()) {
if (in.isSetDeleteType() &&
in.getDeleteType().equals(TDeleteType.DELETE_COLUMNS))
out.addColumns(column.getFamily(), column.getQualifier(), column.getTimestamp());
else
out.addColumn(column.getFamily(), column.getQualifier(), column.getTimestamp());
} else {
if (in.isSetDeleteType() &&
in.getDeleteType().equals(TDeleteType.DELETE_COLUMNS))
out.addColumns(column.getFamily(), column.getQualifier());
else
out.addColumn(column.getFamily(), column.getQualifier());
}
} else {
if (column.isSetTimestamp()) {
out.addFamily(column.getFamily(), column.getTimestamp());
} else {
out.addFamily(column.getFamily());
}
}
}
} else {
if (in.isSetTimestamp()) {
out = new Delete(in.getRow(), in.getTimestamp());
} else {
out = new Delete(in.getRow());
}
}
if (in.isSetAttributes()) {
addAttributes(out,in.getAttributes());
}
if (in.isSetDurability()) {
out.setDurability(durabilityFromThrift(in.getDurability()));
}
return out;
}
/**
* Converts multiple {@link TDelete}s (Thrift) into a list of {@link Delete}s (HBase).
*
* @param in list of <code>TDelete</code>s to convert
*
* @return list of converted <code>Delete</code>s
*
* @see #deleteFromThrift(TDelete)
*/
public static List<Delete> deletesFromThrift(List<TDelete> in) {
List<Delete> out = new ArrayList<Delete>(in.size());
for (TDelete delete : in) {
out.add(deleteFromThrift(delete));
}
return out;
}
public static TDelete deleteFromHBase(Delete in) {
TDelete out = new TDelete(ByteBuffer.wrap(in.getRow()));
List<TColumn> columns = new ArrayList<TColumn>();
long rowTimestamp = in.getTimeStamp();
if (rowTimestamp != HConstants.LATEST_TIMESTAMP) {
out.setTimestamp(rowTimestamp);
}
// Map<family, List<KeyValue>>
for (Map.Entry<byte[], List<org.apache.hadoop.hbase.Cell>> familyEntry:
in.getFamilyCellMap().entrySet()) {
TColumn column = new TColumn(ByteBuffer.wrap(familyEntry.getKey()));
for (org.apache.hadoop.hbase.Cell cell: familyEntry.getValue()) {
byte[] family = CellUtil.cloneFamily(cell);
byte[] qualifier = CellUtil.cloneQualifier(cell);
long timestamp = cell.getTimestamp();
if (family != null) {
column.setFamily(family);
}
if (qualifier != null) {
column.setQualifier(qualifier);
}
if (timestamp != HConstants.LATEST_TIMESTAMP) {
column.setTimestamp(timestamp);
}
}
columns.add(column);
}
out.setColumns(columns);
return out;
}
/**
* Creates a {@link RowMutations} (HBase) from a {@link TRowMutations} (Thrift)
*
* @param in the <code>TRowMutations</code> to convert
*
* @return converted <code>RowMutations</code>
*/
public static RowMutations rowMutationsFromThrift(TRowMutations in) throws IOException {
RowMutations out = new RowMutations(in.getRow());
List<TMutation> mutations = in.getMutations();
for (TMutation mutation : mutations) {
if (mutation.isSetPut()) {
out.add(putFromThrift(mutation.getPut()));
}
if (mutation.isSetDeleteSingle()) {
out.add(deleteFromThrift(mutation.getDeleteSingle()));
}
}
return out;
}
public static Scan scanFromThrift(TScan in) throws IOException {
Scan out = new Scan();
if (in.isSetStartRow())
out.setStartRow(in.getStartRow());
if (in.isSetStopRow())
out.setStopRow(in.getStopRow());
if (in.isSetCaching())
out.setCaching(in.getCaching());
if (in.isSetMaxVersions()) {
out.setMaxVersions(in.getMaxVersions());
}
if (in.isSetColumns()) {
for (TColumn column : in.getColumns()) {
if (column.isSetQualifier()) {
out.addColumn(column.getFamily(), column.getQualifier());
} else {
out.addFamily(column.getFamily());
}
}
}
TTimeRange timeRange = in.getTimeRange();
if (timeRange != null &&
timeRange.isSetMinStamp() && timeRange.isSetMaxStamp()) {
out.setTimeRange(timeRange.getMinStamp(), timeRange.getMaxStamp());
}
if (in.isSetBatchSize()) {
out.setBatch(in.getBatchSize());
}
if (in.isSetFilterString()) {
ParseFilter parseFilter = new ParseFilter();
out.setFilter(parseFilter.parseFilterString(in.getFilterString()));
}
if (in.isSetAttributes()) {
addAttributes(out,in.getAttributes());
}
if (in.isSetAuthorizations()) {
out.setAuthorizations(new Authorizations(in.getAuthorizations().getLabels()));
}
if (in.isSetReversed()) {
out.setReversed(in.isReversed());
}
if (in.isSetCacheBlocks()) {
out.setCacheBlocks(in.isCacheBlocks());
}
return out;
}
public static Increment incrementFromThrift(TIncrement in) throws IOException {
Increment out = new Increment(in.getRow());
for (TColumnIncrement column : in.getColumns()) {
out.addColumn(column.getFamily(), column.getQualifier(), column.getAmount());
}
if (in.isSetAttributes()) {
addAttributes(out,in.getAttributes());
}
if (in.isSetDurability()) {
out.setDurability(durabilityFromThrift(in.getDurability()));
}
if(in.getCellVisibility() != null) {
out.setCellVisibility(new CellVisibility(in.getCellVisibility().getExpression()));
}
return out;
}
public static Append appendFromThrift(TAppend append) throws IOException {
Append out = new Append(append.getRow());
for (TColumnValue column : append.getColumns()) {
out.add(column.getFamily(), column.getQualifier(), column.getValue());
}
if (append.isSetAttributes()) {
addAttributes(out, append.getAttributes());
}
if (append.isSetDurability()) {
out.setDurability(durabilityFromThrift(append.getDurability()));
}
if(append.getCellVisibility() != null) {
out.setCellVisibility(new CellVisibility(append.getCellVisibility().getExpression()));
}
return out;
}
public static THRegionLocation regionLocationFromHBase(HRegionLocation hrl) {
HRegionInfo hri = hrl.getRegionInfo();
ServerName serverName = hrl.getServerName();
THRegionInfo thRegionInfo = new THRegionInfo();
THRegionLocation thRegionLocation = new THRegionLocation();
TServerName tServerName = new TServerName();
tServerName.setHostName(serverName.getHostname());
tServerName.setPort(serverName.getPort());
tServerName.setStartCode(serverName.getStartcode());
thRegionInfo.setTableName(hri.getTable().getName());
thRegionInfo.setEndKey(hri.getEndKey());
thRegionInfo.setStartKey(hri.getStartKey());
thRegionInfo.setOffline(hri.isOffline());
thRegionInfo.setSplit(hri.isSplit());
thRegionInfo.setReplicaId(hri.getReplicaId());
thRegionLocation.setRegionInfo(thRegionInfo);
thRegionLocation.setServerName(tServerName);
return thRegionLocation;
}
public static List<THRegionLocation> regionLocationsFromHBase(List<HRegionLocation> locations) {
List<THRegionLocation> tlocations = new ArrayList<THRegionLocation>(locations.size());
for (HRegionLocation hrl:locations) {
tlocations.add(regionLocationFromHBase(hrl));
}
return tlocations;
}
/**
* Adds all the attributes into the Operation object
*/
private static void addAttributes(OperationWithAttributes op,
Map<ByteBuffer, ByteBuffer> attributes) {
if (attributes == null || attributes.size() == 0) {
return;
}
for (Map.Entry<ByteBuffer, ByteBuffer> entry : attributes.entrySet()) {
String name = Bytes.toStringBinary(getBytes(entry.getKey()));
byte[] value = getBytes(entry.getValue());
op.setAttribute(name, value);
}
}
private static Durability durabilityFromThrift(TDurability tDurability) {
switch (tDurability.getValue()) {
case 1: return Durability.SKIP_WAL;
case 2: return Durability.ASYNC_WAL;
case 3: return Durability.SYNC_WAL;
case 4: return Durability.FSYNC_WAL;
default: return null;
}
}
public static CompareOp compareOpFromThrift(TCompareOp tCompareOp) {
switch (tCompareOp.getValue()) {
case 0: return CompareOp.LESS;
case 1: return CompareOp.LESS_OR_EQUAL;
case 2: return CompareOp.EQUAL;
case 3: return CompareOp.NOT_EQUAL;
case 4: return CompareOp.GREATER_OR_EQUAL;
case 5: return CompareOp.GREATER;
case 6: return CompareOp.NO_OP;
default: return null;
}
}
}

View File

@ -0,0 +1,954 @@
/**
* Autogenerated by Thrift Compiler (0.9.3)
*
* DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
* @generated
*/
package org.apache.hadoop.hbase.thrift2.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 javax.annotation.Generated;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2016-05-25")
public class TAppend implements org.apache.thrift.TBase<TAppend, TAppend._Fields>, java.io.Serializable, Cloneable, Comparable<TAppend> {
private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TAppend");
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 COLUMNS_FIELD_DESC = new org.apache.thrift.protocol.TField("columns", org.apache.thrift.protocol.TType.LIST, (short)2);
private static final org.apache.thrift.protocol.TField ATTRIBUTES_FIELD_DESC = new org.apache.thrift.protocol.TField("attributes", org.apache.thrift.protocol.TType.MAP, (short)3);
private static final org.apache.thrift.protocol.TField DURABILITY_FIELD_DESC = new org.apache.thrift.protocol.TField("durability", org.apache.thrift.protocol.TType.I32, (short)4);
private static final org.apache.thrift.protocol.TField CELL_VISIBILITY_FIELD_DESC = new org.apache.thrift.protocol.TField("cellVisibility", org.apache.thrift.protocol.TType.STRUCT, (short)5);
private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
static {
schemes.put(StandardScheme.class, new TAppendStandardSchemeFactory());
schemes.put(TupleScheme.class, new TAppendTupleSchemeFactory());
}
public ByteBuffer row; // required
public List<TColumnValue> columns; // required
public Map<ByteBuffer,ByteBuffer> attributes; // optional
/**
*
* @see TDurability
*/
public TDurability durability; // optional
public TCellVisibility cellVisibility; // 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 {
ROW((short)1, "row"),
COLUMNS((short)2, "columns"),
ATTRIBUTES((short)3, "attributes"),
/**
*
* @see TDurability
*/
DURABILITY((short)4, "durability"),
CELL_VISIBILITY((short)5, "cellVisibility");
private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
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: // COLUMNS
return COLUMNS;
case 3: // ATTRIBUTES
return ATTRIBUTES;
case 4: // DURABILITY
return DURABILITY;
case 5: // CELL_VISIBILITY
return CELL_VISIBILITY;
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 _Fields optionals[] = {_Fields.ATTRIBUTES,_Fields.DURABILITY,_Fields.CELL_VISIBILITY};
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.COLUMNS, new org.apache.thrift.meta_data.FieldMetaData("columns", 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))));
tmpMap.put(_Fields.ATTRIBUTES, new org.apache.thrift.meta_data.FieldMetaData("attributes", org.apache.thrift.TFieldRequirementType.OPTIONAL,
new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP,
new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , true),
new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , true))));
tmpMap.put(_Fields.DURABILITY, new org.apache.thrift.meta_data.FieldMetaData("durability", org.apache.thrift.TFieldRequirementType.OPTIONAL,
new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, TDurability.class)));
tmpMap.put(_Fields.CELL_VISIBILITY, new org.apache.thrift.meta_data.FieldMetaData("cellVisibility", org.apache.thrift.TFieldRequirementType.OPTIONAL,
new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TCellVisibility.class)));
metaDataMap = Collections.unmodifiableMap(tmpMap);
org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TAppend.class, metaDataMap);
}
public TAppend() {
}
public TAppend(
ByteBuffer row,
List<TColumnValue> columns)
{
this();
this.row = org.apache.thrift.TBaseHelper.copyBinary(row);
this.columns = columns;
}
/**
* Performs a deep copy on <i>other</i>.
*/
public TAppend(TAppend other) {
if (other.isSetRow()) {
this.row = org.apache.thrift.TBaseHelper.copyBinary(other.row);
}
if (other.isSetColumns()) {
List<TColumnValue> __this__columns = new ArrayList<TColumnValue>(other.columns.size());
for (TColumnValue other_element : other.columns) {
__this__columns.add(new TColumnValue(other_element));
}
this.columns = __this__columns;
}
if (other.isSetAttributes()) {
Map<ByteBuffer,ByteBuffer> __this__attributes = new HashMap<ByteBuffer,ByteBuffer>(other.attributes);
this.attributes = __this__attributes;
}
if (other.isSetDurability()) {
this.durability = other.durability;
}
if (other.isSetCellVisibility()) {
this.cellVisibility = new TCellVisibility(other.cellVisibility);
}
}
public TAppend deepCopy() {
return new TAppend(this);
}
@Override
public void clear() {
this.row = null;
this.columns = null;
this.attributes = null;
this.durability = null;
this.cellVisibility = null;
}
public byte[] getRow() {
setRow(org.apache.thrift.TBaseHelper.rightSize(row));
return row == null ? null : row.array();
}
public ByteBuffer bufferForRow() {
return org.apache.thrift.TBaseHelper.copyBinary(row);
}
public TAppend setRow(byte[] row) {
this.row = row == null ? (ByteBuffer)null : ByteBuffer.wrap(Arrays.copyOf(row, row.length));
return this;
}
public TAppend setRow(ByteBuffer row) {
this.row = org.apache.thrift.TBaseHelper.copyBinary(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 int getColumnsSize() {
return (this.columns == null) ? 0 : this.columns.size();
}
public java.util.Iterator<TColumnValue> getColumnsIterator() {
return (this.columns == null) ? null : this.columns.iterator();
}
public void addToColumns(TColumnValue elem) {
if (this.columns == null) {
this.columns = new ArrayList<TColumnValue>();
}
this.columns.add(elem);
}
public List<TColumnValue> getColumns() {
return this.columns;
}
public TAppend setColumns(List<TColumnValue> columns) {
this.columns = columns;
return this;
}
public void unsetColumns() {
this.columns = null;
}
/** Returns true if field columns is set (has been assigned a value) and false otherwise */
public boolean isSetColumns() {
return this.columns != null;
}
public void setColumnsIsSet(boolean value) {
if (!value) {
this.columns = null;
}
}
public int getAttributesSize() {
return (this.attributes == null) ? 0 : this.attributes.size();
}
public void putToAttributes(ByteBuffer key, ByteBuffer val) {
if (this.attributes == null) {
this.attributes = new HashMap<ByteBuffer,ByteBuffer>();
}
this.attributes.put(key, val);
}
public Map<ByteBuffer,ByteBuffer> getAttributes() {
return this.attributes;
}
public TAppend setAttributes(Map<ByteBuffer,ByteBuffer> attributes) {
this.attributes = attributes;
return this;
}
public void unsetAttributes() {
this.attributes = null;
}
/** Returns true if field attributes is set (has been assigned a value) and false otherwise */
public boolean isSetAttributes() {
return this.attributes != null;
}
public void setAttributesIsSet(boolean value) {
if (!value) {
this.attributes = null;
}
}
/**
*
* @see TDurability
*/
public TDurability getDurability() {
return this.durability;
}
/**
*
* @see TDurability
*/
public TAppend setDurability(TDurability durability) {
this.durability = durability;
return this;
}
public void unsetDurability() {
this.durability = null;
}
/** Returns true if field durability is set (has been assigned a value) and false otherwise */
public boolean isSetDurability() {
return this.durability != null;
}
public void setDurabilityIsSet(boolean value) {
if (!value) {
this.durability = null;
}
}
public TCellVisibility getCellVisibility() {
return this.cellVisibility;
}
public TAppend setCellVisibility(TCellVisibility cellVisibility) {
this.cellVisibility = cellVisibility;
return this;
}
public void unsetCellVisibility() {
this.cellVisibility = null;
}
/** Returns true if field cellVisibility is set (has been assigned a value) and false otherwise */
public boolean isSetCellVisibility() {
return this.cellVisibility != null;
}
public void setCellVisibilityIsSet(boolean value) {
if (!value) {
this.cellVisibility = null;
}
}
public void setFieldValue(_Fields field, Object value) {
switch (field) {
case ROW:
if (value == null) {
unsetRow();
} else {
setRow((ByteBuffer)value);
}
break;
case COLUMNS:
if (value == null) {
unsetColumns();
} else {
setColumns((List<TColumnValue>)value);
}
break;
case ATTRIBUTES:
if (value == null) {
unsetAttributes();
} else {
setAttributes((Map<ByteBuffer,ByteBuffer>)value);
}
break;
case DURABILITY:
if (value == null) {
unsetDurability();
} else {
setDurability((TDurability)value);
}
break;
case CELL_VISIBILITY:
if (value == null) {
unsetCellVisibility();
} else {
setCellVisibility((TCellVisibility)value);
}
break;
}
}
public Object getFieldValue(_Fields field) {
switch (field) {
case ROW:
return getRow();
case COLUMNS:
return getColumns();
case ATTRIBUTES:
return getAttributes();
case DURABILITY:
return getDurability();
case CELL_VISIBILITY:
return getCellVisibility();
}
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 COLUMNS:
return isSetColumns();
case ATTRIBUTES:
return isSetAttributes();
case DURABILITY:
return isSetDurability();
case CELL_VISIBILITY:
return isSetCellVisibility();
}
throw new IllegalStateException();
}
@Override
public boolean equals(Object that) {
if (that == null)
return false;
if (that instanceof TAppend)
return this.equals((TAppend)that);
return false;
}
public boolean equals(TAppend 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_columns = true && this.isSetColumns();
boolean that_present_columns = true && that.isSetColumns();
if (this_present_columns || that_present_columns) {
if (!(this_present_columns && that_present_columns))
return false;
if (!this.columns.equals(that.columns))
return false;
}
boolean this_present_attributes = true && this.isSetAttributes();
boolean that_present_attributes = true && that.isSetAttributes();
if (this_present_attributes || that_present_attributes) {
if (!(this_present_attributes && that_present_attributes))
return false;
if (!this.attributes.equals(that.attributes))
return false;
}
boolean this_present_durability = true && this.isSetDurability();
boolean that_present_durability = true && that.isSetDurability();
if (this_present_durability || that_present_durability) {
if (!(this_present_durability && that_present_durability))
return false;
if (!this.durability.equals(that.durability))
return false;
}
boolean this_present_cellVisibility = true && this.isSetCellVisibility();
boolean that_present_cellVisibility = true && that.isSetCellVisibility();
if (this_present_cellVisibility || that_present_cellVisibility) {
if (!(this_present_cellVisibility && that_present_cellVisibility))
return false;
if (!this.cellVisibility.equals(that.cellVisibility))
return false;
}
return true;
}
@Override
public int hashCode() {
List<Object> list = new ArrayList<Object>();
boolean present_row = true && (isSetRow());
list.add(present_row);
if (present_row)
list.add(row);
boolean present_columns = true && (isSetColumns());
list.add(present_columns);
if (present_columns)
list.add(columns);
boolean present_attributes = true && (isSetAttributes());
list.add(present_attributes);
if (present_attributes)
list.add(attributes);
boolean present_durability = true && (isSetDurability());
list.add(present_durability);
if (present_durability)
list.add(durability.getValue());
boolean present_cellVisibility = true && (isSetCellVisibility());
list.add(present_cellVisibility);
if (present_cellVisibility)
list.add(cellVisibility);
return list.hashCode();
}
@Override
public int compareTo(TAppend 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(isSetColumns()).compareTo(other.isSetColumns());
if (lastComparison != 0) {
return lastComparison;
}
if (isSetColumns()) {
lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.columns, other.columns);
if (lastComparison != 0) {
return lastComparison;
}
}
lastComparison = Boolean.valueOf(isSetAttributes()).compareTo(other.isSetAttributes());
if (lastComparison != 0) {
return lastComparison;
}
if (isSetAttributes()) {
lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.attributes, other.attributes);
if (lastComparison != 0) {
return lastComparison;
}
}
lastComparison = Boolean.valueOf(isSetDurability()).compareTo(other.isSetDurability());
if (lastComparison != 0) {
return lastComparison;
}
if (isSetDurability()) {
lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.durability, other.durability);
if (lastComparison != 0) {
return lastComparison;
}
}
lastComparison = Boolean.valueOf(isSetCellVisibility()).compareTo(other.isSetCellVisibility());
if (lastComparison != 0) {
return lastComparison;
}
if (isSetCellVisibility()) {
lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.cellVisibility, other.cellVisibility);
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("TAppend(");
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("columns:");
if (this.columns == null) {
sb.append("null");
} else {
sb.append(this.columns);
}
first = false;
if (isSetAttributes()) {
if (!first) sb.append(", ");
sb.append("attributes:");
if (this.attributes == null) {
sb.append("null");
} else {
sb.append(this.attributes);
}
first = false;
}
if (isSetDurability()) {
if (!first) sb.append(", ");
sb.append("durability:");
if (this.durability == null) {
sb.append("null");
} else {
sb.append(this.durability);
}
first = false;
}
if (isSetCellVisibility()) {
if (!first) sb.append(", ");
sb.append("cellVisibility:");
if (this.cellVisibility == null) {
sb.append("null");
} else {
sb.append(this.cellVisibility);
}
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 (columns == null) {
throw new org.apache.thrift.protocol.TProtocolException("Required field 'columns' was not present! Struct: " + toString());
}
// check for sub-struct validity
if (cellVisibility != null) {
cellVisibility.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 TAppendStandardSchemeFactory implements SchemeFactory {
public TAppendStandardScheme getScheme() {
return new TAppendStandardScheme();
}
}
private static class TAppendStandardScheme extends StandardScheme<TAppend> {
public void read(org.apache.thrift.protocol.TProtocol iprot, TAppend 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: // COLUMNS
if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
{
org.apache.thrift.protocol.TList _list88 = iprot.readListBegin();
struct.columns = new ArrayList<TColumnValue>(_list88.size);
TColumnValue _elem89;
for (int _i90 = 0; _i90 < _list88.size; ++_i90)
{
_elem89 = new TColumnValue();
_elem89.read(iprot);
struct.columns.add(_elem89);
}
iprot.readListEnd();
}
struct.setColumnsIsSet(true);
} else {
org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
}
break;
case 3: // ATTRIBUTES
if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
{
org.apache.thrift.protocol.TMap _map91 = iprot.readMapBegin();
struct.attributes = new HashMap<ByteBuffer,ByteBuffer>(2*_map91.size);
ByteBuffer _key92;
ByteBuffer _val93;
for (int _i94 = 0; _i94 < _map91.size; ++_i94)
{
_key92 = iprot.readBinary();
_val93 = iprot.readBinary();
struct.attributes.put(_key92, _val93);
}
iprot.readMapEnd();
}
struct.setAttributesIsSet(true);
} else {
org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
}
break;
case 4: // DURABILITY
if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
struct.durability = org.apache.hadoop.hbase.thrift2.generated.TDurability.findByValue(iprot.readI32());
struct.setDurabilityIsSet(true);
} else {
org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
}
break;
case 5: // CELL_VISIBILITY
if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
struct.cellVisibility = new TCellVisibility();
struct.cellVisibility.read(iprot);
struct.setCellVisibilityIsSet(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, TAppend 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.columns != null) {
oprot.writeFieldBegin(COLUMNS_FIELD_DESC);
{
oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.columns.size()));
for (TColumnValue _iter95 : struct.columns)
{
_iter95.write(oprot);
}
oprot.writeListEnd();
}
oprot.writeFieldEnd();
}
if (struct.attributes != null) {
if (struct.isSetAttributes()) {
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<ByteBuffer, ByteBuffer> _iter96 : struct.attributes.entrySet())
{
oprot.writeBinary(_iter96.getKey());
oprot.writeBinary(_iter96.getValue());
}
oprot.writeMapEnd();
}
oprot.writeFieldEnd();
}
}
if (struct.durability != null) {
if (struct.isSetDurability()) {
oprot.writeFieldBegin(DURABILITY_FIELD_DESC);
oprot.writeI32(struct.durability.getValue());
oprot.writeFieldEnd();
}
}
if (struct.cellVisibility != null) {
if (struct.isSetCellVisibility()) {
oprot.writeFieldBegin(CELL_VISIBILITY_FIELD_DESC);
struct.cellVisibility.write(oprot);
oprot.writeFieldEnd();
}
}
oprot.writeFieldStop();
oprot.writeStructEnd();
}
}
private static class TAppendTupleSchemeFactory implements SchemeFactory {
public TAppendTupleScheme getScheme() {
return new TAppendTupleScheme();
}
}
private static class TAppendTupleScheme extends TupleScheme<TAppend> {
@Override
public void write(org.apache.thrift.protocol.TProtocol prot, TAppend struct) throws org.apache.thrift.TException {
TTupleProtocol oprot = (TTupleProtocol) prot;
oprot.writeBinary(struct.row);
{
oprot.writeI32(struct.columns.size());
for (TColumnValue _iter97 : struct.columns)
{
_iter97.write(oprot);
}
}
BitSet optionals = new BitSet();
if (struct.isSetAttributes()) {
optionals.set(0);
}
if (struct.isSetDurability()) {
optionals.set(1);
}
if (struct.isSetCellVisibility()) {
optionals.set(2);
}
oprot.writeBitSet(optionals, 3);
if (struct.isSetAttributes()) {
{
oprot.writeI32(struct.attributes.size());
for (Map.Entry<ByteBuffer, ByteBuffer> _iter98 : struct.attributes.entrySet())
{
oprot.writeBinary(_iter98.getKey());
oprot.writeBinary(_iter98.getValue());
}
}
}
if (struct.isSetDurability()) {
oprot.writeI32(struct.durability.getValue());
}
if (struct.isSetCellVisibility()) {
struct.cellVisibility.write(oprot);
}
}
@Override
public void read(org.apache.thrift.protocol.TProtocol prot, TAppend struct) throws org.apache.thrift.TException {
TTupleProtocol iprot = (TTupleProtocol) prot;
struct.row = iprot.readBinary();
struct.setRowIsSet(true);
{
org.apache.thrift.protocol.TList _list99 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
struct.columns = new ArrayList<TColumnValue>(_list99.size);
TColumnValue _elem100;
for (int _i101 = 0; _i101 < _list99.size; ++_i101)
{
_elem100 = new TColumnValue();
_elem100.read(iprot);
struct.columns.add(_elem100);
}
}
struct.setColumnsIsSet(true);
BitSet incoming = iprot.readBitSet(3);
if (incoming.get(0)) {
{
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<ByteBuffer,ByteBuffer>(2*_map102.size);
ByteBuffer _key103;
ByteBuffer _val104;
for (int _i105 = 0; _i105 < _map102.size; ++_i105)
{
_key103 = iprot.readBinary();
_val104 = iprot.readBinary();
struct.attributes.put(_key103, _val104);
}
}
struct.setAttributesIsSet(true);
}
if (incoming.get(1)) {
struct.durability = org.apache.hadoop.hbase.thrift2.generated.TDurability.findByValue(iprot.readI32());
struct.setDurabilityIsSet(true);
}
if (incoming.get(2)) {
struct.cellVisibility = new TCellVisibility();
struct.cellVisibility.read(iprot);
struct.setCellVisibilityIsSet(true);
}
}
}
}

View File

@ -0,0 +1,445 @@
/**
* Autogenerated by Thrift Compiler (0.9.3)
*
* DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
* @generated
*/
package org.apache.hadoop.hbase.thrift2.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 javax.annotation.Generated;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2016-05-25")
public class TAuthorization implements org.apache.thrift.TBase<TAuthorization, TAuthorization._Fields>, java.io.Serializable, Cloneable, Comparable<TAuthorization> {
private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TAuthorization");
private static final org.apache.thrift.protocol.TField LABELS_FIELD_DESC = new org.apache.thrift.protocol.TField("labels", org.apache.thrift.protocol.TType.LIST, (short)1);
private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
static {
schemes.put(StandardScheme.class, new TAuthorizationStandardSchemeFactory());
schemes.put(TupleScheme.class, new TAuthorizationTupleSchemeFactory());
}
public List<String> labels; // 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 {
LABELS((short)1, "labels");
private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
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: // LABELS
return LABELS;
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 _Fields optionals[] = {_Fields.LABELS};
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.LABELS, new org.apache.thrift.meta_data.FieldMetaData("labels", org.apache.thrift.TFieldRequirementType.OPTIONAL,
new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST,
new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))));
metaDataMap = Collections.unmodifiableMap(tmpMap);
org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TAuthorization.class, metaDataMap);
}
public TAuthorization() {
}
/**
* Performs a deep copy on <i>other</i>.
*/
public TAuthorization(TAuthorization other) {
if (other.isSetLabels()) {
List<String> __this__labels = new ArrayList<String>(other.labels);
this.labels = __this__labels;
}
}
public TAuthorization deepCopy() {
return new TAuthorization(this);
}
@Override
public void clear() {
this.labels = null;
}
public int getLabelsSize() {
return (this.labels == null) ? 0 : this.labels.size();
}
public java.util.Iterator<String> getLabelsIterator() {
return (this.labels == null) ? null : this.labels.iterator();
}
public void addToLabels(String elem) {
if (this.labels == null) {
this.labels = new ArrayList<String>();
}
this.labels.add(elem);
}
public List<String> getLabels() {
return this.labels;
}
public TAuthorization setLabels(List<String> labels) {
this.labels = labels;
return this;
}
public void unsetLabels() {
this.labels = null;
}
/** Returns true if field labels is set (has been assigned a value) and false otherwise */
public boolean isSetLabels() {
return this.labels != null;
}
public void setLabelsIsSet(boolean value) {
if (!value) {
this.labels = null;
}
}
public void setFieldValue(_Fields field, Object value) {
switch (field) {
case LABELS:
if (value == null) {
unsetLabels();
} else {
setLabels((List<String>)value);
}
break;
}
}
public Object getFieldValue(_Fields field) {
switch (field) {
case LABELS:
return getLabels();
}
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 LABELS:
return isSetLabels();
}
throw new IllegalStateException();
}
@Override
public boolean equals(Object that) {
if (that == null)
return false;
if (that instanceof TAuthorization)
return this.equals((TAuthorization)that);
return false;
}
public boolean equals(TAuthorization that) {
if (that == null)
return false;
boolean this_present_labels = true && this.isSetLabels();
boolean that_present_labels = true && that.isSetLabels();
if (this_present_labels || that_present_labels) {
if (!(this_present_labels && that_present_labels))
return false;
if (!this.labels.equals(that.labels))
return false;
}
return true;
}
@Override
public int hashCode() {
List<Object> list = new ArrayList<Object>();
boolean present_labels = true && (isSetLabels());
list.add(present_labels);
if (present_labels)
list.add(labels);
return list.hashCode();
}
@Override
public int compareTo(TAuthorization other) {
if (!getClass().equals(other.getClass())) {
return getClass().getName().compareTo(other.getClass().getName());
}
int lastComparison = 0;
lastComparison = Boolean.valueOf(isSetLabels()).compareTo(other.isSetLabels());
if (lastComparison != 0) {
return lastComparison;
}
if (isSetLabels()) {
lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.labels, other.labels);
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("TAuthorization(");
boolean first = true;
if (isSetLabels()) {
sb.append("labels:");
if (this.labels == null) {
sb.append("null");
} else {
sb.append(this.labels);
}
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 TAuthorizationStandardSchemeFactory implements SchemeFactory {
public TAuthorizationStandardScheme getScheme() {
return new TAuthorizationStandardScheme();
}
}
private static class TAuthorizationStandardScheme extends StandardScheme<TAuthorization> {
public void read(org.apache.thrift.protocol.TProtocol iprot, TAuthorization 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: // LABELS
if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
{
org.apache.thrift.protocol.TList _list8 = iprot.readListBegin();
struct.labels = new ArrayList<String>(_list8.size);
String _elem9;
for (int _i10 = 0; _i10 < _list8.size; ++_i10)
{
_elem9 = iprot.readString();
struct.labels.add(_elem9);
}
iprot.readListEnd();
}
struct.setLabelsIsSet(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, TAuthorization struct) throws org.apache.thrift.TException {
struct.validate();
oprot.writeStructBegin(STRUCT_DESC);
if (struct.labels != null) {
if (struct.isSetLabels()) {
oprot.writeFieldBegin(LABELS_FIELD_DESC);
{
oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.labels.size()));
for (String _iter11 : struct.labels)
{
oprot.writeString(_iter11);
}
oprot.writeListEnd();
}
oprot.writeFieldEnd();
}
}
oprot.writeFieldStop();
oprot.writeStructEnd();
}
}
private static class TAuthorizationTupleSchemeFactory implements SchemeFactory {
public TAuthorizationTupleScheme getScheme() {
return new TAuthorizationTupleScheme();
}
}
private static class TAuthorizationTupleScheme extends TupleScheme<TAuthorization> {
@Override
public void write(org.apache.thrift.protocol.TProtocol prot, TAuthorization struct) throws org.apache.thrift.TException {
TTupleProtocol oprot = (TTupleProtocol) prot;
BitSet optionals = new BitSet();
if (struct.isSetLabels()) {
optionals.set(0);
}
oprot.writeBitSet(optionals, 1);
if (struct.isSetLabels()) {
{
oprot.writeI32(struct.labels.size());
for (String _iter12 : struct.labels)
{
oprot.writeString(_iter12);
}
}
}
}
@Override
public void read(org.apache.thrift.protocol.TProtocol prot, TAuthorization struct) throws org.apache.thrift.TException {
TTupleProtocol iprot = (TTupleProtocol) prot;
BitSet incoming = iprot.readBitSet(1);
if (incoming.get(0)) {
{
org.apache.thrift.protocol.TList _list13 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
struct.labels = new ArrayList<String>(_list13.size);
String _elem14;
for (int _i15 = 0; _i15 < _list13.size; ++_i15)
{
_elem14 = iprot.readString();
struct.labels.add(_elem14);
}
}
struct.setLabelsIsSet(true);
}
}
}
}

View File

@ -0,0 +1,396 @@
/**
* Autogenerated by Thrift Compiler (0.9.3)
*
* DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
* @generated
*/
package org.apache.hadoop.hbase.thrift2.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 javax.annotation.Generated;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2016-05-25")
public class TCellVisibility implements org.apache.thrift.TBase<TCellVisibility, TCellVisibility._Fields>, java.io.Serializable, Cloneable, Comparable<TCellVisibility> {
private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TCellVisibility");
private static final org.apache.thrift.protocol.TField EXPRESSION_FIELD_DESC = new org.apache.thrift.protocol.TField("expression", org.apache.thrift.protocol.TType.STRING, (short)1);
private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
static {
schemes.put(StandardScheme.class, new TCellVisibilityStandardSchemeFactory());
schemes.put(TupleScheme.class, new TCellVisibilityTupleSchemeFactory());
}
public String expression; // 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 {
EXPRESSION((short)1, "expression");
private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
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: // EXPRESSION
return EXPRESSION;
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 _Fields optionals[] = {_Fields.EXPRESSION};
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.EXPRESSION, new org.apache.thrift.meta_data.FieldMetaData("expression", 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(TCellVisibility.class, metaDataMap);
}
public TCellVisibility() {
}
/**
* Performs a deep copy on <i>other</i>.
*/
public TCellVisibility(TCellVisibility other) {
if (other.isSetExpression()) {
this.expression = other.expression;
}
}
public TCellVisibility deepCopy() {
return new TCellVisibility(this);
}
@Override
public void clear() {
this.expression = null;
}
public String getExpression() {
return this.expression;
}
public TCellVisibility setExpression(String expression) {
this.expression = expression;
return this;
}
public void unsetExpression() {
this.expression = null;
}
/** Returns true if field expression is set (has been assigned a value) and false otherwise */
public boolean isSetExpression() {
return this.expression != null;
}
public void setExpressionIsSet(boolean value) {
if (!value) {
this.expression = null;
}
}
public void setFieldValue(_Fields field, Object value) {
switch (field) {
case EXPRESSION:
if (value == null) {
unsetExpression();
} else {
setExpression((String)value);
}
break;
}
}
public Object getFieldValue(_Fields field) {
switch (field) {
case EXPRESSION:
return getExpression();
}
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 EXPRESSION:
return isSetExpression();
}
throw new IllegalStateException();
}
@Override
public boolean equals(Object that) {
if (that == null)
return false;
if (that instanceof TCellVisibility)
return this.equals((TCellVisibility)that);
return false;
}
public boolean equals(TCellVisibility that) {
if (that == null)
return false;
boolean this_present_expression = true && this.isSetExpression();
boolean that_present_expression = true && that.isSetExpression();
if (this_present_expression || that_present_expression) {
if (!(this_present_expression && that_present_expression))
return false;
if (!this.expression.equals(that.expression))
return false;
}
return true;
}
@Override
public int hashCode() {
List<Object> list = new ArrayList<Object>();
boolean present_expression = true && (isSetExpression());
list.add(present_expression);
if (present_expression)
list.add(expression);
return list.hashCode();
}
@Override
public int compareTo(TCellVisibility other) {
if (!getClass().equals(other.getClass())) {
return getClass().getName().compareTo(other.getClass().getName());
}
int lastComparison = 0;
lastComparison = Boolean.valueOf(isSetExpression()).compareTo(other.isSetExpression());
if (lastComparison != 0) {
return lastComparison;
}
if (isSetExpression()) {
lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.expression, other.expression);
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("TCellVisibility(");
boolean first = true;
if (isSetExpression()) {
sb.append("expression:");
if (this.expression == null) {
sb.append("null");
} else {
sb.append(this.expression);
}
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 TCellVisibilityStandardSchemeFactory implements SchemeFactory {
public TCellVisibilityStandardScheme getScheme() {
return new TCellVisibilityStandardScheme();
}
}
private static class TCellVisibilityStandardScheme extends StandardScheme<TCellVisibility> {
public void read(org.apache.thrift.protocol.TProtocol iprot, TCellVisibility 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: // EXPRESSION
if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
struct.expression = iprot.readString();
struct.setExpressionIsSet(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, TCellVisibility struct) throws org.apache.thrift.TException {
struct.validate();
oprot.writeStructBegin(STRUCT_DESC);
if (struct.expression != null) {
if (struct.isSetExpression()) {
oprot.writeFieldBegin(EXPRESSION_FIELD_DESC);
oprot.writeString(struct.expression);
oprot.writeFieldEnd();
}
}
oprot.writeFieldStop();
oprot.writeStructEnd();
}
}
private static class TCellVisibilityTupleSchemeFactory implements SchemeFactory {
public TCellVisibilityTupleScheme getScheme() {
return new TCellVisibilityTupleScheme();
}
}
private static class TCellVisibilityTupleScheme extends TupleScheme<TCellVisibility> {
@Override
public void write(org.apache.thrift.protocol.TProtocol prot, TCellVisibility struct) throws org.apache.thrift.TException {
TTupleProtocol oprot = (TTupleProtocol) prot;
BitSet optionals = new BitSet();
if (struct.isSetExpression()) {
optionals.set(0);
}
oprot.writeBitSet(optionals, 1);
if (struct.isSetExpression()) {
oprot.writeString(struct.expression);
}
}
@Override
public void read(org.apache.thrift.protocol.TProtocol prot, TCellVisibility struct) throws org.apache.thrift.TException {
TTupleProtocol iprot = (TTupleProtocol) prot;
BitSet incoming = iprot.readBitSet(1);
if (incoming.get(0)) {
struct.expression = iprot.readString();
struct.setExpressionIsSet(true);
}
}
}
}

View File

@ -0,0 +1,631 @@
/**
* Autogenerated by Thrift Compiler (0.9.3)
*
* DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
* @generated
*/
package org.apache.hadoop.hbase.thrift2.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 javax.annotation.Generated;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
/**
* Addresses a single cell or multiple cells
* in a HBase table by column family and optionally
* a column qualifier and timestamp
*/
@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2016-05-25")
public class TColumn implements org.apache.thrift.TBase<TColumn, TColumn._Fields>, java.io.Serializable, Cloneable, Comparable<TColumn> {
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);
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)2);
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)3);
private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
static {
schemes.put(StandardScheme.class, new TColumnStandardSchemeFactory());
schemes.put(TupleScheme.class, new TColumnTupleSchemeFactory());
}
public ByteBuffer family; // required
public ByteBuffer qualifier; // optional
public long timestamp; // 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 {
FAMILY((short)1, "family"),
QUALIFIER((short)2, "qualifier"),
TIMESTAMP((short)3, "timestamp");
private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
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: // FAMILY
return FAMILY;
case 2: // QUALIFIER
return QUALIFIER;
case 3: // TIMESTAMP
return TIMESTAMP;
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 byte __isset_bitfield = 0;
private static final _Fields optionals[] = {_Fields.QUALIFIER,_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.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.OPTIONAL,
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)));
metaDataMap = Collections.unmodifiableMap(tmpMap);
org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TColumn.class, metaDataMap);
}
public TColumn() {
}
public TColumn(
ByteBuffer family)
{
this();
this.family = org.apache.thrift.TBaseHelper.copyBinary(family);
}
/**
* Performs a deep copy on <i>other</i>.
*/
public TColumn(TColumn other) {
__isset_bitfield = other.__isset_bitfield;
if (other.isSetFamily()) {
this.family = org.apache.thrift.TBaseHelper.copyBinary(other.family);
}
if (other.isSetQualifier()) {
this.qualifier = org.apache.thrift.TBaseHelper.copyBinary(other.qualifier);
}
this.timestamp = other.timestamp;
}
public TColumn deepCopy() {
return new TColumn(this);
}
@Override
public void clear() {
this.family = null;
this.qualifier = null;
setTimestampIsSet(false);
this.timestamp = 0;
}
public byte[] getFamily() {
setFamily(org.apache.thrift.TBaseHelper.rightSize(family));
return family == null ? null : family.array();
}
public ByteBuffer bufferForFamily() {
return org.apache.thrift.TBaseHelper.copyBinary(family);
}
public TColumn setFamily(byte[] family) {
this.family = family == null ? (ByteBuffer)null : ByteBuffer.wrap(Arrays.copyOf(family, family.length));
return this;
}
public TColumn setFamily(ByteBuffer family) {
this.family = org.apache.thrift.TBaseHelper.copyBinary(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 org.apache.thrift.TBaseHelper.copyBinary(qualifier);
}
public TColumn setQualifier(byte[] qualifier) {
this.qualifier = qualifier == null ? (ByteBuffer)null : ByteBuffer.wrap(Arrays.copyOf(qualifier, qualifier.length));
return this;
}
public TColumn setQualifier(ByteBuffer qualifier) {
this.qualifier = org.apache.thrift.TBaseHelper.copyBinary(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 long getTimestamp() {
return this.timestamp;
}
public TColumn 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 void setFieldValue(_Fields field, Object value) {
switch (field) {
case FAMILY:
if (value == null) {
unsetFamily();
} else {
setFamily((ByteBuffer)value);
}
break;
case QUALIFIER:
if (value == null) {
unsetQualifier();
} else {
setQualifier((ByteBuffer)value);
}
break;
case TIMESTAMP:
if (value == null) {
unsetTimestamp();
} else {
setTimestamp((Long)value);
}
break;
}
}
public Object getFieldValue(_Fields field) {
switch (field) {
case FAMILY:
return getFamily();
case QUALIFIER:
return getQualifier();
case TIMESTAMP:
return getTimestamp();
}
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 FAMILY:
return isSetFamily();
case QUALIFIER:
return isSetQualifier();
case TIMESTAMP:
return isSetTimestamp();
}
throw new IllegalStateException();
}
@Override
public boolean equals(Object that) {
if (that == null)
return false;
if (that instanceof TColumn)
return this.equals((TColumn)that);
return false;
}
public boolean equals(TColumn that) {
if (that == null)
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_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;
}
return true;
}
@Override
public int hashCode() {
List<Object> list = new ArrayList<Object>();
boolean present_family = true && (isSetFamily());
list.add(present_family);
if (present_family)
list.add(family);
boolean present_qualifier = true && (isSetQualifier());
list.add(present_qualifier);
if (present_qualifier)
list.add(qualifier);
boolean present_timestamp = true && (isSetTimestamp());
list.add(present_timestamp);
if (present_timestamp)
list.add(timestamp);
return list.hashCode();
}
@Override
public int compareTo(TColumn other) {
if (!getClass().equals(other.getClass())) {
return getClass().getName().compareTo(other.getClass().getName());
}
int lastComparison = 0;
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(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;
}
}
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("TColumn(");
boolean first = true;
sb.append("family:");
if (this.family == null) {
sb.append("null");
} else {
org.apache.thrift.TBaseHelper.toString(this.family, sb);
}
first = false;
if (isSetQualifier()) {
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 (isSetTimestamp()) {
if (!first) sb.append(", ");
sb.append("timestamp:");
sb.append(this.timestamp);
first = false;
}
sb.append(")");
return sb.toString();
}
public void validate() throws org.apache.thrift.TException {
// check for required fields
if (family == null) {
throw new org.apache.thrift.protocol.TProtocolException("Required field 'family' 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 {
// 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 TColumnStandardSchemeFactory implements SchemeFactory {
public TColumnStandardScheme getScheme() {
return new TColumnStandardScheme();
}
}
private static class TColumnStandardScheme extends StandardScheme<TColumn> {
public void read(org.apache.thrift.protocol.TProtocol iprot, TColumn 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: // 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 2: // 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 3: // 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;
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, TColumn struct) throws org.apache.thrift.TException {
struct.validate();
oprot.writeStructBegin(STRUCT_DESC);
if (struct.family != null) {
oprot.writeFieldBegin(FAMILY_FIELD_DESC);
oprot.writeBinary(struct.family);
oprot.writeFieldEnd();
}
if (struct.qualifier != null) {
if (struct.isSetQualifier()) {
oprot.writeFieldBegin(QUALIFIER_FIELD_DESC);
oprot.writeBinary(struct.qualifier);
oprot.writeFieldEnd();
}
}
if (struct.isSetTimestamp()) {
oprot.writeFieldBegin(TIMESTAMP_FIELD_DESC);
oprot.writeI64(struct.timestamp);
oprot.writeFieldEnd();
}
oprot.writeFieldStop();
oprot.writeStructEnd();
}
}
private static class TColumnTupleSchemeFactory implements SchemeFactory {
public TColumnTupleScheme getScheme() {
return new TColumnTupleScheme();
}
}
private static class TColumnTupleScheme extends TupleScheme<TColumn> {
@Override
public void write(org.apache.thrift.protocol.TProtocol prot, TColumn struct) throws org.apache.thrift.TException {
TTupleProtocol oprot = (TTupleProtocol) prot;
oprot.writeBinary(struct.family);
BitSet optionals = new BitSet();
if (struct.isSetQualifier()) {
optionals.set(0);
}
if (struct.isSetTimestamp()) {
optionals.set(1);
}
oprot.writeBitSet(optionals, 2);
if (struct.isSetQualifier()) {
oprot.writeBinary(struct.qualifier);
}
if (struct.isSetTimestamp()) {
oprot.writeI64(struct.timestamp);
}
}
@Override
public void read(org.apache.thrift.protocol.TProtocol prot, TColumn struct) throws org.apache.thrift.TException {
TTupleProtocol iprot = (TTupleProtocol) prot;
struct.family = iprot.readBinary();
struct.setFamilyIsSet(true);
BitSet incoming = iprot.readBitSet(2);
if (incoming.get(0)) {
struct.qualifier = iprot.readBinary();
struct.setQualifierIsSet(true);
}
if (incoming.get(1)) {
struct.timestamp = iprot.readI64();
struct.setTimestampIsSet(true);
}
}
}
}

View File

@ -0,0 +1,625 @@
/**
* Autogenerated by Thrift Compiler (0.9.3)
*
* DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
* @generated
*/
package org.apache.hadoop.hbase.thrift2.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 javax.annotation.Generated;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
/**
* Represents a single cell and the amount to increment it by
*/
@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2016-05-25")
public class TColumnIncrement implements org.apache.thrift.TBase<TColumnIncrement, TColumnIncrement._Fields>, java.io.Serializable, Cloneable, Comparable<TColumnIncrement> {
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);
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)2);
private static final org.apache.thrift.protocol.TField AMOUNT_FIELD_DESC = new org.apache.thrift.protocol.TField("amount", org.apache.thrift.protocol.TType.I64, (short)3);
private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
static {
schemes.put(StandardScheme.class, new TColumnIncrementStandardSchemeFactory());
schemes.put(TupleScheme.class, new TColumnIncrementTupleSchemeFactory());
}
public ByteBuffer family; // required
public ByteBuffer qualifier; // required
public long amount; // 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 {
FAMILY((short)1, "family"),
QUALIFIER((short)2, "qualifier"),
AMOUNT((short)3, "amount");
private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
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: // FAMILY
return FAMILY;
case 2: // QUALIFIER
return QUALIFIER;
case 3: // AMOUNT
return AMOUNT;
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 __AMOUNT_ISSET_ID = 0;
private byte __isset_bitfield = 0;
private static final _Fields optionals[] = {_Fields.AMOUNT};
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.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.AMOUNT, new org.apache.thrift.meta_data.FieldMetaData("amount", org.apache.thrift.TFieldRequirementType.OPTIONAL,
new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
metaDataMap = Collections.unmodifiableMap(tmpMap);
org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TColumnIncrement.class, metaDataMap);
}
public TColumnIncrement() {
this.amount = 1L;
}
public TColumnIncrement(
ByteBuffer family,
ByteBuffer qualifier)
{
this();
this.family = org.apache.thrift.TBaseHelper.copyBinary(family);
this.qualifier = org.apache.thrift.TBaseHelper.copyBinary(qualifier);
}
/**
* Performs a deep copy on <i>other</i>.
*/
public TColumnIncrement(TColumnIncrement other) {
__isset_bitfield = other.__isset_bitfield;
if (other.isSetFamily()) {
this.family = org.apache.thrift.TBaseHelper.copyBinary(other.family);
}
if (other.isSetQualifier()) {
this.qualifier = org.apache.thrift.TBaseHelper.copyBinary(other.qualifier);
}
this.amount = other.amount;
}
public TColumnIncrement deepCopy() {
return new TColumnIncrement(this);
}
@Override
public void clear() {
this.family = null;
this.qualifier = null;
this.amount = 1L;
}
public byte[] getFamily() {
setFamily(org.apache.thrift.TBaseHelper.rightSize(family));
return family == null ? null : family.array();
}
public ByteBuffer bufferForFamily() {
return org.apache.thrift.TBaseHelper.copyBinary(family);
}
public TColumnIncrement setFamily(byte[] family) {
this.family = family == null ? (ByteBuffer)null : ByteBuffer.wrap(Arrays.copyOf(family, family.length));
return this;
}
public TColumnIncrement setFamily(ByteBuffer family) {
this.family = org.apache.thrift.TBaseHelper.copyBinary(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 org.apache.thrift.TBaseHelper.copyBinary(qualifier);
}
public TColumnIncrement setQualifier(byte[] qualifier) {
this.qualifier = qualifier == null ? (ByteBuffer)null : ByteBuffer.wrap(Arrays.copyOf(qualifier, qualifier.length));
return this;
}
public TColumnIncrement setQualifier(ByteBuffer qualifier) {
this.qualifier = org.apache.thrift.TBaseHelper.copyBinary(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 long getAmount() {
return this.amount;
}
public TColumnIncrement setAmount(long amount) {
this.amount = amount;
setAmountIsSet(true);
return this;
}
public void unsetAmount() {
__isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __AMOUNT_ISSET_ID);
}
/** Returns true if field amount is set (has been assigned a value) and false otherwise */
public boolean isSetAmount() {
return EncodingUtils.testBit(__isset_bitfield, __AMOUNT_ISSET_ID);
}
public void setAmountIsSet(boolean value) {
__isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __AMOUNT_ISSET_ID, value);
}
public void setFieldValue(_Fields field, Object value) {
switch (field) {
case FAMILY:
if (value == null) {
unsetFamily();
} else {
setFamily((ByteBuffer)value);
}
break;
case QUALIFIER:
if (value == null) {
unsetQualifier();
} else {
setQualifier((ByteBuffer)value);
}
break;
case AMOUNT:
if (value == null) {
unsetAmount();
} else {
setAmount((Long)value);
}
break;
}
}
public Object getFieldValue(_Fields field) {
switch (field) {
case FAMILY:
return getFamily();
case QUALIFIER:
return getQualifier();
case AMOUNT:
return getAmount();
}
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 FAMILY:
return isSetFamily();
case QUALIFIER:
return isSetQualifier();
case AMOUNT:
return isSetAmount();
}
throw new IllegalStateException();
}
@Override
public boolean equals(Object that) {
if (that == null)
return false;
if (that instanceof TColumnIncrement)
return this.equals((TColumnIncrement)that);
return false;
}
public boolean equals(TColumnIncrement that) {
if (that == null)
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_amount = true && this.isSetAmount();
boolean that_present_amount = true && that.isSetAmount();
if (this_present_amount || that_present_amount) {
if (!(this_present_amount && that_present_amount))
return false;
if (this.amount != that.amount)
return false;
}
return true;
}
@Override
public int hashCode() {
List<Object> list = new ArrayList<Object>();
boolean present_family = true && (isSetFamily());
list.add(present_family);
if (present_family)
list.add(family);
boolean present_qualifier = true && (isSetQualifier());
list.add(present_qualifier);
if (present_qualifier)
list.add(qualifier);
boolean present_amount = true && (isSetAmount());
list.add(present_amount);
if (present_amount)
list.add(amount);
return list.hashCode();
}
@Override
public int compareTo(TColumnIncrement other) {
if (!getClass().equals(other.getClass())) {
return getClass().getName().compareTo(other.getClass().getName());
}
int lastComparison = 0;
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(isSetAmount()).compareTo(other.isSetAmount());
if (lastComparison != 0) {
return lastComparison;
}
if (isSetAmount()) {
lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.amount, other.amount);
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("TColumnIncrement(");
boolean first = true;
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 (isSetAmount()) {
if (!first) sb.append(", ");
sb.append("amount:");
sb.append(this.amount);
first = false;
}
sb.append(")");
return sb.toString();
}
public void validate() throws org.apache.thrift.TException {
// check for required fields
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());
}
// 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 TColumnIncrementStandardSchemeFactory implements SchemeFactory {
public TColumnIncrementStandardScheme getScheme() {
return new TColumnIncrementStandardScheme();
}
}
private static class TColumnIncrementStandardScheme extends StandardScheme<TColumnIncrement> {
public void read(org.apache.thrift.protocol.TProtocol iprot, TColumnIncrement 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: // 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 2: // 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 3: // AMOUNT
if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
struct.amount = iprot.readI64();
struct.setAmountIsSet(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, TColumnIncrement struct) throws org.apache.thrift.TException {
struct.validate();
oprot.writeStructBegin(STRUCT_DESC);
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.isSetAmount()) {
oprot.writeFieldBegin(AMOUNT_FIELD_DESC);
oprot.writeI64(struct.amount);
oprot.writeFieldEnd();
}
oprot.writeFieldStop();
oprot.writeStructEnd();
}
}
private static class TColumnIncrementTupleSchemeFactory implements SchemeFactory {
public TColumnIncrementTupleScheme getScheme() {
return new TColumnIncrementTupleScheme();
}
}
private static class TColumnIncrementTupleScheme extends TupleScheme<TColumnIncrement> {
@Override
public void write(org.apache.thrift.protocol.TProtocol prot, TColumnIncrement struct) throws org.apache.thrift.TException {
TTupleProtocol oprot = (TTupleProtocol) prot;
oprot.writeBinary(struct.family);
oprot.writeBinary(struct.qualifier);
BitSet optionals = new BitSet();
if (struct.isSetAmount()) {
optionals.set(0);
}
oprot.writeBitSet(optionals, 1);
if (struct.isSetAmount()) {
oprot.writeI64(struct.amount);
}
}
@Override
public void read(org.apache.thrift.protocol.TProtocol prot, TColumnIncrement struct) throws org.apache.thrift.TException {
TTupleProtocol iprot = (TTupleProtocol) prot;
struct.family = iprot.readBinary();
struct.setFamilyIsSet(true);
struct.qualifier = iprot.readBinary();
struct.setQualifierIsSet(true);
BitSet incoming = iprot.readBitSet(1);
if (incoming.get(0)) {
struct.amount = iprot.readI64();
struct.setAmountIsSet(true);
}
}
}
}

View File

@ -0,0 +1,851 @@
/**
* Autogenerated by Thrift Compiler (0.9.3)
*
* DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
* @generated
*/
package org.apache.hadoop.hbase.thrift2.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 javax.annotation.Generated;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
/**
* Represents a single cell and its value.
*/
@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2016-05-25")
public class TColumnValue implements org.apache.thrift.TBase<TColumnValue, TColumnValue._Fields>, java.io.Serializable, Cloneable, Comparable<TColumnValue> {
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);
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)2);
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)3);
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)4);
private static final org.apache.thrift.protocol.TField TAGS_FIELD_DESC = new org.apache.thrift.protocol.TField("tags", org.apache.thrift.protocol.TType.STRING, (short)5);
private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
static {
schemes.put(StandardScheme.class, new TColumnValueStandardSchemeFactory());
schemes.put(TupleScheme.class, new TColumnValueTupleSchemeFactory());
}
public ByteBuffer family; // required
public ByteBuffer qualifier; // required
public ByteBuffer value; // required
public long timestamp; // optional
public ByteBuffer tags; // 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 {
FAMILY((short)1, "family"),
QUALIFIER((short)2, "qualifier"),
VALUE((short)3, "value"),
TIMESTAMP((short)4, "timestamp"),
TAGS((short)5, "tags");
private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
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: // FAMILY
return FAMILY;
case 2: // QUALIFIER
return QUALIFIER;
case 3: // VALUE
return VALUE;
case 4: // TIMESTAMP
return TIMESTAMP;
case 5: // TAGS
return TAGS;
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 byte __isset_bitfield = 0;
private static final _Fields optionals[] = {_Fields.TIMESTAMP,_Fields.TAGS};
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.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.TAGS, new org.apache.thrift.meta_data.FieldMetaData("tags", org.apache.thrift.TFieldRequirementType.OPTIONAL,
new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , true)));
metaDataMap = Collections.unmodifiableMap(tmpMap);
org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TColumnValue.class, metaDataMap);
}
public TColumnValue() {
}
public TColumnValue(
ByteBuffer family,
ByteBuffer qualifier,
ByteBuffer value)
{
this();
this.family = org.apache.thrift.TBaseHelper.copyBinary(family);
this.qualifier = org.apache.thrift.TBaseHelper.copyBinary(qualifier);
this.value = org.apache.thrift.TBaseHelper.copyBinary(value);
}
/**
* Performs a deep copy on <i>other</i>.
*/
public TColumnValue(TColumnValue other) {
__isset_bitfield = other.__isset_bitfield;
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;
if (other.isSetTags()) {
this.tags = org.apache.thrift.TBaseHelper.copyBinary(other.tags);
}
}
public TColumnValue deepCopy() {
return new TColumnValue(this);
}
@Override
public void clear() {
this.family = null;
this.qualifier = null;
this.value = null;
setTimestampIsSet(false);
this.timestamp = 0;
this.tags = null;
}
public byte[] getFamily() {
setFamily(org.apache.thrift.TBaseHelper.rightSize(family));
return family == null ? null : family.array();
}
public ByteBuffer bufferForFamily() {
return org.apache.thrift.TBaseHelper.copyBinary(family);
}
public TColumnValue setFamily(byte[] family) {
this.family = family == null ? (ByteBuffer)null : ByteBuffer.wrap(Arrays.copyOf(family, family.length));
return this;
}
public TColumnValue setFamily(ByteBuffer family) {
this.family = org.apache.thrift.TBaseHelper.copyBinary(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 org.apache.thrift.TBaseHelper.copyBinary(qualifier);
}
public TColumnValue setQualifier(byte[] qualifier) {
this.qualifier = qualifier == null ? (ByteBuffer)null : ByteBuffer.wrap(Arrays.copyOf(qualifier, qualifier.length));
return this;
}
public TColumnValue setQualifier(ByteBuffer qualifier) {
this.qualifier = org.apache.thrift.TBaseHelper.copyBinary(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 org.apache.thrift.TBaseHelper.copyBinary(value);
}
public TColumnValue setValue(byte[] value) {
this.value = value == null ? (ByteBuffer)null : ByteBuffer.wrap(Arrays.copyOf(value, value.length));
return this;
}
public TColumnValue setValue(ByteBuffer value) {
this.value = org.apache.thrift.TBaseHelper.copyBinary(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[] getTags() {
setTags(org.apache.thrift.TBaseHelper.rightSize(tags));
return tags == null ? null : tags.array();
}
public ByteBuffer bufferForTags() {
return org.apache.thrift.TBaseHelper.copyBinary(tags);
}
public TColumnValue setTags(byte[] tags) {
this.tags = tags == null ? (ByteBuffer)null : ByteBuffer.wrap(Arrays.copyOf(tags, tags.length));
return this;
}
public TColumnValue setTags(ByteBuffer tags) {
this.tags = org.apache.thrift.TBaseHelper.copyBinary(tags);
return this;
}
public void unsetTags() {
this.tags = null;
}
/** Returns true if field tags is set (has been assigned a value) and false otherwise */
public boolean isSetTags() {
return this.tags != null;
}
public void setTagsIsSet(boolean value) {
if (!value) {
this.tags = null;
}
}
public void setFieldValue(_Fields field, Object value) {
switch (field) {
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 TAGS:
if (value == null) {
unsetTags();
} else {
setTags((ByteBuffer)value);
}
break;
}
}
public Object getFieldValue(_Fields field) {
switch (field) {
case FAMILY:
return getFamily();
case QUALIFIER:
return getQualifier();
case VALUE:
return getValue();
case TIMESTAMP:
return getTimestamp();
case TAGS:
return getTags();
}
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 FAMILY:
return isSetFamily();
case QUALIFIER:
return isSetQualifier();
case VALUE:
return isSetValue();
case TIMESTAMP:
return isSetTimestamp();
case TAGS:
return isSetTags();
}
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_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_tags = true && this.isSetTags();
boolean that_present_tags = true && that.isSetTags();
if (this_present_tags || that_present_tags) {
if (!(this_present_tags && that_present_tags))
return false;
if (!this.tags.equals(that.tags))
return false;
}
return true;
}
@Override
public int hashCode() {
List<Object> list = new ArrayList<Object>();
boolean present_family = true && (isSetFamily());
list.add(present_family);
if (present_family)
list.add(family);
boolean present_qualifier = true && (isSetQualifier());
list.add(present_qualifier);
if (present_qualifier)
list.add(qualifier);
boolean present_value = true && (isSetValue());
list.add(present_value);
if (present_value)
list.add(value);
boolean present_timestamp = true && (isSetTimestamp());
list.add(present_timestamp);
if (present_timestamp)
list.add(timestamp);
boolean present_tags = true && (isSetTags());
list.add(present_tags);
if (present_tags)
list.add(tags);
return list.hashCode();
}
@Override
public int compareTo(TColumnValue other) {
if (!getClass().equals(other.getClass())) {
return getClass().getName().compareTo(other.getClass().getName());
}
int lastComparison = 0;
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(isSetTags()).compareTo(other.isSetTags());
if (lastComparison != 0) {
return lastComparison;
}
if (isSetTags()) {
lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tags, other.tags);
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("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 (isSetTags()) {
if (!first) sb.append(", ");
sb.append("tags:");
if (this.tags == null) {
sb.append("null");
} else {
org.apache.thrift.TBaseHelper.toString(this.tags, sb);
}
first = false;
}
sb.append(")");
return sb.toString();
}
public void validate() throws org.apache.thrift.TException {
// check for required fields
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());
}
// 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<TColumnValue> {
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: // 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 2: // 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 3: // 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 4: // 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 5: // TAGS
if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
struct.tags = iprot.readBinary();
struct.setTagsIsSet(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, TColumnValue struct) throws org.apache.thrift.TException {
struct.validate();
oprot.writeStructBegin(STRUCT_DESC);
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();
}
if (struct.tags != null) {
if (struct.isSetTags()) {
oprot.writeFieldBegin(TAGS_FIELD_DESC);
oprot.writeBinary(struct.tags);
oprot.writeFieldEnd();
}
}
oprot.writeFieldStop();
oprot.writeStructEnd();
}
}
private static class TColumnValueTupleSchemeFactory implements SchemeFactory {
public TColumnValueTupleScheme getScheme() {
return new TColumnValueTupleScheme();
}
}
private static class TColumnValueTupleScheme extends TupleScheme<TColumnValue> {
@Override
public void write(org.apache.thrift.protocol.TProtocol prot, TColumnValue struct) throws org.apache.thrift.TException {
TTupleProtocol oprot = (TTupleProtocol) prot;
oprot.writeBinary(struct.family);
oprot.writeBinary(struct.qualifier);
oprot.writeBinary(struct.value);
BitSet optionals = new BitSet();
if (struct.isSetTimestamp()) {
optionals.set(0);
}
if (struct.isSetTags()) {
optionals.set(1);
}
oprot.writeBitSet(optionals, 2);
if (struct.isSetTimestamp()) {
oprot.writeI64(struct.timestamp);
}
if (struct.isSetTags()) {
oprot.writeBinary(struct.tags);
}
}
@Override
public void read(org.apache.thrift.protocol.TProtocol prot, TColumnValue struct) throws org.apache.thrift.TException {
TTupleProtocol iprot = (TTupleProtocol) prot;
struct.family = iprot.readBinary();
struct.setFamilyIsSet(true);
struct.qualifier = iprot.readBinary();
struct.setQualifierIsSet(true);
struct.value = iprot.readBinary();
struct.setValueIsSet(true);
BitSet incoming = iprot.readBitSet(2);
if (incoming.get(0)) {
struct.timestamp = iprot.readI64();
struct.setTimestampIsSet(true);
}
if (incoming.get(1)) {
struct.tags = iprot.readBinary();
struct.setTagsIsSet(true);
}
}
}
}

View File

@ -0,0 +1,64 @@
/**
* Autogenerated by Thrift Compiler (0.9.3)
*
* DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
* @generated
*/
package org.apache.hadoop.hbase.thrift2.generated;
import java.util.Map;
import java.util.HashMap;
import org.apache.thrift.TEnum;
/**
* Thrift wrapper around
* org.apache.hadoop.hbase.filter.CompareFilter$CompareOp.
*/
public enum TCompareOp implements org.apache.thrift.TEnum {
LESS(0),
LESS_OR_EQUAL(1),
EQUAL(2),
NOT_EQUAL(3),
GREATER_OR_EQUAL(4),
GREATER(5),
NO_OP(6);
private final int value;
private TCompareOp(int value) {
this.value = value;
}
/**
* Get the integer value of this enum value, as defined in the Thrift IDL.
*/
public int getValue() {
return value;
}
/**
* Find a the enum type by its integer value, as defined in the Thrift IDL.
* @return null if the value is not found.
*/
public static TCompareOp findByValue(int value) {
switch (value) {
case 0:
return LESS;
case 1:
return LESS_OR_EQUAL;
case 2:
return EQUAL;
case 3:
return NOT_EQUAL;
case 4:
return GREATER_OR_EQUAL;
case 5:
return GREATER;
case 6:
return NO_OP;
default:
return null;
}
}
}

File diff suppressed because it is too large Load Diff

View File

@ -0,0 +1,50 @@
/**
* Autogenerated by Thrift Compiler (0.9.3)
*
* DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
* @generated
*/
package org.apache.hadoop.hbase.thrift2.generated;
import java.util.Map;
import java.util.HashMap;
import org.apache.thrift.TEnum;
/**
* Specify type of delete:
* - DELETE_COLUMN means exactly one version will be removed,
* - DELETE_COLUMNS means previous versions will also be removed.
*/
public enum TDeleteType implements org.apache.thrift.TEnum {
DELETE_COLUMN(0),
DELETE_COLUMNS(1);
private final int value;
private TDeleteType(int value) {
this.value = value;
}
/**
* Get the integer value of this enum value, as defined in the Thrift IDL.
*/
public int getValue() {
return value;
}
/**
* Find a the enum type by its integer value, as defined in the Thrift IDL.
* @return null if the value is not found.
*/
public static TDeleteType findByValue(int value) {
switch (value) {
case 0:
return DELETE_COLUMN;
case 1:
return DELETE_COLUMNS;
default:
return null;
}
}
}

View File

@ -0,0 +1,58 @@
/**
* Autogenerated by Thrift Compiler (0.9.3)
*
* DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
* @generated
*/
package org.apache.hadoop.hbase.thrift2.generated;
import java.util.Map;
import java.util.HashMap;
import org.apache.thrift.TEnum;
/**
* Specify Durability:
* - SKIP_WAL means do not write the Mutation to the WAL.
* - ASYNC_WAL means write the Mutation to the WAL asynchronously,
* - SYNC_WAL means write the Mutation to the WAL synchronously,
* - FSYNC_WAL means Write the Mutation to the WAL synchronously and force the entries to disk.
*/
public enum TDurability implements org.apache.thrift.TEnum {
SKIP_WAL(1),
ASYNC_WAL(2),
SYNC_WAL(3),
FSYNC_WAL(4);
private final int value;
private TDurability(int value) {
this.value = value;
}
/**
* Get the integer value of this enum value, as defined in the Thrift IDL.
*/
public int getValue() {
return value;
}
/**
* Find a the enum type by its integer value, as defined in the Thrift IDL.
* @return null if the value is not found.
*/
public static TDurability findByValue(int value) {
switch (value) {
case 1:
return SKIP_WAL;
case 2:
return ASYNC_WAL;
case 3:
return SYNC_WAL;
case 4:
return FSYNC_WAL;
default:
return null;
}
}
}

File diff suppressed because it is too large Load Diff

View File

@ -0,0 +1,502 @@
/**
* Autogenerated by Thrift Compiler (0.9.3)
*
* DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
* @generated
*/
package org.apache.hadoop.hbase.thrift2.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 javax.annotation.Generated;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2016-05-25")
public class THRegionLocation implements org.apache.thrift.TBase<THRegionLocation, THRegionLocation._Fields>, java.io.Serializable, Cloneable, Comparable<THRegionLocation> {
private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("THRegionLocation");
private static final org.apache.thrift.protocol.TField SERVER_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("serverName", org.apache.thrift.protocol.TType.STRUCT, (short)1);
private static final org.apache.thrift.protocol.TField REGION_INFO_FIELD_DESC = new org.apache.thrift.protocol.TField("regionInfo", org.apache.thrift.protocol.TType.STRUCT, (short)2);
private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
static {
schemes.put(StandardScheme.class, new THRegionLocationStandardSchemeFactory());
schemes.put(TupleScheme.class, new THRegionLocationTupleSchemeFactory());
}
public TServerName serverName; // required
public THRegionInfo regionInfo; // 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 {
SERVER_NAME((short)1, "serverName"),
REGION_INFO((short)2, "regionInfo");
private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
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: // SERVER_NAME
return SERVER_NAME;
case 2: // REGION_INFO
return REGION_INFO;
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.SERVER_NAME, new org.apache.thrift.meta_data.FieldMetaData("serverName", org.apache.thrift.TFieldRequirementType.REQUIRED,
new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TServerName.class)));
tmpMap.put(_Fields.REGION_INFO, new org.apache.thrift.meta_data.FieldMetaData("regionInfo", org.apache.thrift.TFieldRequirementType.REQUIRED,
new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, THRegionInfo.class)));
metaDataMap = Collections.unmodifiableMap(tmpMap);
org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(THRegionLocation.class, metaDataMap);
}
public THRegionLocation() {
}
public THRegionLocation(
TServerName serverName,
THRegionInfo regionInfo)
{
this();
this.serverName = serverName;
this.regionInfo = regionInfo;
}
/**
* Performs a deep copy on <i>other</i>.
*/
public THRegionLocation(THRegionLocation other) {
if (other.isSetServerName()) {
this.serverName = new TServerName(other.serverName);
}
if (other.isSetRegionInfo()) {
this.regionInfo = new THRegionInfo(other.regionInfo);
}
}
public THRegionLocation deepCopy() {
return new THRegionLocation(this);
}
@Override
public void clear() {
this.serverName = null;
this.regionInfo = null;
}
public TServerName getServerName() {
return this.serverName;
}
public THRegionLocation setServerName(TServerName serverName) {
this.serverName = serverName;
return this;
}
public void unsetServerName() {
this.serverName = null;
}
/** Returns true if field serverName is set (has been assigned a value) and false otherwise */
public boolean isSetServerName() {
return this.serverName != null;
}
public void setServerNameIsSet(boolean value) {
if (!value) {
this.serverName = null;
}
}
public THRegionInfo getRegionInfo() {
return this.regionInfo;
}
public THRegionLocation setRegionInfo(THRegionInfo regionInfo) {
this.regionInfo = regionInfo;
return this;
}
public void unsetRegionInfo() {
this.regionInfo = null;
}
/** Returns true if field regionInfo is set (has been assigned a value) and false otherwise */
public boolean isSetRegionInfo() {
return this.regionInfo != null;
}
public void setRegionInfoIsSet(boolean value) {
if (!value) {
this.regionInfo = null;
}
}
public void setFieldValue(_Fields field, Object value) {
switch (field) {
case SERVER_NAME:
if (value == null) {
unsetServerName();
} else {
setServerName((TServerName)value);
}
break;
case REGION_INFO:
if (value == null) {
unsetRegionInfo();
} else {
setRegionInfo((THRegionInfo)value);
}
break;
}
}
public Object getFieldValue(_Fields field) {
switch (field) {
case SERVER_NAME:
return getServerName();
case REGION_INFO:
return getRegionInfo();
}
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 SERVER_NAME:
return isSetServerName();
case REGION_INFO:
return isSetRegionInfo();
}
throw new IllegalStateException();
}
@Override
public boolean equals(Object that) {
if (that == null)
return false;
if (that instanceof THRegionLocation)
return this.equals((THRegionLocation)that);
return false;
}
public boolean equals(THRegionLocation that) {
if (that == null)
return false;
boolean this_present_serverName = true && this.isSetServerName();
boolean that_present_serverName = true && that.isSetServerName();
if (this_present_serverName || that_present_serverName) {
if (!(this_present_serverName && that_present_serverName))
return false;
if (!this.serverName.equals(that.serverName))
return false;
}
boolean this_present_regionInfo = true && this.isSetRegionInfo();
boolean that_present_regionInfo = true && that.isSetRegionInfo();
if (this_present_regionInfo || that_present_regionInfo) {
if (!(this_present_regionInfo && that_present_regionInfo))
return false;
if (!this.regionInfo.equals(that.regionInfo))
return false;
}
return true;
}
@Override
public int hashCode() {
List<Object> list = new ArrayList<Object>();
boolean present_serverName = true && (isSetServerName());
list.add(present_serverName);
if (present_serverName)
list.add(serverName);
boolean present_regionInfo = true && (isSetRegionInfo());
list.add(present_regionInfo);
if (present_regionInfo)
list.add(regionInfo);
return list.hashCode();
}
@Override
public int compareTo(THRegionLocation other) {
if (!getClass().equals(other.getClass())) {
return getClass().getName().compareTo(other.getClass().getName());
}
int lastComparison = 0;
lastComparison = Boolean.valueOf(isSetServerName()).compareTo(other.isSetServerName());
if (lastComparison != 0) {
return lastComparison;
}
if (isSetServerName()) {
lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.serverName, other.serverName);
if (lastComparison != 0) {
return lastComparison;
}
}
lastComparison = Boolean.valueOf(isSetRegionInfo()).compareTo(other.isSetRegionInfo());
if (lastComparison != 0) {
return lastComparison;
}
if (isSetRegionInfo()) {
lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.regionInfo, other.regionInfo);
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("THRegionLocation(");
boolean first = true;
sb.append("serverName:");
if (this.serverName == null) {
sb.append("null");
} else {
sb.append(this.serverName);
}
first = false;
if (!first) sb.append(", ");
sb.append("regionInfo:");
if (this.regionInfo == null) {
sb.append("null");
} else {
sb.append(this.regionInfo);
}
first = false;
sb.append(")");
return sb.toString();
}
public void validate() throws org.apache.thrift.TException {
// check for required fields
if (serverName == null) {
throw new org.apache.thrift.protocol.TProtocolException("Required field 'serverName' was not present! Struct: " + toString());
}
if (regionInfo == null) {
throw new org.apache.thrift.protocol.TProtocolException("Required field 'regionInfo' was not present! Struct: " + toString());
}
// check for sub-struct validity
if (serverName != null) {
serverName.validate();
}
if (regionInfo != null) {
regionInfo.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 THRegionLocationStandardSchemeFactory implements SchemeFactory {
public THRegionLocationStandardScheme getScheme() {
return new THRegionLocationStandardScheme();
}
}
private static class THRegionLocationStandardScheme extends StandardScheme<THRegionLocation> {
public void read(org.apache.thrift.protocol.TProtocol iprot, THRegionLocation 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: // SERVER_NAME
if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
struct.serverName = new TServerName();
struct.serverName.read(iprot);
struct.setServerNameIsSet(true);
} else {
org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
}
break;
case 2: // REGION_INFO
if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
struct.regionInfo = new THRegionInfo();
struct.regionInfo.read(iprot);
struct.setRegionInfoIsSet(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, THRegionLocation struct) throws org.apache.thrift.TException {
struct.validate();
oprot.writeStructBegin(STRUCT_DESC);
if (struct.serverName != null) {
oprot.writeFieldBegin(SERVER_NAME_FIELD_DESC);
struct.serverName.write(oprot);
oprot.writeFieldEnd();
}
if (struct.regionInfo != null) {
oprot.writeFieldBegin(REGION_INFO_FIELD_DESC);
struct.regionInfo.write(oprot);
oprot.writeFieldEnd();
}
oprot.writeFieldStop();
oprot.writeStructEnd();
}
}
private static class THRegionLocationTupleSchemeFactory implements SchemeFactory {
public THRegionLocationTupleScheme getScheme() {
return new THRegionLocationTupleScheme();
}
}
private static class THRegionLocationTupleScheme extends TupleScheme<THRegionLocation> {
@Override
public void write(org.apache.thrift.protocol.TProtocol prot, THRegionLocation struct) throws org.apache.thrift.TException {
TTupleProtocol oprot = (TTupleProtocol) prot;
struct.serverName.write(oprot);
struct.regionInfo.write(oprot);
}
@Override
public void read(org.apache.thrift.protocol.TProtocol prot, THRegionLocation struct) throws org.apache.thrift.TException {
TTupleProtocol iprot = (TTupleProtocol) prot;
struct.serverName = new TServerName();
struct.serverName.read(iprot);
struct.setServerNameIsSet(true);
struct.regionInfo = new THRegionInfo();
struct.regionInfo.read(iprot);
struct.setRegionInfoIsSet(true);
}
}
}

View File

@ -0,0 +1,401 @@
/**
* Autogenerated by Thrift Compiler (0.9.3)
*
* DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
* @generated
*/
package org.apache.hadoop.hbase.thrift2.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 javax.annotation.Generated;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
/**
* 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.
*/
@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2016-05-25")
public class TIOError extends TException implements org.apache.thrift.TBase<TIOError, TIOError._Fields>, java.io.Serializable, Cloneable, Comparable<TIOError> {
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<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, 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<String, _Fields> byName = new HashMap<String, _Fields>();
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 static final _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 <i>other</i>.
*/
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() {
List<Object> list = new ArrayList<Object>();
boolean present_message = true && (isSetMessage());
list.add(present_message);
if (present_message)
list.add(message);
return list.hashCode();
}
@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<TIOError> {
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<TIOError> {
@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);
}
}
}
}

View File

@ -0,0 +1,400 @@
/**
* Autogenerated by Thrift Compiler (0.9.3)
*
* DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
* @generated
*/
package org.apache.hadoop.hbase.thrift2.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 javax.annotation.Generated;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
/**
* A TIllegalArgument exception indicates an illegal or invalid
* argument was passed into a procedure.
*/
@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2016-05-25")
public class TIllegalArgument extends TException implements org.apache.thrift.TBase<TIllegalArgument, TIllegalArgument._Fields>, java.io.Serializable, Cloneable, Comparable<TIllegalArgument> {
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<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, 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<String, _Fields> byName = new HashMap<String, _Fields>();
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 static final _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 <i>other</i>.
*/
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() {
List<Object> list = new ArrayList<Object>();
boolean present_message = true && (isSetMessage());
list.add(present_message);
if (present_message)
list.add(message);
return list.hashCode();
}
@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<TIllegalArgument> {
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<TIllegalArgument> {
@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);
}
}
}
}

View File

@ -0,0 +1,961 @@
/**
* Autogenerated by Thrift Compiler (0.9.3)
*
* DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
* @generated
*/
package org.apache.hadoop.hbase.thrift2.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 javax.annotation.Generated;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
/**
* Used to perform Increment operations for a single row.
*
* You can specify how this Increment should be written to the write-ahead Log (WAL)
* by changing the durability. If you don't provide durability, it defaults to
* column family's default setting for durability.
*/
@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2016-05-25")
public class TIncrement implements org.apache.thrift.TBase<TIncrement, TIncrement._Fields>, java.io.Serializable, Cloneable, Comparable<TIncrement> {
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);
private static final org.apache.thrift.protocol.TField COLUMNS_FIELD_DESC = new org.apache.thrift.protocol.TField("columns", org.apache.thrift.protocol.TType.LIST, (short)2);
private static final org.apache.thrift.protocol.TField ATTRIBUTES_FIELD_DESC = new org.apache.thrift.protocol.TField("attributes", org.apache.thrift.protocol.TType.MAP, (short)4);
private static final org.apache.thrift.protocol.TField DURABILITY_FIELD_DESC = new org.apache.thrift.protocol.TField("durability", org.apache.thrift.protocol.TType.I32, (short)5);
private static final org.apache.thrift.protocol.TField CELL_VISIBILITY_FIELD_DESC = new org.apache.thrift.protocol.TField("cellVisibility", org.apache.thrift.protocol.TType.STRUCT, (short)6);
private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
static {
schemes.put(StandardScheme.class, new TIncrementStandardSchemeFactory());
schemes.put(TupleScheme.class, new TIncrementTupleSchemeFactory());
}
public ByteBuffer row; // required
public List<TColumnIncrement> columns; // required
public Map<ByteBuffer,ByteBuffer> attributes; // optional
/**
*
* @see TDurability
*/
public TDurability durability; // optional
public TCellVisibility cellVisibility; // 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 {
ROW((short)1, "row"),
COLUMNS((short)2, "columns"),
ATTRIBUTES((short)4, "attributes"),
/**
*
* @see TDurability
*/
DURABILITY((short)5, "durability"),
CELL_VISIBILITY((short)6, "cellVisibility");
private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
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: // COLUMNS
return COLUMNS;
case 4: // ATTRIBUTES
return ATTRIBUTES;
case 5: // DURABILITY
return DURABILITY;
case 6: // CELL_VISIBILITY
return CELL_VISIBILITY;
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 _Fields optionals[] = {_Fields.ATTRIBUTES,_Fields.DURABILITY,_Fields.CELL_VISIBILITY};
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.COLUMNS, new org.apache.thrift.meta_data.FieldMetaData("columns", 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, TColumnIncrement.class))));
tmpMap.put(_Fields.ATTRIBUTES, new org.apache.thrift.meta_data.FieldMetaData("attributes", org.apache.thrift.TFieldRequirementType.OPTIONAL,
new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP,
new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , true),
new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , true))));
tmpMap.put(_Fields.DURABILITY, new org.apache.thrift.meta_data.FieldMetaData("durability", org.apache.thrift.TFieldRequirementType.OPTIONAL,
new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, TDurability.class)));
tmpMap.put(_Fields.CELL_VISIBILITY, new org.apache.thrift.meta_data.FieldMetaData("cellVisibility", org.apache.thrift.TFieldRequirementType.OPTIONAL,
new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TCellVisibility.class)));
metaDataMap = Collections.unmodifiableMap(tmpMap);
org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TIncrement.class, metaDataMap);
}
public TIncrement() {
}
public TIncrement(
ByteBuffer row,
List<TColumnIncrement> columns)
{
this();
this.row = org.apache.thrift.TBaseHelper.copyBinary(row);
this.columns = columns;
}
/**
* Performs a deep copy on <i>other</i>.
*/
public TIncrement(TIncrement other) {
if (other.isSetRow()) {
this.row = org.apache.thrift.TBaseHelper.copyBinary(other.row);
}
if (other.isSetColumns()) {
List<TColumnIncrement> __this__columns = new ArrayList<TColumnIncrement>(other.columns.size());
for (TColumnIncrement other_element : other.columns) {
__this__columns.add(new TColumnIncrement(other_element));
}
this.columns = __this__columns;
}
if (other.isSetAttributes()) {
Map<ByteBuffer,ByteBuffer> __this__attributes = new HashMap<ByteBuffer,ByteBuffer>(other.attributes);
this.attributes = __this__attributes;
}
if (other.isSetDurability()) {
this.durability = other.durability;
}
if (other.isSetCellVisibility()) {
this.cellVisibility = new TCellVisibility(other.cellVisibility);
}
}
public TIncrement deepCopy() {
return new TIncrement(this);
}
@Override
public void clear() {
this.row = null;
this.columns = null;
this.attributes = null;
this.durability = null;
this.cellVisibility = null;
}
public byte[] getRow() {
setRow(org.apache.thrift.TBaseHelper.rightSize(row));
return row == null ? null : row.array();
}
public ByteBuffer bufferForRow() {
return org.apache.thrift.TBaseHelper.copyBinary(row);
}
public TIncrement setRow(byte[] row) {
this.row = row == null ? (ByteBuffer)null : ByteBuffer.wrap(Arrays.copyOf(row, row.length));
return this;
}
public TIncrement setRow(ByteBuffer row) {
this.row = org.apache.thrift.TBaseHelper.copyBinary(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 int getColumnsSize() {
return (this.columns == null) ? 0 : this.columns.size();
}
public java.util.Iterator<TColumnIncrement> getColumnsIterator() {
return (this.columns == null) ? null : this.columns.iterator();
}
public void addToColumns(TColumnIncrement elem) {
if (this.columns == null) {
this.columns = new ArrayList<TColumnIncrement>();
}
this.columns.add(elem);
}
public List<TColumnIncrement> getColumns() {
return this.columns;
}
public TIncrement setColumns(List<TColumnIncrement> columns) {
this.columns = columns;
return this;
}
public void unsetColumns() {
this.columns = null;
}
/** Returns true if field columns is set (has been assigned a value) and false otherwise */
public boolean isSetColumns() {
return this.columns != null;
}
public void setColumnsIsSet(boolean value) {
if (!value) {
this.columns = null;
}
}
public int getAttributesSize() {
return (this.attributes == null) ? 0 : this.attributes.size();
}
public void putToAttributes(ByteBuffer key, ByteBuffer val) {
if (this.attributes == null) {
this.attributes = new HashMap<ByteBuffer,ByteBuffer>();
}
this.attributes.put(key, val);
}
public Map<ByteBuffer,ByteBuffer> getAttributes() {
return this.attributes;
}
public TIncrement setAttributes(Map<ByteBuffer,ByteBuffer> attributes) {
this.attributes = attributes;
return this;
}
public void unsetAttributes() {
this.attributes = null;
}
/** Returns true if field attributes is set (has been assigned a value) and false otherwise */
public boolean isSetAttributes() {
return this.attributes != null;
}
public void setAttributesIsSet(boolean value) {
if (!value) {
this.attributes = null;
}
}
/**
*
* @see TDurability
*/
public TDurability getDurability() {
return this.durability;
}
/**
*
* @see TDurability
*/
public TIncrement setDurability(TDurability durability) {
this.durability = durability;
return this;
}
public void unsetDurability() {
this.durability = null;
}
/** Returns true if field durability is set (has been assigned a value) and false otherwise */
public boolean isSetDurability() {
return this.durability != null;
}
public void setDurabilityIsSet(boolean value) {
if (!value) {
this.durability = null;
}
}
public TCellVisibility getCellVisibility() {
return this.cellVisibility;
}
public TIncrement setCellVisibility(TCellVisibility cellVisibility) {
this.cellVisibility = cellVisibility;
return this;
}
public void unsetCellVisibility() {
this.cellVisibility = null;
}
/** Returns true if field cellVisibility is set (has been assigned a value) and false otherwise */
public boolean isSetCellVisibility() {
return this.cellVisibility != null;
}
public void setCellVisibilityIsSet(boolean value) {
if (!value) {
this.cellVisibility = null;
}
}
public void setFieldValue(_Fields field, Object value) {
switch (field) {
case ROW:
if (value == null) {
unsetRow();
} else {
setRow((ByteBuffer)value);
}
break;
case COLUMNS:
if (value == null) {
unsetColumns();
} else {
setColumns((List<TColumnIncrement>)value);
}
break;
case ATTRIBUTES:
if (value == null) {
unsetAttributes();
} else {
setAttributes((Map<ByteBuffer,ByteBuffer>)value);
}
break;
case DURABILITY:
if (value == null) {
unsetDurability();
} else {
setDurability((TDurability)value);
}
break;
case CELL_VISIBILITY:
if (value == null) {
unsetCellVisibility();
} else {
setCellVisibility((TCellVisibility)value);
}
break;
}
}
public Object getFieldValue(_Fields field) {
switch (field) {
case ROW:
return getRow();
case COLUMNS:
return getColumns();
case ATTRIBUTES:
return getAttributes();
case DURABILITY:
return getDurability();
case CELL_VISIBILITY:
return getCellVisibility();
}
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 COLUMNS:
return isSetColumns();
case ATTRIBUTES:
return isSetAttributes();
case DURABILITY:
return isSetDurability();
case CELL_VISIBILITY:
return isSetCellVisibility();
}
throw new IllegalStateException();
}
@Override
public boolean equals(Object that) {
if (that == null)
return false;
if (that instanceof TIncrement)
return this.equals((TIncrement)that);
return false;
}
public boolean equals(TIncrement 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_columns = true && this.isSetColumns();
boolean that_present_columns = true && that.isSetColumns();
if (this_present_columns || that_present_columns) {
if (!(this_present_columns && that_present_columns))
return false;
if (!this.columns.equals(that.columns))
return false;
}
boolean this_present_attributes = true && this.isSetAttributes();
boolean that_present_attributes = true && that.isSetAttributes();
if (this_present_attributes || that_present_attributes) {
if (!(this_present_attributes && that_present_attributes))
return false;
if (!this.attributes.equals(that.attributes))
return false;
}
boolean this_present_durability = true && this.isSetDurability();
boolean that_present_durability = true && that.isSetDurability();
if (this_present_durability || that_present_durability) {
if (!(this_present_durability && that_present_durability))
return false;
if (!this.durability.equals(that.durability))
return false;
}
boolean this_present_cellVisibility = true && this.isSetCellVisibility();
boolean that_present_cellVisibility = true && that.isSetCellVisibility();
if (this_present_cellVisibility || that_present_cellVisibility) {
if (!(this_present_cellVisibility && that_present_cellVisibility))
return false;
if (!this.cellVisibility.equals(that.cellVisibility))
return false;
}
return true;
}
@Override
public int hashCode() {
List<Object> list = new ArrayList<Object>();
boolean present_row = true && (isSetRow());
list.add(present_row);
if (present_row)
list.add(row);
boolean present_columns = true && (isSetColumns());
list.add(present_columns);
if (present_columns)
list.add(columns);
boolean present_attributes = true && (isSetAttributes());
list.add(present_attributes);
if (present_attributes)
list.add(attributes);
boolean present_durability = true && (isSetDurability());
list.add(present_durability);
if (present_durability)
list.add(durability.getValue());
boolean present_cellVisibility = true && (isSetCellVisibility());
list.add(present_cellVisibility);
if (present_cellVisibility)
list.add(cellVisibility);
return list.hashCode();
}
@Override
public int compareTo(TIncrement 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(isSetColumns()).compareTo(other.isSetColumns());
if (lastComparison != 0) {
return lastComparison;
}
if (isSetColumns()) {
lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.columns, other.columns);
if (lastComparison != 0) {
return lastComparison;
}
}
lastComparison = Boolean.valueOf(isSetAttributes()).compareTo(other.isSetAttributes());
if (lastComparison != 0) {
return lastComparison;
}
if (isSetAttributes()) {
lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.attributes, other.attributes);
if (lastComparison != 0) {
return lastComparison;
}
}
lastComparison = Boolean.valueOf(isSetDurability()).compareTo(other.isSetDurability());
if (lastComparison != 0) {
return lastComparison;
}
if (isSetDurability()) {
lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.durability, other.durability);
if (lastComparison != 0) {
return lastComparison;
}
}
lastComparison = Boolean.valueOf(isSetCellVisibility()).compareTo(other.isSetCellVisibility());
if (lastComparison != 0) {
return lastComparison;
}
if (isSetCellVisibility()) {
lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.cellVisibility, other.cellVisibility);
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("TIncrement(");
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("columns:");
if (this.columns == null) {
sb.append("null");
} else {
sb.append(this.columns);
}
first = false;
if (isSetAttributes()) {
if (!first) sb.append(", ");
sb.append("attributes:");
if (this.attributes == null) {
sb.append("null");
} else {
sb.append(this.attributes);
}
first = false;
}
if (isSetDurability()) {
if (!first) sb.append(", ");
sb.append("durability:");
if (this.durability == null) {
sb.append("null");
} else {
sb.append(this.durability);
}
first = false;
}
if (isSetCellVisibility()) {
if (!first) sb.append(", ");
sb.append("cellVisibility:");
if (this.cellVisibility == null) {
sb.append("null");
} else {
sb.append(this.cellVisibility);
}
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 (columns == null) {
throw new org.apache.thrift.protocol.TProtocolException("Required field 'columns' was not present! Struct: " + toString());
}
// check for sub-struct validity
if (cellVisibility != null) {
cellVisibility.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 TIncrementStandardSchemeFactory implements SchemeFactory {
public TIncrementStandardScheme getScheme() {
return new TIncrementStandardScheme();
}
}
private static class TIncrementStandardScheme extends StandardScheme<TIncrement> {
public void read(org.apache.thrift.protocol.TProtocol iprot, TIncrement 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: // COLUMNS
if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
{
org.apache.thrift.protocol.TList _list70 = iprot.readListBegin();
struct.columns = new ArrayList<TColumnIncrement>(_list70.size);
TColumnIncrement _elem71;
for (int _i72 = 0; _i72 < _list70.size; ++_i72)
{
_elem71 = new TColumnIncrement();
_elem71.read(iprot);
struct.columns.add(_elem71);
}
iprot.readListEnd();
}
struct.setColumnsIsSet(true);
} else {
org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
}
break;
case 4: // ATTRIBUTES
if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
{
org.apache.thrift.protocol.TMap _map73 = iprot.readMapBegin();
struct.attributes = new HashMap<ByteBuffer,ByteBuffer>(2*_map73.size);
ByteBuffer _key74;
ByteBuffer _val75;
for (int _i76 = 0; _i76 < _map73.size; ++_i76)
{
_key74 = iprot.readBinary();
_val75 = iprot.readBinary();
struct.attributes.put(_key74, _val75);
}
iprot.readMapEnd();
}
struct.setAttributesIsSet(true);
} else {
org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
}
break;
case 5: // DURABILITY
if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
struct.durability = org.apache.hadoop.hbase.thrift2.generated.TDurability.findByValue(iprot.readI32());
struct.setDurabilityIsSet(true);
} else {
org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
}
break;
case 6: // CELL_VISIBILITY
if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
struct.cellVisibility = new TCellVisibility();
struct.cellVisibility.read(iprot);
struct.setCellVisibilityIsSet(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, TIncrement 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.columns != null) {
oprot.writeFieldBegin(COLUMNS_FIELD_DESC);
{
oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.columns.size()));
for (TColumnIncrement _iter77 : struct.columns)
{
_iter77.write(oprot);
}
oprot.writeListEnd();
}
oprot.writeFieldEnd();
}
if (struct.attributes != null) {
if (struct.isSetAttributes()) {
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<ByteBuffer, ByteBuffer> _iter78 : struct.attributes.entrySet())
{
oprot.writeBinary(_iter78.getKey());
oprot.writeBinary(_iter78.getValue());
}
oprot.writeMapEnd();
}
oprot.writeFieldEnd();
}
}
if (struct.durability != null) {
if (struct.isSetDurability()) {
oprot.writeFieldBegin(DURABILITY_FIELD_DESC);
oprot.writeI32(struct.durability.getValue());
oprot.writeFieldEnd();
}
}
if (struct.cellVisibility != null) {
if (struct.isSetCellVisibility()) {
oprot.writeFieldBegin(CELL_VISIBILITY_FIELD_DESC);
struct.cellVisibility.write(oprot);
oprot.writeFieldEnd();
}
}
oprot.writeFieldStop();
oprot.writeStructEnd();
}
}
private static class TIncrementTupleSchemeFactory implements SchemeFactory {
public TIncrementTupleScheme getScheme() {
return new TIncrementTupleScheme();
}
}
private static class TIncrementTupleScheme extends TupleScheme<TIncrement> {
@Override
public void write(org.apache.thrift.protocol.TProtocol prot, TIncrement struct) throws org.apache.thrift.TException {
TTupleProtocol oprot = (TTupleProtocol) prot;
oprot.writeBinary(struct.row);
{
oprot.writeI32(struct.columns.size());
for (TColumnIncrement _iter79 : struct.columns)
{
_iter79.write(oprot);
}
}
BitSet optionals = new BitSet();
if (struct.isSetAttributes()) {
optionals.set(0);
}
if (struct.isSetDurability()) {
optionals.set(1);
}
if (struct.isSetCellVisibility()) {
optionals.set(2);
}
oprot.writeBitSet(optionals, 3);
if (struct.isSetAttributes()) {
{
oprot.writeI32(struct.attributes.size());
for (Map.Entry<ByteBuffer, ByteBuffer> _iter80 : struct.attributes.entrySet())
{
oprot.writeBinary(_iter80.getKey());
oprot.writeBinary(_iter80.getValue());
}
}
}
if (struct.isSetDurability()) {
oprot.writeI32(struct.durability.getValue());
}
if (struct.isSetCellVisibility()) {
struct.cellVisibility.write(oprot);
}
}
@Override
public void read(org.apache.thrift.protocol.TProtocol prot, TIncrement struct) throws org.apache.thrift.TException {
TTupleProtocol iprot = (TTupleProtocol) prot;
struct.row = iprot.readBinary();
struct.setRowIsSet(true);
{
org.apache.thrift.protocol.TList _list81 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
struct.columns = new ArrayList<TColumnIncrement>(_list81.size);
TColumnIncrement _elem82;
for (int _i83 = 0; _i83 < _list81.size; ++_i83)
{
_elem82 = new TColumnIncrement();
_elem82.read(iprot);
struct.columns.add(_elem82);
}
}
struct.setColumnsIsSet(true);
BitSet incoming = iprot.readBitSet(3);
if (incoming.get(0)) {
{
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<ByteBuffer,ByteBuffer>(2*_map84.size);
ByteBuffer _key85;
ByteBuffer _val86;
for (int _i87 = 0; _i87 < _map84.size; ++_i87)
{
_key85 = iprot.readBinary();
_val86 = iprot.readBinary();
struct.attributes.put(_key85, _val86);
}
}
struct.setAttributesIsSet(true);
}
if (incoming.get(1)) {
struct.durability = org.apache.hadoop.hbase.thrift2.generated.TDurability.findByValue(iprot.readI32());
struct.setDurabilityIsSet(true);
}
if (incoming.get(2)) {
struct.cellVisibility = new TCellVisibility();
struct.cellVisibility.read(iprot);
struct.setCellVisibilityIsSet(true);
}
}
}
}

View File

@ -0,0 +1,373 @@
/**
* Autogenerated by Thrift Compiler (0.9.3)
*
* DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
* @generated
*/
package org.apache.hadoop.hbase.thrift2.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 javax.annotation.Generated;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
/**
* Atomic mutation for the specified row. It can be either Put or Delete.
*/
public class TMutation extends org.apache.thrift.TUnion<TMutation, TMutation._Fields> {
private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TMutation");
private static final org.apache.thrift.protocol.TField PUT_FIELD_DESC = new org.apache.thrift.protocol.TField("put", org.apache.thrift.protocol.TType.STRUCT, (short)1);
private static final org.apache.thrift.protocol.TField DELETE_SINGLE_FIELD_DESC = new org.apache.thrift.protocol.TField("deleteSingle", org.apache.thrift.protocol.TType.STRUCT, (short)2);
/** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
public enum _Fields implements org.apache.thrift.TFieldIdEnum {
PUT((short)1, "put"),
DELETE_SINGLE((short)2, "deleteSingle");
private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
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: // PUT
return PUT;
case 2: // DELETE_SINGLE
return DELETE_SINGLE;
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);
tmpMap.put(_Fields.PUT, new org.apache.thrift.meta_data.FieldMetaData("put", org.apache.thrift.TFieldRequirementType.DEFAULT,
new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TPut.class)));
tmpMap.put(_Fields.DELETE_SINGLE, new org.apache.thrift.meta_data.FieldMetaData("deleteSingle", org.apache.thrift.TFieldRequirementType.DEFAULT,
new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TDelete.class)));
metaDataMap = Collections.unmodifiableMap(tmpMap);
org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TMutation.class, metaDataMap);
}
public TMutation() {
super();
}
public TMutation(_Fields setField, Object value) {
super(setField, value);
}
public TMutation(TMutation other) {
super(other);
}
public TMutation deepCopy() {
return new TMutation(this);
}
public static TMutation put(TPut value) {
TMutation x = new TMutation();
x.setPut(value);
return x;
}
public static TMutation deleteSingle(TDelete value) {
TMutation x = new TMutation();
x.setDeleteSingle(value);
return x;
}
@Override
protected void checkType(_Fields setField, Object value) throws ClassCastException {
switch (setField) {
case PUT:
if (value instanceof TPut) {
break;
}
throw new ClassCastException("Was expecting value of type TPut for field 'put', but got " + value.getClass().getSimpleName());
case DELETE_SINGLE:
if (value instanceof TDelete) {
break;
}
throw new ClassCastException("Was expecting value of type TDelete for field 'deleteSingle', but got " + value.getClass().getSimpleName());
default:
throw new IllegalArgumentException("Unknown field id " + setField);
}
}
@Override
protected Object standardSchemeReadValue(org.apache.thrift.protocol.TProtocol iprot, org.apache.thrift.protocol.TField field) throws org.apache.thrift.TException {
_Fields setField = _Fields.findByThriftId(field.id);
if (setField != null) {
switch (setField) {
case PUT:
if (field.type == PUT_FIELD_DESC.type) {
TPut put;
put = new TPut();
put.read(iprot);
return put;
} else {
org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
return null;
}
case DELETE_SINGLE:
if (field.type == DELETE_SINGLE_FIELD_DESC.type) {
TDelete deleteSingle;
deleteSingle = new TDelete();
deleteSingle.read(iprot);
return deleteSingle;
} else {
org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
return null;
}
default:
throw new IllegalStateException("setField wasn't null, but didn't match any of the case statements!");
}
} else {
org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
return null;
}
}
@Override
protected void standardSchemeWriteValue(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
switch (setField_) {
case PUT:
TPut put = (TPut)value_;
put.write(oprot);
return;
case DELETE_SINGLE:
TDelete deleteSingle = (TDelete)value_;
deleteSingle.write(oprot);
return;
default:
throw new IllegalStateException("Cannot write union with unknown field " + setField_);
}
}
@Override
protected Object tupleSchemeReadValue(org.apache.thrift.protocol.TProtocol iprot, short fieldID) throws org.apache.thrift.TException {
_Fields setField = _Fields.findByThriftId(fieldID);
if (setField != null) {
switch (setField) {
case PUT:
TPut put;
put = new TPut();
put.read(iprot);
return put;
case DELETE_SINGLE:
TDelete deleteSingle;
deleteSingle = new TDelete();
deleteSingle.read(iprot);
return deleteSingle;
default:
throw new IllegalStateException("setField wasn't null, but didn't match any of the case statements!");
}
} else {
throw new TProtocolException("Couldn't find a field with field id " + fieldID);
}
}
@Override
protected void tupleSchemeWriteValue(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
switch (setField_) {
case PUT:
TPut put = (TPut)value_;
put.write(oprot);
return;
case DELETE_SINGLE:
TDelete deleteSingle = (TDelete)value_;
deleteSingle.write(oprot);
return;
default:
throw new IllegalStateException("Cannot write union with unknown field " + setField_);
}
}
@Override
protected org.apache.thrift.protocol.TField getFieldDesc(_Fields setField) {
switch (setField) {
case PUT:
return PUT_FIELD_DESC;
case DELETE_SINGLE:
return DELETE_SINGLE_FIELD_DESC;
default:
throw new IllegalArgumentException("Unknown field id " + setField);
}
}
@Override
protected org.apache.thrift.protocol.TStruct getStructDesc() {
return STRUCT_DESC;
}
@Override
protected _Fields enumForId(short id) {
return _Fields.findByThriftIdOrThrow(id);
}
public _Fields fieldForId(int fieldId) {
return _Fields.findByThriftId(fieldId);
}
public TPut getPut() {
if (getSetField() == _Fields.PUT) {
return (TPut)getFieldValue();
} else {
throw new RuntimeException("Cannot get field 'put' because union is currently set to " + getFieldDesc(getSetField()).name);
}
}
public void setPut(TPut value) {
if (value == null) throw new NullPointerException();
setField_ = _Fields.PUT;
value_ = value;
}
public TDelete getDeleteSingle() {
if (getSetField() == _Fields.DELETE_SINGLE) {
return (TDelete)getFieldValue();
} else {
throw new RuntimeException("Cannot get field 'deleteSingle' because union is currently set to " + getFieldDesc(getSetField()).name);
}
}
public void setDeleteSingle(TDelete value) {
if (value == null) throw new NullPointerException();
setField_ = _Fields.DELETE_SINGLE;
value_ = value;
}
public boolean isSetPut() {
return setField_ == _Fields.PUT;
}
public boolean isSetDeleteSingle() {
return setField_ == _Fields.DELETE_SINGLE;
}
public boolean equals(Object other) {
if (other instanceof TMutation) {
return equals((TMutation)other);
} else {
return false;
}
}
public boolean equals(TMutation other) {
return other != null && getSetField() == other.getSetField() && getFieldValue().equals(other.getFieldValue());
}
@Override
public int compareTo(TMutation other) {
int lastComparison = org.apache.thrift.TBaseHelper.compareTo(getSetField(), other.getSetField());
if (lastComparison == 0) {
return org.apache.thrift.TBaseHelper.compareTo(getFieldValue(), other.getFieldValue());
}
return lastComparison;
}
@Override
public int hashCode() {
List<Object> list = new ArrayList<Object>();
list.add(this.getClass().getName());
org.apache.thrift.TFieldIdEnum setField = getSetField();
if (setField != null) {
list.add(setField.getThriftFieldId());
Object value = getFieldValue();
if (value instanceof org.apache.thrift.TEnum) {
list.add(((org.apache.thrift.TEnum)getFieldValue()).getValue());
} else {
list.add(value);
}
}
return list.hashCode();
}
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);
}
}
}

File diff suppressed because it is too large Load Diff

View File

@ -0,0 +1,569 @@
/**
* Autogenerated by Thrift Compiler (0.9.3)
*
* DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
* @generated
*/
package org.apache.hadoop.hbase.thrift2.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 javax.annotation.Generated;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
/**
* if no Result is found, row and columnValues will not be set.
*/
@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2016-05-25")
public class TResult implements org.apache.thrift.TBase<TResult, TResult._Fields>, java.io.Serializable, Cloneable, Comparable<TResult> {
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);
private static final org.apache.thrift.protocol.TField COLUMN_VALUES_FIELD_DESC = new org.apache.thrift.protocol.TField("columnValues", org.apache.thrift.protocol.TType.LIST, (short)2);
private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
static {
schemes.put(StandardScheme.class, new TResultStandardSchemeFactory());
schemes.put(TupleScheme.class, new TResultTupleSchemeFactory());
}
public ByteBuffer row; // optional
public List<TColumnValue> columnValues; // 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"),
COLUMN_VALUES((short)2, "columnValues");
private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
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: // COLUMN_VALUES
return COLUMN_VALUES;
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 _Fields optionals[] = {_Fields.ROW};
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.OPTIONAL,
new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , true)));
tmpMap.put(_Fields.COLUMN_VALUES, new org.apache.thrift.meta_data.FieldMetaData("columnValues", 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(TResult.class, metaDataMap);
}
public TResult() {
}
public TResult(
List<TColumnValue> columnValues)
{
this();
this.columnValues = columnValues;
}
/**
* Performs a deep copy on <i>other</i>.
*/
public TResult(TResult other) {
if (other.isSetRow()) {
this.row = org.apache.thrift.TBaseHelper.copyBinary(other.row);
}
if (other.isSetColumnValues()) {
List<TColumnValue> __this__columnValues = new ArrayList<TColumnValue>(other.columnValues.size());
for (TColumnValue other_element : other.columnValues) {
__this__columnValues.add(new TColumnValue(other_element));
}
this.columnValues = __this__columnValues;
}
}
public TResult deepCopy() {
return new TResult(this);
}
@Override
public void clear() {
this.row = null;
this.columnValues = null;
}
public byte[] getRow() {
setRow(org.apache.thrift.TBaseHelper.rightSize(row));
return row == null ? null : row.array();
}
public ByteBuffer bufferForRow() {
return org.apache.thrift.TBaseHelper.copyBinary(row);
}
public TResult setRow(byte[] row) {
this.row = row == null ? (ByteBuffer)null : ByteBuffer.wrap(Arrays.copyOf(row, row.length));
return this;
}
public TResult setRow(ByteBuffer row) {
this.row = org.apache.thrift.TBaseHelper.copyBinary(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 int getColumnValuesSize() {
return (this.columnValues == null) ? 0 : this.columnValues.size();
}
public java.util.Iterator<TColumnValue> getColumnValuesIterator() {
return (this.columnValues == null) ? null : this.columnValues.iterator();
}
public void addToColumnValues(TColumnValue elem) {
if (this.columnValues == null) {
this.columnValues = new ArrayList<TColumnValue>();
}
this.columnValues.add(elem);
}
public List<TColumnValue> getColumnValues() {
return this.columnValues;
}
public TResult setColumnValues(List<TColumnValue> columnValues) {
this.columnValues = columnValues;
return this;
}
public void unsetColumnValues() {
this.columnValues = null;
}
/** Returns true if field columnValues is set (has been assigned a value) and false otherwise */
public boolean isSetColumnValues() {
return this.columnValues != null;
}
public void setColumnValuesIsSet(boolean value) {
if (!value) {
this.columnValues = null;
}
}
public void setFieldValue(_Fields field, Object value) {
switch (field) {
case ROW:
if (value == null) {
unsetRow();
} else {
setRow((ByteBuffer)value);
}
break;
case COLUMN_VALUES:
if (value == null) {
unsetColumnValues();
} else {
setColumnValues((List<TColumnValue>)value);
}
break;
}
}
public Object getFieldValue(_Fields field) {
switch (field) {
case ROW:
return getRow();
case COLUMN_VALUES:
return getColumnValues();
}
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 COLUMN_VALUES:
return isSetColumnValues();
}
throw new IllegalStateException();
}
@Override
public boolean equals(Object that) {
if (that == null)
return false;
if (that instanceof TResult)
return this.equals((TResult)that);
return false;
}
public boolean equals(TResult 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_columnValues = true && this.isSetColumnValues();
boolean that_present_columnValues = true && that.isSetColumnValues();
if (this_present_columnValues || that_present_columnValues) {
if (!(this_present_columnValues && that_present_columnValues))
return false;
if (!this.columnValues.equals(that.columnValues))
return false;
}
return true;
}
@Override
public int hashCode() {
List<Object> list = new ArrayList<Object>();
boolean present_row = true && (isSetRow());
list.add(present_row);
if (present_row)
list.add(row);
boolean present_columnValues = true && (isSetColumnValues());
list.add(present_columnValues);
if (present_columnValues)
list.add(columnValues);
return list.hashCode();
}
@Override
public int compareTo(TResult 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(isSetColumnValues()).compareTo(other.isSetColumnValues());
if (lastComparison != 0) {
return lastComparison;
}
if (isSetColumnValues()) {
lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.columnValues, other.columnValues);
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("TResult(");
boolean first = true;
if (isSetRow()) {
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("columnValues:");
if (this.columnValues == null) {
sb.append("null");
} else {
sb.append(this.columnValues);
}
first = false;
sb.append(")");
return sb.toString();
}
public void validate() throws org.apache.thrift.TException {
// check for required fields
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 {
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 TResultStandardSchemeFactory implements SchemeFactory {
public TResultStandardScheme getScheme() {
return new TResultStandardScheme();
}
}
private static class TResultStandardScheme extends StandardScheme<TResult> {
public void read(org.apache.thrift.protocol.TProtocol iprot, TResult 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: // COLUMN_VALUES
if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
{
org.apache.thrift.protocol.TList _list0 = iprot.readListBegin();
struct.columnValues = new ArrayList<TColumnValue>(_list0.size);
TColumnValue _elem1;
for (int _i2 = 0; _i2 < _list0.size; ++_i2)
{
_elem1 = new TColumnValue();
_elem1.read(iprot);
struct.columnValues.add(_elem1);
}
iprot.readListEnd();
}
struct.setColumnValuesIsSet(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, TResult struct) throws org.apache.thrift.TException {
struct.validate();
oprot.writeStructBegin(STRUCT_DESC);
if (struct.row != null) {
if (struct.isSetRow()) {
oprot.writeFieldBegin(ROW_FIELD_DESC);
oprot.writeBinary(struct.row);
oprot.writeFieldEnd();
}
}
if (struct.columnValues != null) {
oprot.writeFieldBegin(COLUMN_VALUES_FIELD_DESC);
{
oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.columnValues.size()));
for (TColumnValue _iter3 : struct.columnValues)
{
_iter3.write(oprot);
}
oprot.writeListEnd();
}
oprot.writeFieldEnd();
}
oprot.writeFieldStop();
oprot.writeStructEnd();
}
}
private static class TResultTupleSchemeFactory implements SchemeFactory {
public TResultTupleScheme getScheme() {
return new TResultTupleScheme();
}
}
private static class TResultTupleScheme extends TupleScheme<TResult> {
@Override
public void write(org.apache.thrift.protocol.TProtocol prot, TResult struct) throws org.apache.thrift.TException {
TTupleProtocol oprot = (TTupleProtocol) prot;
{
oprot.writeI32(struct.columnValues.size());
for (TColumnValue _iter4 : struct.columnValues)
{
_iter4.write(oprot);
}
}
BitSet optionals = new BitSet();
if (struct.isSetRow()) {
optionals.set(0);
}
oprot.writeBitSet(optionals, 1);
if (struct.isSetRow()) {
oprot.writeBinary(struct.row);
}
}
@Override
public void read(org.apache.thrift.protocol.TProtocol prot, TResult 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.columnValues = new ArrayList<TColumnValue>(_list5.size);
TColumnValue _elem6;
for (int _i7 = 0; _i7 < _list5.size; ++_i7)
{
_elem6 = new TColumnValue();
_elem6.read(iprot);
struct.columnValues.add(_elem6);
}
}
struct.setColumnValuesIsSet(true);
BitSet incoming = iprot.readBitSet(1);
if (incoming.get(0)) {
struct.row = iprot.readBinary();
struct.setRowIsSet(true);
}
}
}
}

View File

@ -0,0 +1,559 @@
/**
* Autogenerated by Thrift Compiler (0.9.3)
*
* DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
* @generated
*/
package org.apache.hadoop.hbase.thrift2.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 javax.annotation.Generated;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
/**
* A TRowMutations object is used to apply a number of Mutations to a single row.
*/
@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2016-05-25")
public class TRowMutations implements org.apache.thrift.TBase<TRowMutations, TRowMutations._Fields>, java.io.Serializable, Cloneable, Comparable<TRowMutations> {
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);
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)2);
private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
static {
schemes.put(StandardScheme.class, new TRowMutationsStandardSchemeFactory());
schemes.put(TupleScheme.class, new TRowMutationsTupleSchemeFactory());
}
public ByteBuffer row; // required
public List<TMutation> 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 {
ROW((short)1, "row"),
MUTATIONS((short)2, "mutations");
private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
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: // 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.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.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, TMutation.class))));
metaDataMap = Collections.unmodifiableMap(tmpMap);
org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TRowMutations.class, metaDataMap);
}
public TRowMutations() {
}
public TRowMutations(
ByteBuffer row,
List<TMutation> mutations)
{
this();
this.row = org.apache.thrift.TBaseHelper.copyBinary(row);
this.mutations = mutations;
}
/**
* Performs a deep copy on <i>other</i>.
*/
public TRowMutations(TRowMutations other) {
if (other.isSetRow()) {
this.row = org.apache.thrift.TBaseHelper.copyBinary(other.row);
}
if (other.isSetMutations()) {
List<TMutation> __this__mutations = new ArrayList<TMutation>(other.mutations.size());
for (TMutation other_element : other.mutations) {
__this__mutations.add(new TMutation(other_element));
}
this.mutations = __this__mutations;
}
}
public TRowMutations deepCopy() {
return new TRowMutations(this);
}
@Override
public void clear() {
this.row = null;
this.mutations = null;
}
public byte[] getRow() {
setRow(org.apache.thrift.TBaseHelper.rightSize(row));
return row == null ? null : row.array();
}
public ByteBuffer bufferForRow() {
return org.apache.thrift.TBaseHelper.copyBinary(row);
}
public TRowMutations setRow(byte[] row) {
this.row = row == null ? (ByteBuffer)null : ByteBuffer.wrap(Arrays.copyOf(row, row.length));
return this;
}
public TRowMutations setRow(ByteBuffer row) {
this.row = org.apache.thrift.TBaseHelper.copyBinary(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 int getMutationsSize() {
return (this.mutations == null) ? 0 : this.mutations.size();
}
public java.util.Iterator<TMutation> getMutationsIterator() {
return (this.mutations == null) ? null : this.mutations.iterator();
}
public void addToMutations(TMutation elem) {
if (this.mutations == null) {
this.mutations = new ArrayList<TMutation>();
}
this.mutations.add(elem);
}
public List<TMutation> getMutations() {
return this.mutations;
}
public TRowMutations setMutations(List<TMutation> 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 ROW:
if (value == null) {
unsetRow();
} else {
setRow((ByteBuffer)value);
}
break;
case MUTATIONS:
if (value == null) {
unsetMutations();
} else {
setMutations((List<TMutation>)value);
}
break;
}
}
public Object getFieldValue(_Fields field) {
switch (field) {
case ROW:
return getRow();
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 ROW:
return isSetRow();
case MUTATIONS:
return isSetMutations();
}
throw new IllegalStateException();
}
@Override
public boolean equals(Object that) {
if (that == null)
return false;
if (that instanceof TRowMutations)
return this.equals((TRowMutations)that);
return false;
}
public boolean equals(TRowMutations 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_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() {
List<Object> list = new ArrayList<Object>();
boolean present_row = true && (isSetRow());
list.add(present_row);
if (present_row)
list.add(row);
boolean present_mutations = true && (isSetMutations());
list.add(present_mutations);
if (present_mutations)
list.add(mutations);
return list.hashCode();
}
@Override
public int compareTo(TRowMutations 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(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("TRowMutations(");
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("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 (row == null) {
throw new org.apache.thrift.protocol.TProtocolException("Required field 'row' was not present! Struct: " + toString());
}
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 TRowMutationsStandardSchemeFactory implements SchemeFactory {
public TRowMutationsStandardScheme getScheme() {
return new TRowMutationsStandardScheme();
}
}
private static class TRowMutationsStandardScheme extends StandardScheme<TRowMutations> {
public void read(org.apache.thrift.protocol.TProtocol iprot, TRowMutations 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: // MUTATIONS
if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
{
org.apache.thrift.protocol.TList _list124 = iprot.readListBegin();
struct.mutations = new ArrayList<TMutation>(_list124.size);
TMutation _elem125;
for (int _i126 = 0; _i126 < _list124.size; ++_i126)
{
_elem125 = new TMutation();
_elem125.read(iprot);
struct.mutations.add(_elem125);
}
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, TRowMutations 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.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 (TMutation _iter127 : struct.mutations)
{
_iter127.write(oprot);
}
oprot.writeListEnd();
}
oprot.writeFieldEnd();
}
oprot.writeFieldStop();
oprot.writeStructEnd();
}
}
private static class TRowMutationsTupleSchemeFactory implements SchemeFactory {
public TRowMutationsTupleScheme getScheme() {
return new TRowMutationsTupleScheme();
}
}
private static class TRowMutationsTupleScheme extends TupleScheme<TRowMutations> {
@Override
public void write(org.apache.thrift.protocol.TProtocol prot, TRowMutations struct) throws org.apache.thrift.TException {
TTupleProtocol oprot = (TTupleProtocol) prot;
oprot.writeBinary(struct.row);
{
oprot.writeI32(struct.mutations.size());
for (TMutation _iter128 : struct.mutations)
{
_iter128.write(oprot);
}
}
}
@Override
public void read(org.apache.thrift.protocol.TProtocol prot, TRowMutations struct) throws org.apache.thrift.TException {
TTupleProtocol iprot = (TTupleProtocol) prot;
struct.row = iprot.readBinary();
struct.setRowIsSet(true);
{
org.apache.thrift.protocol.TList _list129 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
struct.mutations = new ArrayList<TMutation>(_list129.size);
TMutation _elem130;
for (int _i131 = 0; _i131 < _list129.size; ++_i131)
{
_elem130 = new TMutation();
_elem130.read(iprot);
struct.mutations.add(_elem130);
}
}
struct.setMutationsIsSet(true);
}
}
}

File diff suppressed because it is too large Load Diff

View File

@ -0,0 +1,599 @@
/**
* Autogenerated by Thrift Compiler (0.9.3)
*
* DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
* @generated
*/
package org.apache.hadoop.hbase.thrift2.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 javax.annotation.Generated;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2016-05-25")
public class TServerName implements org.apache.thrift.TBase<TServerName, TServerName._Fields>, java.io.Serializable, Cloneable, Comparable<TServerName> {
private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TServerName");
private static final org.apache.thrift.protocol.TField HOST_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("hostName", org.apache.thrift.protocol.TType.STRING, (short)1);
private static final org.apache.thrift.protocol.TField PORT_FIELD_DESC = new org.apache.thrift.protocol.TField("port", org.apache.thrift.protocol.TType.I32, (short)2);
private static final org.apache.thrift.protocol.TField START_CODE_FIELD_DESC = new org.apache.thrift.protocol.TField("startCode", org.apache.thrift.protocol.TType.I64, (short)3);
private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
static {
schemes.put(StandardScheme.class, new TServerNameStandardSchemeFactory());
schemes.put(TupleScheme.class, new TServerNameTupleSchemeFactory());
}
public String hostName; // required
public int port; // optional
public long startCode; // 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 {
HOST_NAME((short)1, "hostName"),
PORT((short)2, "port"),
START_CODE((short)3, "startCode");
private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
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: // HOST_NAME
return HOST_NAME;
case 2: // PORT
return PORT;
case 3: // START_CODE
return START_CODE;
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 __PORT_ISSET_ID = 0;
private static final int __STARTCODE_ISSET_ID = 1;
private byte __isset_bitfield = 0;
private static final _Fields optionals[] = {_Fields.PORT,_Fields.START_CODE};
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.HOST_NAME, new org.apache.thrift.meta_data.FieldMetaData("hostName", org.apache.thrift.TFieldRequirementType.REQUIRED,
new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
tmpMap.put(_Fields.PORT, new org.apache.thrift.meta_data.FieldMetaData("port", org.apache.thrift.TFieldRequirementType.OPTIONAL,
new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
tmpMap.put(_Fields.START_CODE, new org.apache.thrift.meta_data.FieldMetaData("startCode", org.apache.thrift.TFieldRequirementType.OPTIONAL,
new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
metaDataMap = Collections.unmodifiableMap(tmpMap);
org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TServerName.class, metaDataMap);
}
public TServerName() {
}
public TServerName(
String hostName)
{
this();
this.hostName = hostName;
}
/**
* Performs a deep copy on <i>other</i>.
*/
public TServerName(TServerName other) {
__isset_bitfield = other.__isset_bitfield;
if (other.isSetHostName()) {
this.hostName = other.hostName;
}
this.port = other.port;
this.startCode = other.startCode;
}
public TServerName deepCopy() {
return new TServerName(this);
}
@Override
public void clear() {
this.hostName = null;
setPortIsSet(false);
this.port = 0;
setStartCodeIsSet(false);
this.startCode = 0;
}
public String getHostName() {
return this.hostName;
}
public TServerName setHostName(String hostName) {
this.hostName = hostName;
return this;
}
public void unsetHostName() {
this.hostName = null;
}
/** Returns true if field hostName is set (has been assigned a value) and false otherwise */
public boolean isSetHostName() {
return this.hostName != null;
}
public void setHostNameIsSet(boolean value) {
if (!value) {
this.hostName = null;
}
}
public int getPort() {
return this.port;
}
public TServerName setPort(int port) {
this.port = port;
setPortIsSet(true);
return this;
}
public void unsetPort() {
__isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __PORT_ISSET_ID);
}
/** Returns true if field port is set (has been assigned a value) and false otherwise */
public boolean isSetPort() {
return EncodingUtils.testBit(__isset_bitfield, __PORT_ISSET_ID);
}
public void setPortIsSet(boolean value) {
__isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __PORT_ISSET_ID, value);
}
public long getStartCode() {
return this.startCode;
}
public TServerName setStartCode(long startCode) {
this.startCode = startCode;
setStartCodeIsSet(true);
return this;
}
public void unsetStartCode() {
__isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __STARTCODE_ISSET_ID);
}
/** Returns true if field startCode is set (has been assigned a value) and false otherwise */
public boolean isSetStartCode() {
return EncodingUtils.testBit(__isset_bitfield, __STARTCODE_ISSET_ID);
}
public void setStartCodeIsSet(boolean value) {
__isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __STARTCODE_ISSET_ID, value);
}
public void setFieldValue(_Fields field, Object value) {
switch (field) {
case HOST_NAME:
if (value == null) {
unsetHostName();
} else {
setHostName((String)value);
}
break;
case PORT:
if (value == null) {
unsetPort();
} else {
setPort((Integer)value);
}
break;
case START_CODE:
if (value == null) {
unsetStartCode();
} else {
setStartCode((Long)value);
}
break;
}
}
public Object getFieldValue(_Fields field) {
switch (field) {
case HOST_NAME:
return getHostName();
case PORT:
return getPort();
case START_CODE:
return getStartCode();
}
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 HOST_NAME:
return isSetHostName();
case PORT:
return isSetPort();
case START_CODE:
return isSetStartCode();
}
throw new IllegalStateException();
}
@Override
public boolean equals(Object that) {
if (that == null)
return false;
if (that instanceof TServerName)
return this.equals((TServerName)that);
return false;
}
public boolean equals(TServerName that) {
if (that == null)
return false;
boolean this_present_hostName = true && this.isSetHostName();
boolean that_present_hostName = true && that.isSetHostName();
if (this_present_hostName || that_present_hostName) {
if (!(this_present_hostName && that_present_hostName))
return false;
if (!this.hostName.equals(that.hostName))
return false;
}
boolean this_present_port = true && this.isSetPort();
boolean that_present_port = true && that.isSetPort();
if (this_present_port || that_present_port) {
if (!(this_present_port && that_present_port))
return false;
if (this.port != that.port)
return false;
}
boolean this_present_startCode = true && this.isSetStartCode();
boolean that_present_startCode = true && that.isSetStartCode();
if (this_present_startCode || that_present_startCode) {
if (!(this_present_startCode && that_present_startCode))
return false;
if (this.startCode != that.startCode)
return false;
}
return true;
}
@Override
public int hashCode() {
List<Object> list = new ArrayList<Object>();
boolean present_hostName = true && (isSetHostName());
list.add(present_hostName);
if (present_hostName)
list.add(hostName);
boolean present_port = true && (isSetPort());
list.add(present_port);
if (present_port)
list.add(port);
boolean present_startCode = true && (isSetStartCode());
list.add(present_startCode);
if (present_startCode)
list.add(startCode);
return list.hashCode();
}
@Override
public int compareTo(TServerName other) {
if (!getClass().equals(other.getClass())) {
return getClass().getName().compareTo(other.getClass().getName());
}
int lastComparison = 0;
lastComparison = Boolean.valueOf(isSetHostName()).compareTo(other.isSetHostName());
if (lastComparison != 0) {
return lastComparison;
}
if (isSetHostName()) {
lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.hostName, other.hostName);
if (lastComparison != 0) {
return lastComparison;
}
}
lastComparison = Boolean.valueOf(isSetPort()).compareTo(other.isSetPort());
if (lastComparison != 0) {
return lastComparison;
}
if (isSetPort()) {
lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.port, other.port);
if (lastComparison != 0) {
return lastComparison;
}
}
lastComparison = Boolean.valueOf(isSetStartCode()).compareTo(other.isSetStartCode());
if (lastComparison != 0) {
return lastComparison;
}
if (isSetStartCode()) {
lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.startCode, other.startCode);
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("TServerName(");
boolean first = true;
sb.append("hostName:");
if (this.hostName == null) {
sb.append("null");
} else {
sb.append(this.hostName);
}
first = false;
if (isSetPort()) {
if (!first) sb.append(", ");
sb.append("port:");
sb.append(this.port);
first = false;
}
if (isSetStartCode()) {
if (!first) sb.append(", ");
sb.append("startCode:");
sb.append(this.startCode);
first = false;
}
sb.append(")");
return sb.toString();
}
public void validate() throws org.apache.thrift.TException {
// check for required fields
if (hostName == null) {
throw new org.apache.thrift.protocol.TProtocolException("Required field 'hostName' 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 {
// 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 TServerNameStandardSchemeFactory implements SchemeFactory {
public TServerNameStandardScheme getScheme() {
return new TServerNameStandardScheme();
}
}
private static class TServerNameStandardScheme extends StandardScheme<TServerName> {
public void read(org.apache.thrift.protocol.TProtocol iprot, TServerName 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: // HOST_NAME
if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
struct.hostName = iprot.readString();
struct.setHostNameIsSet(true);
} else {
org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
}
break;
case 2: // PORT
if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
struct.port = iprot.readI32();
struct.setPortIsSet(true);
} else {
org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
}
break;
case 3: // START_CODE
if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
struct.startCode = iprot.readI64();
struct.setStartCodeIsSet(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, TServerName struct) throws org.apache.thrift.TException {
struct.validate();
oprot.writeStructBegin(STRUCT_DESC);
if (struct.hostName != null) {
oprot.writeFieldBegin(HOST_NAME_FIELD_DESC);
oprot.writeString(struct.hostName);
oprot.writeFieldEnd();
}
if (struct.isSetPort()) {
oprot.writeFieldBegin(PORT_FIELD_DESC);
oprot.writeI32(struct.port);
oprot.writeFieldEnd();
}
if (struct.isSetStartCode()) {
oprot.writeFieldBegin(START_CODE_FIELD_DESC);
oprot.writeI64(struct.startCode);
oprot.writeFieldEnd();
}
oprot.writeFieldStop();
oprot.writeStructEnd();
}
}
private static class TServerNameTupleSchemeFactory implements SchemeFactory {
public TServerNameTupleScheme getScheme() {
return new TServerNameTupleScheme();
}
}
private static class TServerNameTupleScheme extends TupleScheme<TServerName> {
@Override
public void write(org.apache.thrift.protocol.TProtocol prot, TServerName struct) throws org.apache.thrift.TException {
TTupleProtocol oprot = (TTupleProtocol) prot;
oprot.writeString(struct.hostName);
BitSet optionals = new BitSet();
if (struct.isSetPort()) {
optionals.set(0);
}
if (struct.isSetStartCode()) {
optionals.set(1);
}
oprot.writeBitSet(optionals, 2);
if (struct.isSetPort()) {
oprot.writeI32(struct.port);
}
if (struct.isSetStartCode()) {
oprot.writeI64(struct.startCode);
}
}
@Override
public void read(org.apache.thrift.protocol.TProtocol prot, TServerName struct) throws org.apache.thrift.TException {
TTupleProtocol iprot = (TTupleProtocol) prot;
struct.hostName = iprot.readString();
struct.setHostNameIsSet(true);
BitSet incoming = iprot.readBitSet(2);
if (incoming.get(0)) {
struct.port = iprot.readI32();
struct.setPortIsSet(true);
}
if (incoming.get(1)) {
struct.startCode = iprot.readI64();
struct.setStartCodeIsSet(true);
}
}
}
}

View File

@ -0,0 +1,486 @@
/**
* Autogenerated by Thrift Compiler (0.9.3)
*
* DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
* @generated
*/
package org.apache.hadoop.hbase.thrift2.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 javax.annotation.Generated;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2016-05-25")
public class TTimeRange implements org.apache.thrift.TBase<TTimeRange, TTimeRange._Fields>, java.io.Serializable, Cloneable, Comparable<TTimeRange> {
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);
private static final org.apache.thrift.protocol.TField MAX_STAMP_FIELD_DESC = new org.apache.thrift.protocol.TField("maxStamp", org.apache.thrift.protocol.TType.I64, (short)2);
private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
static {
schemes.put(StandardScheme.class, new TTimeRangeStandardSchemeFactory());
schemes.put(TupleScheme.class, new TTimeRangeTupleSchemeFactory());
}
public long minStamp; // required
public long maxStamp; // 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 {
MIN_STAMP((short)1, "minStamp"),
MAX_STAMP((short)2, "maxStamp");
private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
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: // MIN_STAMP
return MIN_STAMP;
case 2: // MAX_STAMP
return MAX_STAMP;
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 __MINSTAMP_ISSET_ID = 0;
private static final int __MAXSTAMP_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.MIN_STAMP, new org.apache.thrift.meta_data.FieldMetaData("minStamp", org.apache.thrift.TFieldRequirementType.REQUIRED,
new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
tmpMap.put(_Fields.MAX_STAMP, new org.apache.thrift.meta_data.FieldMetaData("maxStamp", 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(TTimeRange.class, metaDataMap);
}
public TTimeRange() {
}
public TTimeRange(
long minStamp,
long maxStamp)
{
this();
this.minStamp = minStamp;
setMinStampIsSet(true);
this.maxStamp = maxStamp;
setMaxStampIsSet(true);
}
/**
* Performs a deep copy on <i>other</i>.
*/
public TTimeRange(TTimeRange other) {
__isset_bitfield = other.__isset_bitfield;
this.minStamp = other.minStamp;
this.maxStamp = other.maxStamp;
}
public TTimeRange deepCopy() {
return new TTimeRange(this);
}
@Override
public void clear() {
setMinStampIsSet(false);
this.minStamp = 0;
setMaxStampIsSet(false);
this.maxStamp = 0;
}
public long getMinStamp() {
return this.minStamp;
}
public TTimeRange setMinStamp(long minStamp) {
this.minStamp = minStamp;
setMinStampIsSet(true);
return this;
}
public void unsetMinStamp() {
__isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __MINSTAMP_ISSET_ID);
}
/** Returns true if field minStamp is set (has been assigned a value) and false otherwise */
public boolean isSetMinStamp() {
return EncodingUtils.testBit(__isset_bitfield, __MINSTAMP_ISSET_ID);
}
public void setMinStampIsSet(boolean value) {
__isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __MINSTAMP_ISSET_ID, value);
}
public long getMaxStamp() {
return this.maxStamp;
}
public TTimeRange setMaxStamp(long maxStamp) {
this.maxStamp = maxStamp;
setMaxStampIsSet(true);
return this;
}
public void unsetMaxStamp() {
__isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __MAXSTAMP_ISSET_ID);
}
/** Returns true if field maxStamp is set (has been assigned a value) and false otherwise */
public boolean isSetMaxStamp() {
return EncodingUtils.testBit(__isset_bitfield, __MAXSTAMP_ISSET_ID);
}
public void setMaxStampIsSet(boolean value) {
__isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __MAXSTAMP_ISSET_ID, value);
}
public void setFieldValue(_Fields field, Object value) {
switch (field) {
case MIN_STAMP:
if (value == null) {
unsetMinStamp();
} else {
setMinStamp((Long)value);
}
break;
case MAX_STAMP:
if (value == null) {
unsetMaxStamp();
} else {
setMaxStamp((Long)value);
}
break;
}
}
public Object getFieldValue(_Fields field) {
switch (field) {
case MIN_STAMP:
return getMinStamp();
case MAX_STAMP:
return getMaxStamp();
}
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 MIN_STAMP:
return isSetMinStamp();
case MAX_STAMP:
return isSetMaxStamp();
}
throw new IllegalStateException();
}
@Override
public boolean equals(Object that) {
if (that == null)
return false;
if (that instanceof TTimeRange)
return this.equals((TTimeRange)that);
return false;
}
public boolean equals(TTimeRange that) {
if (that == null)
return false;
boolean this_present_minStamp = true;
boolean that_present_minStamp = true;
if (this_present_minStamp || that_present_minStamp) {
if (!(this_present_minStamp && that_present_minStamp))
return false;
if (this.minStamp != that.minStamp)
return false;
}
boolean this_present_maxStamp = true;
boolean that_present_maxStamp = true;
if (this_present_maxStamp || that_present_maxStamp) {
if (!(this_present_maxStamp && that_present_maxStamp))
return false;
if (this.maxStamp != that.maxStamp)
return false;
}
return true;
}
@Override
public int hashCode() {
List<Object> list = new ArrayList<Object>();
boolean present_minStamp = true;
list.add(present_minStamp);
if (present_minStamp)
list.add(minStamp);
boolean present_maxStamp = true;
list.add(present_maxStamp);
if (present_maxStamp)
list.add(maxStamp);
return list.hashCode();
}
@Override
public int compareTo(TTimeRange other) {
if (!getClass().equals(other.getClass())) {
return getClass().getName().compareTo(other.getClass().getName());
}
int lastComparison = 0;
lastComparison = Boolean.valueOf(isSetMinStamp()).compareTo(other.isSetMinStamp());
if (lastComparison != 0) {
return lastComparison;
}
if (isSetMinStamp()) {
lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.minStamp, other.minStamp);
if (lastComparison != 0) {
return lastComparison;
}
}
lastComparison = Boolean.valueOf(isSetMaxStamp()).compareTo(other.isSetMaxStamp());
if (lastComparison != 0) {
return lastComparison;
}
if (isSetMaxStamp()) {
lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.maxStamp, other.maxStamp);
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("TTimeRange(");
boolean first = true;
sb.append("minStamp:");
sb.append(this.minStamp);
first = false;
if (!first) sb.append(", ");
sb.append("maxStamp:");
sb.append(this.maxStamp);
first = false;
sb.append(")");
return sb.toString();
}
public void validate() throws org.apache.thrift.TException {
// check for required fields
// alas, we cannot check 'minStamp' because it's a primitive and you chose the non-beans generator.
// alas, we cannot check 'maxStamp' 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 TTimeRangeStandardSchemeFactory implements SchemeFactory {
public TTimeRangeStandardScheme getScheme() {
return new TTimeRangeStandardScheme();
}
}
private static class TTimeRangeStandardScheme extends StandardScheme<TTimeRange> {
public void read(org.apache.thrift.protocol.TProtocol iprot, TTimeRange 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: // MIN_STAMP
if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
struct.minStamp = iprot.readI64();
struct.setMinStampIsSet(true);
} else {
org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
}
break;
case 2: // MAX_STAMP
if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
struct.maxStamp = iprot.readI64();
struct.setMaxStampIsSet(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.isSetMinStamp()) {
throw new org.apache.thrift.protocol.TProtocolException("Required field 'minStamp' was not found in serialized data! Struct: " + toString());
}
if (!struct.isSetMaxStamp()) {
throw new org.apache.thrift.protocol.TProtocolException("Required field 'maxStamp' was not found in serialized data! Struct: " + toString());
}
struct.validate();
}
public void write(org.apache.thrift.protocol.TProtocol oprot, TTimeRange struct) throws org.apache.thrift.TException {
struct.validate();
oprot.writeStructBegin(STRUCT_DESC);
oprot.writeFieldBegin(MIN_STAMP_FIELD_DESC);
oprot.writeI64(struct.minStamp);
oprot.writeFieldEnd();
oprot.writeFieldBegin(MAX_STAMP_FIELD_DESC);
oprot.writeI64(struct.maxStamp);
oprot.writeFieldEnd();
oprot.writeFieldStop();
oprot.writeStructEnd();
}
}
private static class TTimeRangeTupleSchemeFactory implements SchemeFactory {
public TTimeRangeTupleScheme getScheme() {
return new TTimeRangeTupleScheme();
}
}
private static class TTimeRangeTupleScheme extends TupleScheme<TTimeRange> {
@Override
public void write(org.apache.thrift.protocol.TProtocol prot, TTimeRange struct) throws org.apache.thrift.TException {
TTupleProtocol oprot = (TTupleProtocol) prot;
oprot.writeI64(struct.minStamp);
oprot.writeI64(struct.maxStamp);
}
@Override
public void read(org.apache.thrift.protocol.TProtocol prot, TTimeRange struct) throws org.apache.thrift.TException {
TTupleProtocol iprot = (TTupleProtocol) prot;
struct.minStamp = iprot.readI64();
struct.setMinStampIsSet(true);
struct.maxStamp = iprot.readI64();
struct.setMaxStampIsSet(true);
}
}
}

View File

@ -0,0 +1,103 @@
<!DOCTYPE HTML PUBLIC "-//W3C//DTD HTML 3.2 Final//EN">
<html>
<!--
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.
-->
<head />
<body bgcolor="white">
Provides an HBase <a href="http://thrift.apache.org/">Thrift</a>
service.
This package contains a Thrift interface definition file for an HBase RPC
service and a Java server implementation.
There are currently 2 thrift server implementations in HBase, the packages:
<ul>
<li>org.apache.hadoop.hbase.thrift: This may one day be marked as depreceated.</li>
<li>org.apache.hadoop.hbase.thrift2: i.e. this package. This is intended to closely match to the HTable interface and
to one day supercede the older thrift (the old thrift mimics an API HBase no longer has).</li>
</ul>
<h2><a name="whatisthrift">What is Thrift?</a></h2>
<p>"Thrift is a software framework for scalable cross-language services
development. It combines a software stack with a code generation engine to
build services that work efficiently and seamlessly between C++, Java, Python,
PHP, Ruby, Erlang, Perl, Haskell, C#, Cocoa, JavaScript, Node.js, Smalltalk,
and OCaml. Originally developed at Facebook, Thrift was open sourced in April
2007 and entered the Apache Incubator in May, 2008".
From http://thrift.apache.org/</p>
<h2><a name="description">Description</a></h2>
<p>The HBase API is defined in the
file hbase.thrift. A server-side implementation of the API is in
<code>org.apache.hadoop.hbase.thrift2.ThriftHBaseServiceHandler</code> with the
server boiler plate in <code>org.apache.hadoop.hbase.thrift2.ThriftServer</code>.
The generated interfaces, types, and RPC utility files are checked into SVN under the
<code>org.apache.hadoop.hbase.thrift2.generated</code> directory.
</p>
<p>To stop, use:
<pre>
./bin/hbase-daemon.sh stop thrift2
</pre>
These are the command line arguments the Thrift server understands in addition to <code>start</code> and <code>stop</code>:
<dl>
<dt><code>-b, --bind</code></dt>
<dd>Address to bind the Thrift server to. Not supported by the Nonblocking and HsHa server [default: <code>0.0.0.0</code>]</dd>
<dt><code>-p, --port</code></dt>
<dd>Port to bind to [default: <code>9090</code>]</dd>
<dt><code>-f, --framed</code></dt>
<dd>Use framed transport (implied when using one of the non-blocking servers)</dd>
<dt><code>-c, --compact</code></dt>
<dd>Use the compact protocol [default: binary protocol]</dd>
<dt><code>-h, --help</code></dt>
<dd>Displays usage information for the Thrift server</dd>
<dt><code>-threadpool</code></dt>
<dd>Use the TThreadPoolServer. This is the default.</dd>
<dt><code>-hsha</code></dt>
<dd>Use the THsHaServer. This implies the framed transport.</dd>
<dt><code>-nonblocking</code></dt>
<dd>Use the TNonblockingServer. This implies the framed transport.</dd>
</dl>
<h3><a name="details">Details</a></h3>
<p>HBase currently uses version 0.9.0 of Apache Thrift.</p>
<p>The files were generated by running the commands under the hbase checkout dir:
<pre>
mvn compile -Pcompile-thrift
</pre>
<p>The 'thrift' binary is the Thrift compiler, and it is distributed separately from HBase
in a Thrift release. Additionally, specific language runtime libraries are a
part of a Thrift release. A version of the Java runtime is included in HBase via maven.
</p>
</body>
</html>