Author: stack
Date: Thu Jul 24 15:07:03 2008
New Revision: 679561
URL: http://svn.apache.org/viewvc?rev=679561&view=rev
Log:
HBASE-746 Batching row mutations via thrift
Modified:
hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/thrift/Hbase.thrift
hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/thrift/ThriftServer.java
hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/thrift/generated/Hbase.java
Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/thrift/Hbase.thrift
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/thrift/Hbase.thrift?rev=679561&r1=679560&r2=679561&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/thrift/Hbase.thrift (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/thrift/Hbase.thrift Thu Jul 24 15:07:03 2008
@@ -86,6 +86,16 @@
3:Text value
}
+
+/**
+ * A BatchMutation object is used to apply a number of Mutations to a single row.
+ */
+struct BatchMutation {
+ 1:Text row,
+ 2:list<Mutation> mutations
+}
+
+
/**
* A ScanEntry contains the row, column, and value information for a scanner's
* current location.
@@ -287,6 +297,31 @@
throws (1:IOError io, 2:IllegalArgument ia)
/**
+ * Apply a series of batches (each a series of mutations on a single row)
+ * in a single transaction. If an exception is thrown, then the
+ * transaction is aborted. Default current timestamp is used, and
+ * all entries will have an identical timestamp.
+ *
+ * @param tableName name of table
+ * @param rowBatches list of row batches
+ */
+ void mutateRows(1:Text tableName, 2:list<BatchMutation> rowBatches)
+ throws (1:IOError io, 2:IllegalArgument ia)
+
+ /**
+ * Apply a series of batches (each a series of mutations on a single row)
+ * in a single transaction. If an exception is thrown, then the
+ * transaction is aborted. The specified timestamp is used, and
+ * all entries will have an identical timestamp.
+ *
+ * @param tableName name of table
+ * @param rowBatches list of row batches
+ * @param timestamp timestamp
+ */
+ void mutateRowsTs(1:Text tableName, 2:list<BatchMutation> rowBatches, 3:i64 timestamp)
+ throws (1:IOError io, 2:IllegalArgument ia)
+
+ /**
* Delete all cells that match the passed row and column.
*
* @param tableName name of table
Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/thrift/ThriftServer.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/thrift/ThriftServer.java?rev=679561&r1=679560&r2=679561&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/thrift/ThriftServer.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/thrift/ThriftServer.java Thu Jul 24 15:07:03 2008
@@ -41,6 +41,7 @@
import org.apache.hadoop.hbase.io.Cell;
import org.apache.hadoop.hbase.io.RowResult;
import org.apache.hadoop.hbase.thrift.generated.AlreadyExists;
+import org.apache.hadoop.hbase.thrift.generated.BatchMutation;
import org.apache.hadoop.hbase.thrift.generated.ColumnDescriptor;
import org.apache.hadoop.hbase.thrift.generated.Hbase;
import org.apache.hadoop.hbase.thrift.generated.IOError;
@@ -430,6 +431,54 @@
}
}
+
+ public void mutateRows(byte[] tableName, ArrayList<BatchMutation> rowBatches)
+ throws IOError, IllegalArgument, TException {
+ mutateRowsTs(tableName, rowBatches, HConstants.LATEST_TIMESTAMP);
+ }
+
+ public void mutateRowsTs(byte[] tableName, ArrayList<BatchMutation> rowBatches, long timestamp)
+ throws IOError, IllegalArgument, TException {
+ ArrayList<BatchUpdate> batchUpdates = new ArrayList<BatchUpdate>();
+
+ for (BatchMutation batch : rowBatches) {
+ byte[] row = batch.row;
+ ArrayList<Mutation> mutations = batch.mutations;
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("mutateRowTs: table=" + new String(tableName) + ", row="
+ + new String(row) + ", ts=" + timestamp + " mutations="
+ + mutations.size());
+ for (Mutation m : mutations) {
+ if (m.isDelete) {
+ LOG.debug("mutateRowTs: : delete - " + getText(m.column));
+ } else {
+ LOG.debug("mutateRowTs: : put - " + getText(m.column) + " => "
+ + m.value);
+ }
+ }
+ }
+ BatchUpdate batchUpdate = new BatchUpdate(getText(row), timestamp);
+ for (Mutation m : mutations) {
+ if (m.isDelete) {
+ batchUpdate.delete(getText(m.column));
+ } else {
+ batchUpdate.put(getText(m.column), m.value);
+ }
+ }
+ batchUpdates.add(batchUpdate);
+ }
+
+ HTable table = null;
+ try {
+ table = getTable(tableName);
+ table.commit(batchUpdates);
+ } catch (IOException e) {
+ throw new IOError(e.getMessage());
+ } catch (IllegalArgumentException e) {
+ throw new IllegalArgument(e.getMessage());
+ }
+ }
+
public void scannerClose(int id) throws IOError, IllegalArgument {
LOG.debug("scannerClose: id=" + id);
Scanner scanner = getScanner(id);
Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/thrift/generated/Hbase.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/thrift/generated/Hbase.java?rev=679561&r1=679560&r2=679561&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/thrift/generated/Hbase.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/thrift/generated/Hbase.java Thu Jul 24 15:07:03 2008
@@ -173,6 +173,29 @@
public void mutateRowTs(byte[] tableName, byte[] row, ArrayList<Mutation> mutations, long timestamp) throws IOError, IllegalArgument, TException;
/**
+ * Apply a series of batches (each a series of mutations on a single row)
+ * in a single transaction. If an exception is thrown, then the
+ * transaction is aborted. Default current timestamp is used, and
+ * all entries will have an identical timestamp.
+ *
+ * @param tableName name of table
+ * @param rowBatches list of row batches
+ */
+ public void mutateRows(byte[] tableName, ArrayList<BatchMutation> rowBatches) throws IOError, IllegalArgument, TException;
+
+ /**
+ * Apply a series of batches (each a series of mutations on a single row)
+ * in a single transaction. If an exception is thrown, then the
+ * transaction is aborted. The specified timestamp is used, and
+ * all entries will have an identical timestamp.
+ *
+ * @param tableName name of table
+ * @param rowBatches list of row batches
+ * @param timestamp timestamp
+ */
+ public void mutateRowsTs(byte[] tableName, ArrayList<BatchMutation> rowBatches, long timestamp) throws IOError, IllegalArgument, TException;
+
+ /**
* Delete all cells that match the passed row and column.
*
* @param tableName name of table
@@ -820,6 +843,81 @@
return;
}
+ public void mutateRows(byte[] tableName, ArrayList<BatchMutation> rowBatches) throws IOError, IllegalArgument, TException
+ {
+ send_mutateRows(tableName, rowBatches);
+ recv_mutateRows();
+ }
+
+ public void send_mutateRows(byte[] tableName, ArrayList<BatchMutation> rowBatches) throws TException
+ {
+ oprot_.writeMessageBegin(new TMessage("mutateRows", TMessageType.CALL, seqid_));
+ mutateRows_args args = new mutateRows_args();
+ args.tableName = tableName;
+ args.rowBatches = rowBatches;
+ args.write(oprot_);
+ oprot_.writeMessageEnd();
+ oprot_.getTransport().flush();
+ }
+
+ public void recv_mutateRows() throws IOError, IllegalArgument, TException
+ {
+ TMessage msg = iprot_.readMessageBegin();
+ if (msg.type == TMessageType.EXCEPTION) {
+ TApplicationException x = TApplicationException.read(iprot_);
+ iprot_.readMessageEnd();
+ throw x;
+ }
+ mutateRows_result result = new mutateRows_result();
+ result.read(iprot_);
+ iprot_.readMessageEnd();
+ if (result.__isset.io) {
+ throw result.io;
+ }
+ if (result.__isset.ia) {
+ throw result.ia;
+ }
+ return;
+ }
+
+ public void mutateRowsTs(byte[] tableName, ArrayList<BatchMutation> rowBatches, long timestamp) throws IOError, IllegalArgument, TException
+ {
+ send_mutateRowsTs(tableName, rowBatches, timestamp);
+ recv_mutateRowsTs();
+ }
+
+ public void send_mutateRowsTs(byte[] tableName, ArrayList<BatchMutation> rowBatches, long timestamp) throws TException
+ {
+ oprot_.writeMessageBegin(new TMessage("mutateRowsTs", TMessageType.CALL, seqid_));
+ mutateRowsTs_args args = new mutateRowsTs_args();
+ args.tableName = tableName;
+ args.rowBatches = rowBatches;
+ args.timestamp = timestamp;
+ args.write(oprot_);
+ oprot_.writeMessageEnd();
+ oprot_.getTransport().flush();
+ }
+
+ public void recv_mutateRowsTs() throws IOError, IllegalArgument, TException
+ {
+ TMessage msg = iprot_.readMessageBegin();
+ if (msg.type == TMessageType.EXCEPTION) {
+ TApplicationException x = TApplicationException.read(iprot_);
+ iprot_.readMessageEnd();
+ throw x;
+ }
+ mutateRowsTs_result result = new mutateRowsTs_result();
+ result.read(iprot_);
+ iprot_.readMessageEnd();
+ if (result.__isset.io) {
+ throw result.io;
+ }
+ if (result.__isset.ia) {
+ throw result.ia;
+ }
+ return;
+ }
+
public void deleteAll(byte[] tableName, byte[] row, byte[] column) throws IOError, TException
{
send_deleteAll(tableName, row, column);
@@ -1212,6 +1310,8 @@
processMap_.put("put", new put());
processMap_.put("mutateRow", new mutateRow());
processMap_.put("mutateRowTs", new mutateRowTs());
+ processMap_.put("mutateRows", new mutateRows());
+ processMap_.put("mutateRowsTs", new mutateRowsTs());
processMap_.put("deleteAll", new deleteAll());
processMap_.put("deleteAllTs", new deleteAllTs());
processMap_.put("deleteAllRow", new deleteAllRow());
@@ -1557,6 +1657,54 @@
}
+ private class mutateRows implements ProcessFunction {
+ public void process(int seqid, TProtocol iprot, TProtocol oprot) throws TException
+ {
+ mutateRows_args args = new mutateRows_args();
+ args.read(iprot);
+ iprot.readMessageEnd();
+ mutateRows_result result = new mutateRows_result();
+ try {
+ iface_.mutateRows(args.tableName, args.rowBatches);
+ } catch (IOError io) {
+ result.io = io;
+ result.__isset.io = true;
+ } catch (IllegalArgument ia) {
+ result.ia = ia;
+ result.__isset.ia = true;
+ }
+ oprot.writeMessageBegin(new TMessage("mutateRows", TMessageType.REPLY, seqid));
+ result.write(oprot);
+ oprot.writeMessageEnd();
+ oprot.getTransport().flush();
+ }
+
+ }
+
+ private class mutateRowsTs implements ProcessFunction {
+ public void process(int seqid, TProtocol iprot, TProtocol oprot) throws TException
+ {
+ mutateRowsTs_args args = new mutateRowsTs_args();
+ args.read(iprot);
+ iprot.readMessageEnd();
+ mutateRowsTs_result result = new mutateRowsTs_result();
+ try {
+ iface_.mutateRowsTs(args.tableName, args.rowBatches, args.timestamp);
+ } catch (IOError io) {
+ result.io = io;
+ result.__isset.io = true;
+ } catch (IllegalArgument ia) {
+ result.ia = ia;
+ result.__isset.ia = true;
+ }
+ oprot.writeMessageBegin(new TMessage("mutateRowsTs", TMessageType.REPLY, seqid));
+ result.write(oprot);
+ oprot.writeMessageEnd();
+ oprot.getTransport().flush();
+ }
+
+ }
+
private class deleteAll implements ProcessFunction {
public void process(int seqid, TProtocol iprot, TProtocol oprot) throws TException
{
@@ -1860,13 +2008,13 @@
case 0:
if (field.type == TType.LIST) {
{
- TList _list5 = iprot.readListBegin();
- this.success = new ArrayList<byte[]>(_list5.size);
- for (int _i6 = 0; _i6 < _list5.size; ++_i6)
+ TList _list9 = iprot.readListBegin();
+ this.success = new ArrayList<byte[]>(_list9.size);
+ for (int _i10 = 0; _i10 < _list9.size; ++_i10)
{
- byte[] _elem7 = null;
- _elem7 = iprot.readBinary();
- this.success.add(_elem7);
+ byte[] _elem11 = null;
+ _elem11 = iprot.readBinary();
+ this.success.add(_elem11);
}
iprot.readListEnd();
}
@@ -1906,8 +2054,8 @@
oprot.writeFieldBegin(field);
{
oprot.writeListBegin(new TList(TType.STRING, this.success.size()));
- for (byte[] _iter8 : this.success) {
- oprot.writeBinary(_iter8);
+ for (byte[] _iter12 : this.success) {
+ oprot.writeBinary(_iter12);
}
oprot.writeListEnd();
}
@@ -2050,16 +2198,16 @@
case 0:
if (field.type == TType.MAP) {
{
- TMap _map9 = iprot.readMapBegin();
- this.success = new HashMap<byte[],ColumnDescriptor>(2*_map9.size);
- for (int _i10 = 0; _i10 < _map9.size; ++_i10)
+ TMap _map13 = iprot.readMapBegin();
+ this.success = new HashMap<byte[],ColumnDescriptor>(2*_map13.size);
+ for (int _i14 = 0; _i14 < _map13.size; ++_i14)
{
- byte[] _key11;
- ColumnDescriptor _val12;
- _key11 = iprot.readBinary();
- _val12 = new ColumnDescriptor();
- _val12.read(iprot);
- this.success.put(_key11, _val12);
+ byte[] _key15;
+ ColumnDescriptor _val16;
+ _key15 = iprot.readBinary();
+ _val16 = new ColumnDescriptor();
+ _val16.read(iprot);
+ this.success.put(_key15, _val16);
}
iprot.readMapEnd();
}
@@ -2099,9 +2247,9 @@
oprot.writeFieldBegin(field);
{
oprot.writeMapBegin(new TMap(TType.STRING, TType.STRUCT, this.success.size()));
- for (byte[] _iter13 : this.success.keySet()) {
- oprot.writeBinary(_iter13);
- this.success.get(_iter13).write(oprot);
+ for (byte[] _iter17 : this.success.keySet()) {
+ oprot.writeBinary(_iter17);
+ this.success.get(_iter17).write(oprot);
}
oprot.writeMapEnd();
}
@@ -2244,14 +2392,14 @@
case 0:
if (field.type == TType.LIST) {
{
- TList _list14 = iprot.readListBegin();
- this.success = new ArrayList<RegionDescriptor>(_list14.size);
- for (int _i15 = 0; _i15 < _list14.size; ++_i15)
+ TList _list18 = iprot.readListBegin();
+ this.success = new ArrayList<RegionDescriptor>(_list18.size);
+ for (int _i19 = 0; _i19 < _list18.size; ++_i19)
{
- RegionDescriptor _elem16 = new RegionDescriptor();
- _elem16 = new RegionDescriptor();
- _elem16.read(iprot);
- this.success.add(_elem16);
+ RegionDescriptor _elem20 = new RegionDescriptor();
+ _elem20 = new RegionDescriptor();
+ _elem20.read(iprot);
+ this.success.add(_elem20);
}
iprot.readListEnd();
}
@@ -2291,8 +2439,8 @@
oprot.writeFieldBegin(field);
{
oprot.writeListBegin(new TList(TType.STRUCT, this.success.size()));
- for (RegionDescriptor _iter17 : this.success) {
- _iter17.write(oprot);
+ for (RegionDescriptor _iter21 : this.success) {
+ _iter21.write(oprot);
}
oprot.writeListEnd();
}
@@ -2370,14 +2518,14 @@
case 2:
if (field.type == TType.LIST) {
{
- TList _list18 = iprot.readListBegin();
- this.columnFamilies = new ArrayList<ColumnDescriptor>(_list18.size);
- for (int _i19 = 0; _i19 < _list18.size; ++_i19)
+ TList _list22 = iprot.readListBegin();
+ this.columnFamilies = new ArrayList<ColumnDescriptor>(_list22.size);
+ for (int _i23 = 0; _i23 < _list22.size; ++_i23)
{
- ColumnDescriptor _elem20 = new ColumnDescriptor();
- _elem20 = new ColumnDescriptor();
- _elem20.read(iprot);
- this.columnFamilies.add(_elem20);
+ ColumnDescriptor _elem24 = new ColumnDescriptor();
+ _elem24 = new ColumnDescriptor();
+ _elem24.read(iprot);
+ this.columnFamilies.add(_elem24);
}
iprot.readListEnd();
}
@@ -2414,8 +2562,8 @@
oprot.writeFieldBegin(field);
{
oprot.writeListBegin(new TList(TType.STRUCT, this.columnFamilies.size()));
- for (ColumnDescriptor _iter21 : this.columnFamilies) {
- _iter21.write(oprot);
+ for (ColumnDescriptor _iter25 : this.columnFamilies) {
+ _iter25.write(oprot);
}
oprot.writeListEnd();
}
@@ -3167,13 +3315,13 @@
case 0:
if (field.type == TType.LIST) {
{
- TList _list22 = iprot.readListBegin();
- this.success = new ArrayList<byte[]>(_list22.size);
- for (int _i23 = 0; _i23 < _list22.size; ++_i23)
+ TList _list26 = iprot.readListBegin();
+ this.success = new ArrayList<byte[]>(_list26.size);
+ for (int _i27 = 0; _i27 < _list26.size; ++_i27)
{
- byte[] _elem24 = null;
- _elem24 = iprot.readBinary();
- this.success.add(_elem24);
+ byte[] _elem28 = null;
+ _elem28 = iprot.readBinary();
+ this.success.add(_elem28);
}
iprot.readListEnd();
}
@@ -3222,8 +3370,8 @@
oprot.writeFieldBegin(field);
{
oprot.writeListBegin(new TList(TType.STRING, this.success.size()));
- for (byte[] _iter25 : this.success) {
- oprot.writeBinary(_iter25);
+ for (byte[] _iter29 : this.success) {
+ oprot.writeBinary(_iter29);
}
oprot.writeListEnd();
}
@@ -3470,13 +3618,13 @@
case 0:
if (field.type == TType.LIST) {
{
- TList _list26 = iprot.readListBegin();
- this.success = new ArrayList<byte[]>(_list26.size);
- for (int _i27 = 0; _i27 < _list26.size; ++_i27)
+ TList _list30 = iprot.readListBegin();
+ this.success = new ArrayList<byte[]>(_list30.size);
+ for (int _i31 = 0; _i31 < _list30.size; ++_i31)
{
- byte[] _elem28 = null;
- _elem28 = iprot.readBinary();
- this.success.add(_elem28);
+ byte[] _elem32 = null;
+ _elem32 = iprot.readBinary();
+ this.success.add(_elem32);
}
iprot.readListEnd();
}
@@ -3525,8 +3673,8 @@
oprot.writeFieldBegin(field);
{
oprot.writeListBegin(new TList(TType.STRING, this.success.size()));
- for (byte[] _iter29 : this.success) {
- oprot.writeBinary(_iter29);
+ for (byte[] _iter33 : this.success) {
+ oprot.writeBinary(_iter33);
}
oprot.writeListEnd();
}
@@ -3703,15 +3851,15 @@
case 0:
if (field.type == TType.MAP) {
{
- TMap _map30 = iprot.readMapBegin();
- this.success = new HashMap<byte[],byte[]>(2*_map30.size);
- for (int _i31 = 0; _i31 < _map30.size; ++_i31)
+ TMap _map34 = iprot.readMapBegin();
+ this.success = new HashMap<byte[],byte[]>(2*_map34.size);
+ for (int _i35 = 0; _i35 < _map34.size; ++_i35)
{
- byte[] _key32;
- byte[] _val33;
- _key32 = iprot.readBinary();
- _val33 = iprot.readBinary();
- this.success.put(_key32, _val33);
+ byte[] _key36;
+ byte[] _val37;
+ _key36 = iprot.readBinary();
+ _val37 = iprot.readBinary();
+ this.success.put(_key36, _val37);
}
iprot.readMapEnd();
}
@@ -3751,9 +3899,9 @@
oprot.writeFieldBegin(field);
{
oprot.writeMapBegin(new TMap(TType.STRING, TType.STRING, this.success.size()));
- for (byte[] _iter34 : this.success.keySet()) {
- oprot.writeBinary(_iter34);
- oprot.writeBinary(this.success.get(_iter34));
+ for (byte[] _iter38 : this.success.keySet()) {
+ oprot.writeBinary(_iter38);
+ oprot.writeBinary(this.success.get(_iter38));
}
oprot.writeMapEnd();
}
@@ -3940,15 +4088,15 @@
case 0:
if (field.type == TType.MAP) {
{
- TMap _map35 = iprot.readMapBegin();
- this.success = new HashMap<byte[],byte[]>(2*_map35.size);
- for (int _i36 = 0; _i36 < _map35.size; ++_i36)
+ TMap _map39 = iprot.readMapBegin();
+ this.success = new HashMap<byte[],byte[]>(2*_map39.size);
+ for (int _i40 = 0; _i40 < _map39.size; ++_i40)
{
- byte[] _key37;
- byte[] _val38;
- _key37 = iprot.readBinary();
- _val38 = iprot.readBinary();
- this.success.put(_key37, _val38);
+ byte[] _key41;
+ byte[] _val42;
+ _key41 = iprot.readBinary();
+ _val42 = iprot.readBinary();
+ this.success.put(_key41, _val42);
}
iprot.readMapEnd();
}
@@ -3988,9 +4136,9 @@
oprot.writeFieldBegin(field);
{
oprot.writeMapBegin(new TMap(TType.STRING, TType.STRING, this.success.size()));
- for (byte[] _iter39 : this.success.keySet()) {
- oprot.writeBinary(_iter39);
- oprot.writeBinary(this.success.get(_iter39));
+ for (byte[] _iter43 : this.success.keySet()) {
+ oprot.writeBinary(_iter43);
+ oprot.writeBinary(this.success.get(_iter43));
}
oprot.writeMapEnd();
}
@@ -4325,14 +4473,14 @@
case 3:
if (field.type == TType.LIST) {
{
- TList _list40 = iprot.readListBegin();
- this.mutations = new ArrayList<Mutation>(_list40.size);
- for (int _i41 = 0; _i41 < _list40.size; ++_i41)
+ TList _list44 = iprot.readListBegin();
+ this.mutations = new ArrayList<Mutation>(_list44.size);
+ for (int _i45 = 0; _i45 < _list44.size; ++_i45)
{
- Mutation _elem42 = new Mutation();
- _elem42 = new Mutation();
- _elem42.read(iprot);
- this.mutations.add(_elem42);
+ Mutation _elem46 = new Mutation();
+ _elem46 = new Mutation();
+ _elem46.read(iprot);
+ this.mutations.add(_elem46);
}
iprot.readListEnd();
}
@@ -4377,8 +4525,8 @@
oprot.writeFieldBegin(field);
{
oprot.writeListBegin(new TList(TType.STRUCT, this.mutations.size()));
- for (Mutation _iter43 : this.mutations) {
- _iter43.write(oprot);
+ for (Mutation _iter47 : this.mutations) {
+ _iter47.write(oprot);
}
oprot.writeListEnd();
}
@@ -4568,14 +4716,14 @@
case 3:
if (field.type == TType.LIST) {
{
- TList _list44 = iprot.readListBegin();
- this.mutations = new ArrayList<Mutation>(_list44.size);
- for (int _i45 = 0; _i45 < _list44.size; ++_i45)
+ TList _list48 = iprot.readListBegin();
+ this.mutations = new ArrayList<Mutation>(_list48.size);
+ for (int _i49 = 0; _i49 < _list48.size; ++_i49)
{
- Mutation _elem46 = new Mutation();
- _elem46 = new Mutation();
- _elem46.read(iprot);
- this.mutations.add(_elem46);
+ Mutation _elem50 = new Mutation();
+ _elem50 = new Mutation();
+ _elem50.read(iprot);
+ this.mutations.add(_elem50);
}
iprot.readListEnd();
}
@@ -4628,8 +4776,8 @@
oprot.writeFieldBegin(field);
{
oprot.writeListBegin(new TList(TType.STRUCT, this.mutations.size()));
- for (Mutation _iter47 : this.mutations) {
- _iter47.write(oprot);
+ for (Mutation _iter51 : this.mutations) {
+ _iter51.write(oprot);
}
oprot.writeListEnd();
}
@@ -4763,33 +4911,28 @@
}
- public static class deleteAll_args implements TBase, java.io.Serializable {
+ public static class mutateRows_args implements TBase, java.io.Serializable {
public byte[] tableName;
- public byte[] row;
- public byte[] column;
+ public ArrayList<BatchMutation> rowBatches;
public final Isset __isset = new Isset();
public static final class Isset {
public boolean tableName = false;
- public boolean row = false;
- public boolean column = false;
+ public boolean rowBatches = false;
}
- public deleteAll_args() {
+ public mutateRows_args() {
}
- public deleteAll_args(
+ public mutateRows_args(
byte[] tableName,
- byte[] row,
- byte[] column)
+ ArrayList<BatchMutation> rowBatches)
{
this();
this.tableName = tableName;
this.__isset.tableName = true;
- this.row = row;
- this.__isset.row = true;
- this.column = column;
- this.__isset.column = true;
+ this.rowBatches = rowBatches;
+ this.__isset.rowBatches = true;
}
public void read(TProtocol iprot) throws TException {
@@ -4812,17 +4955,20 @@
}
break;
case 2:
- if (field.type == TType.STRING) {
- this.row = iprot.readBinary();
- this.__isset.row = true;
- } else {
- TProtocolUtil.skip(iprot, field.type);
- }
- break;
- case 3:
- if (field.type == TType.STRING) {
- this.column = iprot.readBinary();
- this.__isset.column = true;
+ if (field.type == TType.LIST) {
+ {
+ TList _list52 = iprot.readListBegin();
+ this.rowBatches = new ArrayList<BatchMutation>(_list52.size);
+ for (int _i53 = 0; _i53 < _list52.size; ++_i53)
+ {
+ BatchMutation _elem54 = new BatchMutation();
+ _elem54 = new BatchMutation();
+ _elem54.read(iprot);
+ this.rowBatches.add(_elem54);
+ }
+ iprot.readListEnd();
+ }
+ this.__isset.rowBatches = true;
} else {
TProtocolUtil.skip(iprot, field.type);
}
@@ -4837,7 +4983,7 @@
}
public void write(TProtocol oprot) throws TException {
- TStruct struct = new TStruct("deleteAll_args");
+ TStruct struct = new TStruct("mutateRows_args");
oprot.writeStructBegin(struct);
TField field = new TField();
if (this.tableName != null) {
@@ -4848,20 +4994,18 @@
oprot.writeBinary(this.tableName);
oprot.writeFieldEnd();
}
- if (this.row != null) {
- field.name = "row";
- field.type = TType.STRING;
+ if (this.rowBatches != null) {
+ field.name = "rowBatches";
+ field.type = TType.LIST;
field.id = 2;
oprot.writeFieldBegin(field);
- oprot.writeBinary(this.row);
- oprot.writeFieldEnd();
- }
- if (this.column != null) {
- field.name = "column";
- field.type = TType.STRING;
- field.id = 3;
- oprot.writeFieldBegin(field);
- oprot.writeBinary(this.column);
+ {
+ oprot.writeListBegin(new TList(TType.STRUCT, this.rowBatches.size()));
+ for (BatchMutation _iter55 : this.rowBatches) {
+ _iter55.write(oprot);
+ }
+ oprot.writeListEnd();
+ }
oprot.writeFieldEnd();
}
oprot.writeFieldStop();
@@ -4869,36 +5013,39 @@
}
public String toString() {
- StringBuilder sb = new StringBuilder("deleteAll_args(");
+ StringBuilder sb = new StringBuilder("mutateRows_args(");
sb.append("tableName:");
sb.append(this.tableName);
- sb.append(",row:");
- sb.append(this.row);
- sb.append(",column:");
- sb.append(this.column);
+ sb.append(",rowBatches:");
+ sb.append(this.rowBatches);
sb.append(")");
return sb.toString();
}
}
- public static class deleteAll_result implements TBase, java.io.Serializable {
+ public static class mutateRows_result implements TBase, java.io.Serializable {
public IOError io;
+ public IllegalArgument ia;
public final Isset __isset = new Isset();
public static final class Isset {
public boolean io = false;
+ public boolean ia = false;
}
- public deleteAll_result() {
+ public mutateRows_result() {
}
- public deleteAll_result(
- IOError io)
+ public mutateRows_result(
+ IOError io,
+ IllegalArgument ia)
{
this();
this.io = io;
this.__isset.io = true;
+ this.ia = ia;
+ this.__isset.ia = true;
}
public void read(TProtocol iprot) throws TException {
@@ -4921,6 +5068,15 @@
TProtocolUtil.skip(iprot, field.type);
}
break;
+ case 2:
+ if (field.type == TType.STRUCT) {
+ this.ia = new IllegalArgument();
+ this.ia.read(iprot);
+ this.__isset.ia = true;
+ } else {
+ TProtocolUtil.skip(iprot, field.type);
+ }
+ break;
default:
TProtocolUtil.skip(iprot, field.type);
break;
@@ -4931,7 +5087,7 @@
}
public void write(TProtocol oprot) throws TException {
- TStruct struct = new TStruct("deleteAll_result");
+ TStruct struct = new TStruct("mutateRows_result");
oprot.writeStructBegin(struct);
TField field = new TField();
@@ -4944,51 +5100,57 @@
this.io.write(oprot);
oprot.writeFieldEnd();
}
+ } else if (this.__isset.ia) {
+ if (this.ia != null) {
+ field.name = "ia";
+ field.type = TType.STRUCT;
+ field.id = 2;
+ oprot.writeFieldBegin(field);
+ this.ia.write(oprot);
+ oprot.writeFieldEnd();
+ }
}
oprot.writeFieldStop();
oprot.writeStructEnd();
}
public String toString() {
- StringBuilder sb = new StringBuilder("deleteAll_result(");
+ StringBuilder sb = new StringBuilder("mutateRows_result(");
sb.append("io:");
sb.append(this.io.toString());
+ sb.append(",ia:");
+ sb.append(this.ia.toString());
sb.append(")");
return sb.toString();
}
}
- public static class deleteAllTs_args implements TBase, java.io.Serializable {
+ public static class mutateRowsTs_args implements TBase, java.io.Serializable {
public byte[] tableName;
- public byte[] row;
- public byte[] column;
+ public ArrayList<BatchMutation> rowBatches;
public long timestamp;
public final Isset __isset = new Isset();
public static final class Isset {
public boolean tableName = false;
- public boolean row = false;
- public boolean column = false;
+ public boolean rowBatches = false;
public boolean timestamp = false;
}
- public deleteAllTs_args() {
+ public mutateRowsTs_args() {
}
- public deleteAllTs_args(
+ public mutateRowsTs_args(
byte[] tableName,
- byte[] row,
- byte[] column,
+ ArrayList<BatchMutation> rowBatches,
long timestamp)
{
this();
this.tableName = tableName;
this.__isset.tableName = true;
- this.row = row;
- this.__isset.row = true;
- this.column = column;
- this.__isset.column = true;
+ this.rowBatches = rowBatches;
+ this.__isset.rowBatches = true;
this.timestamp = timestamp;
this.__isset.timestamp = true;
}
@@ -5013,10 +5175,447 @@
}
break;
case 2:
- if (field.type == TType.STRING) {
- this.row = iprot.readBinary();
- this.__isset.row = true;
- } else {
+ if (field.type == TType.LIST) {
+ {
+ TList _list56 = iprot.readListBegin();
+ this.rowBatches = new ArrayList<BatchMutation>(_list56.size);
+ for (int _i57 = 0; _i57 < _list56.size; ++_i57)
+ {
+ BatchMutation _elem58 = new BatchMutation();
+ _elem58 = new BatchMutation();
+ _elem58.read(iprot);
+ this.rowBatches.add(_elem58);
+ }
+ iprot.readListEnd();
+ }
+ this.__isset.rowBatches = true;
+ } else {
+ TProtocolUtil.skip(iprot, field.type);
+ }
+ break;
+ case 3:
+ if (field.type == TType.I64) {
+ this.timestamp = iprot.readI64();
+ this.__isset.timestamp = true;
+ } else {
+ TProtocolUtil.skip(iprot, field.type);
+ }
+ break;
+ default:
+ TProtocolUtil.skip(iprot, field.type);
+ break;
+ }
+ iprot.readFieldEnd();
+ }
+ iprot.readStructEnd();
+ }
+
+ public void write(TProtocol oprot) throws TException {
+ TStruct struct = new TStruct("mutateRowsTs_args");
+ oprot.writeStructBegin(struct);
+ TField field = new TField();
+ if (this.tableName != null) {
+ field.name = "tableName";
+ field.type = TType.STRING;
+ field.id = 1;
+ oprot.writeFieldBegin(field);
+ oprot.writeBinary(this.tableName);
+ oprot.writeFieldEnd();
+ }
+ if (this.rowBatches != null) {
+ field.name = "rowBatches";
+ field.type = TType.LIST;
+ field.id = 2;
+ oprot.writeFieldBegin(field);
+ {
+ oprot.writeListBegin(new TList(TType.STRUCT, this.rowBatches.size()));
+ for (BatchMutation _iter59 : this.rowBatches) {
+ _iter59.write(oprot);
+ }
+ oprot.writeListEnd();
+ }
+ oprot.writeFieldEnd();
+ }
+ field.name = "timestamp";
+ field.type = TType.I64;
+ field.id = 3;
+ oprot.writeFieldBegin(field);
+ oprot.writeI64(this.timestamp);
+ oprot.writeFieldEnd();
+ oprot.writeFieldStop();
+ oprot.writeStructEnd();
+ }
+
+ public String toString() {
+ StringBuilder sb = new StringBuilder("mutateRowsTs_args(");
+ sb.append("tableName:");
+ sb.append(this.tableName);
+ sb.append(",rowBatches:");
+ sb.append(this.rowBatches);
+ sb.append(",timestamp:");
+ sb.append(this.timestamp);
+ sb.append(")");
+ return sb.toString();
+ }
+
+ }
+
+ public static class mutateRowsTs_result implements TBase, java.io.Serializable {
+ public IOError io;
+ public IllegalArgument ia;
+
+ public final Isset __isset = new Isset();
+ public static final class Isset {
+ public boolean io = false;
+ public boolean ia = false;
+ }
+
+ public mutateRowsTs_result() {
+ }
+
+ public mutateRowsTs_result(
+ IOError io,
+ IllegalArgument ia)
+ {
+ this();
+ this.io = io;
+ this.__isset.io = true;
+ this.ia = ia;
+ this.__isset.ia = true;
+ }
+
+ public void read(TProtocol iprot) throws TException {
+ TField field;
+ iprot.readStructBegin();
+ while (true)
+ {
+ field = iprot.readFieldBegin();
+ if (field.type == TType.STOP) {
+ break;
+ }
+ switch (field.id)
+ {
+ case 1:
+ if (field.type == TType.STRUCT) {
+ this.io = new IOError();
+ this.io.read(iprot);
+ this.__isset.io = true;
+ } else {
+ TProtocolUtil.skip(iprot, field.type);
+ }
+ break;
+ case 2:
+ if (field.type == TType.STRUCT) {
+ this.ia = new IllegalArgument();
+ this.ia.read(iprot);
+ this.__isset.ia = true;
+ } else {
+ TProtocolUtil.skip(iprot, field.type);
+ }
+ break;
+ default:
+ TProtocolUtil.skip(iprot, field.type);
+ break;
+ }
+ iprot.readFieldEnd();
+ }
+ iprot.readStructEnd();
+ }
+
+ public void write(TProtocol oprot) throws TException {
+ TStruct struct = new TStruct("mutateRowsTs_result");
+ oprot.writeStructBegin(struct);
+ TField field = new TField();
+
+ if (this.__isset.io) {
+ if (this.io != null) {
+ field.name = "io";
+ field.type = TType.STRUCT;
+ field.id = 1;
+ oprot.writeFieldBegin(field);
+ this.io.write(oprot);
+ oprot.writeFieldEnd();
+ }
+ } else if (this.__isset.ia) {
+ if (this.ia != null) {
+ field.name = "ia";
+ field.type = TType.STRUCT;
+ field.id = 2;
+ oprot.writeFieldBegin(field);
+ this.ia.write(oprot);
+ oprot.writeFieldEnd();
+ }
+ }
+ oprot.writeFieldStop();
+ oprot.writeStructEnd();
+ }
+
+ public String toString() {
+ StringBuilder sb = new StringBuilder("mutateRowsTs_result(");
+ sb.append("io:");
+ sb.append(this.io.toString());
+ sb.append(",ia:");
+ sb.append(this.ia.toString());
+ sb.append(")");
+ return sb.toString();
+ }
+
+ }
+
+ public static class deleteAll_args implements TBase, java.io.Serializable {
+ public byte[] tableName;
+ public byte[] row;
+ public byte[] column;
+
+ public final Isset __isset = new Isset();
+ public static final class Isset {
+ public boolean tableName = false;
+ public boolean row = false;
+ public boolean column = false;
+ }
+
+ public deleteAll_args() {
+ }
+
+ public deleteAll_args(
+ byte[] tableName,
+ byte[] row,
+ byte[] column)
+ {
+ this();
+ this.tableName = tableName;
+ this.__isset.tableName = true;
+ this.row = row;
+ this.__isset.row = true;
+ this.column = column;
+ this.__isset.column = true;
+ }
+
+ public void read(TProtocol iprot) throws TException {
+ TField field;
+ iprot.readStructBegin();
+ while (true)
+ {
+ field = iprot.readFieldBegin();
+ if (field.type == TType.STOP) {
+ break;
+ }
+ switch (field.id)
+ {
+ case 1:
+ if (field.type == TType.STRING) {
+ this.tableName = iprot.readBinary();
+ this.__isset.tableName = true;
+ } else {
+ TProtocolUtil.skip(iprot, field.type);
+ }
+ break;
+ case 2:
+ if (field.type == TType.STRING) {
+ this.row = iprot.readBinary();
+ this.__isset.row = true;
+ } else {
+ TProtocolUtil.skip(iprot, field.type);
+ }
+ break;
+ case 3:
+ if (field.type == TType.STRING) {
+ this.column = iprot.readBinary();
+ this.__isset.column = true;
+ } else {
+ TProtocolUtil.skip(iprot, field.type);
+ }
+ break;
+ default:
+ TProtocolUtil.skip(iprot, field.type);
+ break;
+ }
+ iprot.readFieldEnd();
+ }
+ iprot.readStructEnd();
+ }
+
+ public void write(TProtocol oprot) throws TException {
+ TStruct struct = new TStruct("deleteAll_args");
+ oprot.writeStructBegin(struct);
+ TField field = new TField();
+ if (this.tableName != null) {
+ field.name = "tableName";
+ field.type = TType.STRING;
+ field.id = 1;
+ oprot.writeFieldBegin(field);
+ oprot.writeBinary(this.tableName);
+ oprot.writeFieldEnd();
+ }
+ if (this.row != null) {
+ field.name = "row";
+ field.type = TType.STRING;
+ field.id = 2;
+ oprot.writeFieldBegin(field);
+ oprot.writeBinary(this.row);
+ oprot.writeFieldEnd();
+ }
+ if (this.column != null) {
+ field.name = "column";
+ field.type = TType.STRING;
+ field.id = 3;
+ oprot.writeFieldBegin(field);
+ oprot.writeBinary(this.column);
+ oprot.writeFieldEnd();
+ }
+ oprot.writeFieldStop();
+ oprot.writeStructEnd();
+ }
+
+ public String toString() {
+ StringBuilder sb = new StringBuilder("deleteAll_args(");
+ sb.append("tableName:");
+ sb.append(this.tableName);
+ sb.append(",row:");
+ sb.append(this.row);
+ sb.append(",column:");
+ sb.append(this.column);
+ sb.append(")");
+ return sb.toString();
+ }
+
+ }
+
+ public static class deleteAll_result implements TBase, java.io.Serializable {
+ public IOError io;
+
+ public final Isset __isset = new Isset();
+ public static final class Isset {
+ public boolean io = false;
+ }
+
+ public deleteAll_result() {
+ }
+
+ public deleteAll_result(
+ IOError io)
+ {
+ this();
+ this.io = io;
+ this.__isset.io = true;
+ }
+
+ public void read(TProtocol iprot) throws TException {
+ TField field;
+ iprot.readStructBegin();
+ while (true)
+ {
+ field = iprot.readFieldBegin();
+ if (field.type == TType.STOP) {
+ break;
+ }
+ switch (field.id)
+ {
+ case 1:
+ if (field.type == TType.STRUCT) {
+ this.io = new IOError();
+ this.io.read(iprot);
+ this.__isset.io = true;
+ } else {
+ TProtocolUtil.skip(iprot, field.type);
+ }
+ break;
+ default:
+ TProtocolUtil.skip(iprot, field.type);
+ break;
+ }
+ iprot.readFieldEnd();
+ }
+ iprot.readStructEnd();
+ }
+
+ public void write(TProtocol oprot) throws TException {
+ TStruct struct = new TStruct("deleteAll_result");
+ oprot.writeStructBegin(struct);
+ TField field = new TField();
+
+ if (this.__isset.io) {
+ if (this.io != null) {
+ field.name = "io";
+ field.type = TType.STRUCT;
+ field.id = 1;
+ oprot.writeFieldBegin(field);
+ this.io.write(oprot);
+ oprot.writeFieldEnd();
+ }
+ }
+ oprot.writeFieldStop();
+ oprot.writeStructEnd();
+ }
+
+ public String toString() {
+ StringBuilder sb = new StringBuilder("deleteAll_result(");
+ sb.append("io:");
+ sb.append(this.io.toString());
+ sb.append(")");
+ return sb.toString();
+ }
+
+ }
+
+ public static class deleteAllTs_args implements TBase, java.io.Serializable {
+ public byte[] tableName;
+ public byte[] row;
+ public byte[] column;
+ public long timestamp;
+
+ public final Isset __isset = new Isset();
+ public static final class Isset {
+ public boolean tableName = false;
+ public boolean row = false;
+ public boolean column = false;
+ public boolean timestamp = false;
+ }
+
+ public deleteAllTs_args() {
+ }
+
+ public deleteAllTs_args(
+ byte[] tableName,
+ byte[] row,
+ byte[] column,
+ long timestamp)
+ {
+ this();
+ this.tableName = tableName;
+ this.__isset.tableName = true;
+ this.row = row;
+ this.__isset.row = true;
+ this.column = column;
+ this.__isset.column = true;
+ this.timestamp = timestamp;
+ this.__isset.timestamp = true;
+ }
+
+ public void read(TProtocol iprot) throws TException {
+ TField field;
+ iprot.readStructBegin();
+ while (true)
+ {
+ field = iprot.readFieldBegin();
+ if (field.type == TType.STOP) {
+ break;
+ }
+ switch (field.id)
+ {
+ case 1:
+ if (field.type == TType.STRING) {
+ this.tableName = iprot.readBinary();
+ this.__isset.tableName = true;
+ } else {
+ TProtocolUtil.skip(iprot, field.type);
+ }
+ break;
+ case 2:
+ if (field.type == TType.STRING) {
+ this.row = iprot.readBinary();
+ this.__isset.row = true;
+ } else {
TProtocolUtil.skip(iprot, field.type);
}
break;
@@ -5602,13 +6201,13 @@
case 3:
if (field.type == TType.LIST) {
{
- TList _list48 = iprot.readListBegin();
- this.columns = new ArrayList<byte[]>(_list48.size);
- for (int _i49 = 0; _i49 < _list48.size; ++_i49)
+ TList _list60 = iprot.readListBegin();
+ this.columns = new ArrayList<byte[]>(_list60.size);
+ for (int _i61 = 0; _i61 < _list60.size; ++_i61)
{
- byte[] _elem50 = null;
- _elem50 = iprot.readBinary();
- this.columns.add(_elem50);
+ byte[] _elem62 = null;
+ _elem62 = iprot.readBinary();
+ this.columns.add(_elem62);
}
iprot.readListEnd();
}
@@ -5653,8 +6252,8 @@
oprot.writeFieldBegin(field);
{
oprot.writeListBegin(new TList(TType.STRING, this.columns.size()));
- for (byte[] _iter51 : this.columns) {
- oprot.writeBinary(_iter51);
+ for (byte[] _iter63 : this.columns) {
+ oprot.writeBinary(_iter63);
}
oprot.writeListEnd();
}
@@ -5849,13 +6448,13 @@
case 4:
if (field.type == TType.LIST) {
{
- TList _list52 = iprot.readListBegin();
- this.columns = new ArrayList<byte[]>(_list52.size);
- for (int _i53 = 0; _i53 < _list52.size; ++_i53)
+ TList _list64 = iprot.readListBegin();
+ this.columns = new ArrayList<byte[]>(_list64.size);
+ for (int _i65 = 0; _i65 < _list64.size; ++_i65)
{
- byte[] _elem54 = null;
- _elem54 = iprot.readBinary();
- this.columns.add(_elem54);
+ byte[] _elem66 = null;
+ _elem66 = iprot.readBinary();
+ this.columns.add(_elem66);
}
iprot.readListEnd();
}
@@ -5908,8 +6507,8 @@
oprot.writeFieldBegin(field);
{
oprot.writeListBegin(new TList(TType.STRING, this.columns.size()));
- for (byte[] _iter55 : this.columns) {
- oprot.writeBinary(_iter55);
+ for (byte[] _iter67 : this.columns) {
+ oprot.writeBinary(_iter67);
}
oprot.writeListEnd();
}
@@ -6098,13 +6697,13 @@
case 3:
if (field.type == TType.LIST) {
{
- TList _list56 = iprot.readListBegin();
- this.columns = new ArrayList<byte[]>(_list56.size);
- for (int _i57 = 0; _i57 < _list56.size; ++_i57)
+ TList _list68 = iprot.readListBegin();
+ this.columns = new ArrayList<byte[]>(_list68.size);
+ for (int _i69 = 0; _i69 < _list68.size; ++_i69)
{
- byte[] _elem58 = null;
- _elem58 = iprot.readBinary();
- this.columns.add(_elem58);
+ byte[] _elem70 = null;
+ _elem70 = iprot.readBinary();
+ this.columns.add(_elem70);
}
iprot.readListEnd();
}
@@ -6157,8 +6756,8 @@
oprot.writeFieldBegin(field);
{
oprot.writeListBegin(new TList(TType.STRING, this.columns.size()));
- for (byte[] _iter59 : this.columns) {
- oprot.writeBinary(_iter59);
+ for (byte[] _iter71 : this.columns) {
+ oprot.writeBinary(_iter71);
}
oprot.writeListEnd();
}
@@ -6366,13 +6965,13 @@
case 4:
if (field.type == TType.LIST) {
{
- TList _list60 = iprot.readListBegin();
- this.columns = new ArrayList<byte[]>(_list60.size);
- for (int _i61 = 0; _i61 < _list60.size; ++_i61)
+ TList _list72 = iprot.readListBegin();
+ this.columns = new ArrayList<byte[]>(_list72.size);
+ for (int _i73 = 0; _i73 < _list72.size; ++_i73)
{
- byte[] _elem62 = null;
- _elem62 = iprot.readBinary();
- this.columns.add(_elem62);
+ byte[] _elem74 = null;
+ _elem74 = iprot.readBinary();
+ this.columns.add(_elem74);
}
iprot.readListEnd();
}
@@ -6433,8 +7032,8 @@
oprot.writeFieldBegin(field);
{
oprot.writeListBegin(new TList(TType.STRING, this.columns.size()));
- for (byte[] _iter63 : this.columns) {
- oprot.writeBinary(_iter63);
+ for (byte[] _iter75 : this.columns) {
+ oprot.writeBinary(_iter75);
}
oprot.writeListEnd();
}
|