You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by sl...@apache.org on 2011/04/06 18:19:55 UTC

svn commit: r1089517 [1/2] - in /cassandra/trunk: interface/ interface/thrift/gen-java/org/apache/cassandra/thrift/ src/java/org/apache/cassandra/db/ src/java/org/apache/cassandra/hadoop/ src/java/org/apache/cassandra/thrift/ test/unit/org/apache/cassa...

Author: slebresne
Date: Wed Apr  6 16:19:55 2011
New Revision: 1089517

URL: http://svn.apache.org/viewvc?rev=1089517&view=rev
Log:
Merge Mutation and CounterMutation thrift structure
patch by slebresne; reviewed by jbellis for CASSANDRA-2384

Removed:
    cassandra/trunk/interface/thrift/gen-java/org/apache/cassandra/thrift/CounterDeletion.java
    cassandra/trunk/interface/thrift/gen-java/org/apache/cassandra/thrift/CounterMutation.java
Modified:
    cassandra/trunk/interface/cassandra.thrift
    cassandra/trunk/interface/thrift/gen-java/org/apache/cassandra/thrift/Cassandra.java
    cassandra/trunk/interface/thrift/gen-java/org/apache/cassandra/thrift/Deletion.java
    cassandra/trunk/interface/thrift/gen-java/org/apache/cassandra/thrift/Mutation.java
    cassandra/trunk/src/java/org/apache/cassandra/db/ColumnFamily.java
    cassandra/trunk/src/java/org/apache/cassandra/db/RowMutation.java
    cassandra/trunk/src/java/org/apache/cassandra/hadoop/ColumnFamilyRecordWriter.java
    cassandra/trunk/src/java/org/apache/cassandra/thrift/CassandraServer.java
    cassandra/trunk/src/java/org/apache/cassandra/thrift/ThriftValidation.java
    cassandra/trunk/test/unit/org/apache/cassandra/service/AntiEntropyServiceCounterTest.java
    cassandra/trunk/test/unit/org/apache/cassandra/service/AntiEntropyServiceStandardTest.java
    cassandra/trunk/test/unit/org/apache/cassandra/service/AntiEntropyServiceTest.java
    cassandra/trunk/test/unit/org/apache/cassandra/service/AntiEntropyServiceTestAbstract.java

Modified: cassandra/trunk/interface/cassandra.thrift
URL: http://svn.apache.org/viewvc/cassandra/trunk/interface/cassandra.thrift?rev=1089517&r1=1089516&r2=1089517&view=diff
==============================================================================
--- cassandra/trunk/interface/cassandra.thrift (original)
+++ cassandra/trunk/interface/cassandra.thrift Wed Apr  6 16:19:55 2011
@@ -321,35 +321,25 @@ struct KeyCount {
     2: required i32 count
 }
 
+/**
+ * Note that the timestamp is only optional in case of counter deletion.
+ */
 struct Deletion {
-    1: required i64 timestamp,
+    1: optional i64 timestamp,
     2: optional binary super_column,
     3: optional SlicePredicate predicate,
 }
 
 /**
-    A Mutation is either an insert, represented by filling column_or_supercolumn, or a deletion, represented by filling the deletion attribute.
+    A Mutation is either an insert (represented by filling column_or_supercolumn), a deletion (represented by filling the deletion attribute),
+    a counter addition (represented by filling counter), or a counter deletion (represented by filling counter_deletion).
     @param column_or_supercolumn. An insert to a column or supercolumn
     @param deletion. A deletion of a column or supercolumn
 */
 struct Mutation {
     1: optional ColumnOrSuperColumn column_or_supercolumn,
     2: optional Deletion deletion,
-}
-
-struct CounterDeletion {
-    1: optional binary super_column,
-    2: optional SlicePredicate predicate,
-}
-
-/**
-    A CounterMutation is either an insert, represented by filling counter, or a deletion, represented by filling the deletion attribute.
-    @param counter. An insert to a counter column or supercolumn
-    @param deletion. A deletion of a counter column or supercolumn
-*/
-struct CounterMutation {
-    1: optional Counter counter,
-    2: optional CounterDeletion deletion,
+    3: optional Counter counter,
 }
 
 struct TokenRange {
@@ -568,12 +558,6 @@ service Cassandra {
            3:required CounterColumn column,
            4:required ConsistencyLevel consistency_level=ConsistencyLevel.ONE)
        throws (1:InvalidRequestException ire, 2:UnavailableException ue, 3:TimedOutException te),
-  /**
-   * Batch increment or decrement a counter.
-   */
-  void batch_add(1:required map<binary, map<string, list<CounterMutation>>> update_map,
-                 2:required ConsistencyLevel consistency_level=ConsistencyLevel.ONE)
-       throws (1:InvalidRequestException ire, 2:UnavailableException ue, 3:TimedOutException te),
 
   /**
    * Return the counter at the specified column path.

Modified: cassandra/trunk/interface/thrift/gen-java/org/apache/cassandra/thrift/Cassandra.java
URL: http://svn.apache.org/viewvc/cassandra/trunk/interface/thrift/gen-java/org/apache/cassandra/thrift/Cassandra.java?rev=1089517&r1=1089516&r2=1089517&view=diff
==============================================================================
--- cassandra/trunk/interface/thrift/gen-java/org/apache/cassandra/thrift/Cassandra.java (original)
+++ cassandra/trunk/interface/thrift/gen-java/org/apache/cassandra/thrift/Cassandra.java Wed Apr  6 16:19:55 2011
@@ -189,14 +189,6 @@ public class Cassandra {
     public void add(ByteBuffer key, ColumnParent column_parent, CounterColumn column, ConsistencyLevel consistency_level) throws InvalidRequestException, UnavailableException, TimedOutException, TException;
 
     /**
-     * Batch increment or decrement a counter.
-     * 
-     * @param update_map
-     * @param consistency_level
-     */
-    public void batch_add(Map<ByteBuffer,Map<String,List<CounterMutation>>> update_map, ConsistencyLevel consistency_level) throws InvalidRequestException, UnavailableException, TimedOutException, TException;
-
-    /**
      * Return the counter at the specified column path.
      * 
      * @param key
@@ -384,8 +376,6 @@ public class Cassandra {
 
     public void add(ByteBuffer key, ColumnParent column_parent, CounterColumn column, ConsistencyLevel consistency_level, AsyncMethodCallback<AsyncClient.add_call> resultHandler) throws TException;
 
-    public void batch_add(Map<ByteBuffer,Map<String,List<CounterMutation>>> update_map, ConsistencyLevel consistency_level, AsyncMethodCallback<AsyncClient.batch_add_call> resultHandler) throws TException;
-
     public void get_counter(ByteBuffer key, ColumnPath path, ConsistencyLevel consistency_level, AsyncMethodCallback<AsyncClient.get_counter_call> resultHandler) throws TException;
 
     public void get_counter_slice(ByteBuffer key, ColumnParent column_parent, SlicePredicate predicate, ConsistencyLevel consistency_level, AsyncMethodCallback<AsyncClient.get_counter_slice_call> resultHandler) throws TException;
@@ -1095,49 +1085,6 @@ public class Cassandra {
       return;
     }
 
-    public void batch_add(Map<ByteBuffer,Map<String,List<CounterMutation>>> update_map, ConsistencyLevel consistency_level) throws InvalidRequestException, UnavailableException, TimedOutException, TException
-    {
-      send_batch_add(update_map, consistency_level);
-      recv_batch_add();
-    }
-
-    public void send_batch_add(Map<ByteBuffer,Map<String,List<CounterMutation>>> update_map, ConsistencyLevel consistency_level) throws TException
-    {
-      oprot_.writeMessageBegin(new TMessage("batch_add", TMessageType.CALL, ++seqid_));
-      batch_add_args args = new batch_add_args();
-      args.setUpdate_map(update_map);
-      args.setConsistency_level(consistency_level);
-      args.write(oprot_);
-      oprot_.writeMessageEnd();
-      oprot_.getTransport().flush();
-    }
-
-    public void recv_batch_add() throws InvalidRequestException, UnavailableException, TimedOutException, TException
-    {
-      TMessage msg = iprot_.readMessageBegin();
-      if (msg.type == TMessageType.EXCEPTION) {
-        TApplicationException x = TApplicationException.read(iprot_);
-        iprot_.readMessageEnd();
-        throw x;
-      }
-      if (msg.seqid != seqid_) {
-        throw new TApplicationException(TApplicationException.BAD_SEQUENCE_ID, "batch_add failed: out of sequence response");
-      }
-      batch_add_result result = new batch_add_result();
-      result.read(iprot_);
-      iprot_.readMessageEnd();
-      if (result.ire != null) {
-        throw result.ire;
-      }
-      if (result.ue != null) {
-        throw result.ue;
-      }
-      if (result.te != null) {
-        throw result.te;
-      }
-      return;
-    }
-
     public Counter get_counter(ByteBuffer key, ColumnPath path, ConsistencyLevel consistency_level) throws InvalidRequestException, NotFoundException, UnavailableException, TimedOutException, TException
     {
       send_get_counter(key, path, consistency_level);
@@ -2510,40 +2457,6 @@ public class Cassandra {
       }
     }
 
-    public void batch_add(Map<ByteBuffer,Map<String,List<CounterMutation>>> update_map, ConsistencyLevel consistency_level, AsyncMethodCallback<batch_add_call> resultHandler) throws TException {
-      checkReady();
-      batch_add_call method_call = new batch_add_call(update_map, consistency_level, resultHandler, this, protocolFactory, transport);
-      manager.call(method_call);
-    }
-
-    public static class batch_add_call extends TAsyncMethodCall {
-      private Map<ByteBuffer,Map<String,List<CounterMutation>>> update_map;
-      private ConsistencyLevel consistency_level;
-      public batch_add_call(Map<ByteBuffer,Map<String,List<CounterMutation>>> update_map, ConsistencyLevel consistency_level, AsyncMethodCallback<batch_add_call> resultHandler, TAsyncClient client, TProtocolFactory protocolFactory, TNonblockingTransport transport) throws TException {
-        super(client, protocolFactory, transport, resultHandler, false);
-        this.update_map = update_map;
-        this.consistency_level = consistency_level;
-      }
-
-      public void write_args(TProtocol prot) throws TException {
-        prot.writeMessageBegin(new TMessage("batch_add", TMessageType.CALL, 0));
-        batch_add_args args = new batch_add_args();
-        args.setUpdate_map(update_map);
-        args.setConsistency_level(consistency_level);
-        args.write(prot);
-        prot.writeMessageEnd();
-      }
-
-      public void getResult() throws InvalidRequestException, UnavailableException, TimedOutException, TException {
-        if (getState() != State.RESPONSE_READ) {
-          throw new IllegalStateException("Method call not finished!");
-        }
-        TMemoryInputTransport memoryTransport = new TMemoryInputTransport(getFrameBuffer().array());
-        TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
-        (new Client(prot)).recv_batch_add();
-      }
-    }
-
     public void get_counter(ByteBuffer key, ColumnPath path, ConsistencyLevel consistency_level, AsyncMethodCallback<get_counter_call> resultHandler) throws TException {
       checkReady();
       get_counter_call method_call = new get_counter_call(key, path, consistency_level, resultHandler, this, protocolFactory, transport);
@@ -3209,7 +3122,6 @@ public class Cassandra {
       processMap_.put("batch_mutate", new batch_mutate());
       processMap_.put("truncate", new truncate());
       processMap_.put("add", new add());
-      processMap_.put("batch_add", new batch_add());
       processMap_.put("get_counter", new get_counter());
       processMap_.put("get_counter_slice", new get_counter_slice());
       processMap_.put("multiget_counter_slice", new multiget_counter_slice());
@@ -3840,48 +3752,6 @@ public class Cassandra {
 
     }
 
-    private class batch_add implements ProcessFunction {
-      public void process(int seqid, TProtocol iprot, TProtocol oprot) throws TException
-      {
-        batch_add_args args = new batch_add_args();
-        try {
-          args.read(iprot);
-        } catch (TProtocolException e) {
-          iprot.readMessageEnd();
-          TApplicationException x = new TApplicationException(TApplicationException.PROTOCOL_ERROR, e.getMessage());
-          oprot.writeMessageBegin(new TMessage("batch_add", TMessageType.EXCEPTION, seqid));
-          x.write(oprot);
-          oprot.writeMessageEnd();
-          oprot.getTransport().flush();
-          return;
-        }
-        iprot.readMessageEnd();
-        batch_add_result result = new batch_add_result();
-        try {
-          iface_.batch_add(args.update_map, args.consistency_level);
-        } catch (InvalidRequestException ire) {
-          result.ire = ire;
-        } catch (UnavailableException ue) {
-          result.ue = ue;
-        } catch (TimedOutException te) {
-          result.te = te;
-        } catch (Throwable th) {
-          LOGGER.error("Internal error processing batch_add", th);
-          TApplicationException x = new TApplicationException(TApplicationException.INTERNAL_ERROR, "Internal error processing batch_add");
-          oprot.writeMessageBegin(new TMessage("batch_add", TMessageType.EXCEPTION, seqid));
-          x.write(oprot);
-          oprot.writeMessageEnd();
-          oprot.getTransport().flush();
-          return;
-        }
-        oprot.writeMessageBegin(new TMessage("batch_add", TMessageType.REPLY, seqid));
-        result.write(oprot);
-        oprot.writeMessageEnd();
-        oprot.getTransport().flush();
-      }
-
-    }
-
     private class get_counter implements ProcessFunction {
       public void process(int seqid, TProtocol iprot, TProtocol oprot) throws TException
       {
@@ -19351,990 +19221,6 @@ public class Cassandra {
 
   }
 
-  public static class batch_add_args implements TBase<batch_add_args, batch_add_args._Fields>, java.io.Serializable, Cloneable   {
-    private static final TStruct STRUCT_DESC = new TStruct("batch_add_args");
-
-    private static final TField UPDATE_MAP_FIELD_DESC = new TField("update_map", TType.MAP, (short)1);
-    private static final TField CONSISTENCY_LEVEL_FIELD_DESC = new TField("consistency_level", TType.I32, (short)2);
-
-    public Map<ByteBuffer,Map<String,List<CounterMutation>>> update_map;
-    /**
-     * 
-     * @see ConsistencyLevel
-     */
-    public ConsistencyLevel consistency_level;
-
-    /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
-    public enum _Fields implements TFieldIdEnum {
-      UPDATE_MAP((short)1, "update_map"),
-      /**
-       * 
-       * @see ConsistencyLevel
-       */
-      CONSISTENCY_LEVEL((short)2, "consistency_level");
-
-      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: // UPDATE_MAP
-            return UPDATE_MAP;
-          case 2: // CONSISTENCY_LEVEL
-            return CONSISTENCY_LEVEL;
-          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, FieldMetaData> metaDataMap;
-    static {
-      Map<_Fields, FieldMetaData> tmpMap = new EnumMap<_Fields, FieldMetaData>(_Fields.class);
-      tmpMap.put(_Fields.UPDATE_MAP, new FieldMetaData("update_map", TFieldRequirementType.REQUIRED, 
-          new MapMetaData(TType.MAP, 
-              new FieldValueMetaData(TType.STRING), 
-              new MapMetaData(TType.MAP, 
-                  new FieldValueMetaData(TType.STRING), 
-                  new ListMetaData(TType.LIST, 
-                      new StructMetaData(TType.STRUCT, CounterMutation.class))))));
-      tmpMap.put(_Fields.CONSISTENCY_LEVEL, new FieldMetaData("consistency_level", TFieldRequirementType.REQUIRED, 
-          new EnumMetaData(TType.ENUM, ConsistencyLevel.class)));
-      metaDataMap = Collections.unmodifiableMap(tmpMap);
-      FieldMetaData.addStructMetaDataMap(batch_add_args.class, metaDataMap);
-    }
-
-    public batch_add_args() {
-      this.consistency_level = ConsistencyLevel.ONE;
-
-    }
-
-    public batch_add_args(
-      Map<ByteBuffer,Map<String,List<CounterMutation>>> update_map,
-      ConsistencyLevel consistency_level)
-    {
-      this();
-      this.update_map = update_map;
-      this.consistency_level = consistency_level;
-    }
-
-    /**
-     * Performs a deep copy on <i>other</i>.
-     */
-    public batch_add_args(batch_add_args other) {
-      if (other.isSetUpdate_map()) {
-        Map<ByteBuffer,Map<String,List<CounterMutation>>> __this__update_map = new HashMap<ByteBuffer,Map<String,List<CounterMutation>>>();
-        for (Map.Entry<ByteBuffer, Map<String,List<CounterMutation>>> other_element : other.update_map.entrySet()) {
-
-          ByteBuffer other_element_key = other_element.getKey();
-          Map<String,List<CounterMutation>> other_element_value = other_element.getValue();
-
-          ByteBuffer __this__update_map_copy_key = TBaseHelper.copyBinary(other_element_key);
-;
-
-          Map<String,List<CounterMutation>> __this__update_map_copy_value = new HashMap<String,List<CounterMutation>>();
-          for (Map.Entry<String, List<CounterMutation>> other_element_value_element : other_element_value.entrySet()) {
-
-            String other_element_value_element_key = other_element_value_element.getKey();
-            List<CounterMutation> other_element_value_element_value = other_element_value_element.getValue();
-
-            String __this__update_map_copy_value_copy_key = other_element_value_element_key;
-
-            List<CounterMutation> __this__update_map_copy_value_copy_value = new ArrayList<CounterMutation>();
-            for (CounterMutation other_element_value_element_value_element : other_element_value_element_value) {
-              __this__update_map_copy_value_copy_value.add(new CounterMutation(other_element_value_element_value_element));
-            }
-
-            __this__update_map_copy_value.put(__this__update_map_copy_value_copy_key, __this__update_map_copy_value_copy_value);
-          }
-
-          __this__update_map.put(__this__update_map_copy_key, __this__update_map_copy_value);
-        }
-        this.update_map = __this__update_map;
-      }
-      if (other.isSetConsistency_level()) {
-        this.consistency_level = other.consistency_level;
-      }
-    }
-
-    public batch_add_args deepCopy() {
-      return new batch_add_args(this);
-    }
-
-    @Override
-    public void clear() {
-      this.update_map = null;
-      this.consistency_level = ConsistencyLevel.ONE;
-
-    }
-
-    public int getUpdate_mapSize() {
-      return (this.update_map == null) ? 0 : this.update_map.size();
-    }
-
-    public void putToUpdate_map(ByteBuffer key, Map<String,List<CounterMutation>> val) {
-      if (this.update_map == null) {
-        this.update_map = new HashMap<ByteBuffer,Map<String,List<CounterMutation>>>();
-      }
-      this.update_map.put(key, val);
-    }
-
-    public Map<ByteBuffer,Map<String,List<CounterMutation>>> getUpdate_map() {
-      return this.update_map;
-    }
-
-    public batch_add_args setUpdate_map(Map<ByteBuffer,Map<String,List<CounterMutation>>> update_map) {
-      this.update_map = update_map;
-      return this;
-    }
-
-    public void unsetUpdate_map() {
-      this.update_map = null;
-    }
-
-    /** Returns true if field update_map is set (has been asigned a value) and false otherwise */
-    public boolean isSetUpdate_map() {
-      return this.update_map != null;
-    }
-
-    public void setUpdate_mapIsSet(boolean value) {
-      if (!value) {
-        this.update_map = null;
-      }
-    }
-
-    /**
-     * 
-     * @see ConsistencyLevel
-     */
-    public ConsistencyLevel getConsistency_level() {
-      return this.consistency_level;
-    }
-
-    /**
-     * 
-     * @see ConsistencyLevel
-     */
-    public batch_add_args setConsistency_level(ConsistencyLevel consistency_level) {
-      this.consistency_level = consistency_level;
-      return this;
-    }
-
-    public void unsetConsistency_level() {
-      this.consistency_level = null;
-    }
-
-    /** Returns true if field consistency_level is set (has been asigned a value) and false otherwise */
-    public boolean isSetConsistency_level() {
-      return this.consistency_level != null;
-    }
-
-    public void setConsistency_levelIsSet(boolean value) {
-      if (!value) {
-        this.consistency_level = null;
-      }
-    }
-
-    public void setFieldValue(_Fields field, Object value) {
-      switch (field) {
-      case UPDATE_MAP:
-        if (value == null) {
-          unsetUpdate_map();
-        } else {
-          setUpdate_map((Map<ByteBuffer,Map<String,List<CounterMutation>>>)value);
-        }
-        break;
-
-      case CONSISTENCY_LEVEL:
-        if (value == null) {
-          unsetConsistency_level();
-        } else {
-          setConsistency_level((ConsistencyLevel)value);
-        }
-        break;
-
-      }
-    }
-
-    public Object getFieldValue(_Fields field) {
-      switch (field) {
-      case UPDATE_MAP:
-        return getUpdate_map();
-
-      case CONSISTENCY_LEVEL:
-        return getConsistency_level();
-
-      }
-      throw new IllegalStateException();
-    }
-
-    /** Returns true if field corresponding to fieldID is set (has been asigned a value) and false otherwise */
-    public boolean isSet(_Fields field) {
-      if (field == null) {
-        throw new IllegalArgumentException();
-      }
-
-      switch (field) {
-      case UPDATE_MAP:
-        return isSetUpdate_map();
-      case CONSISTENCY_LEVEL:
-        return isSetConsistency_level();
-      }
-      throw new IllegalStateException();
-    }
-
-    @Override
-    public boolean equals(Object that) {
-      if (that == null)
-        return false;
-      if (that instanceof batch_add_args)
-        return this.equals((batch_add_args)that);
-      return false;
-    }
-
-    public boolean equals(batch_add_args that) {
-      if (that == null)
-        return false;
-
-      boolean this_present_update_map = true && this.isSetUpdate_map();
-      boolean that_present_update_map = true && that.isSetUpdate_map();
-      if (this_present_update_map || that_present_update_map) {
-        if (!(this_present_update_map && that_present_update_map))
-          return false;
-        if (!this.update_map.equals(that.update_map))
-          return false;
-      }
-
-      boolean this_present_consistency_level = true && this.isSetConsistency_level();
-      boolean that_present_consistency_level = true && that.isSetConsistency_level();
-      if (this_present_consistency_level || that_present_consistency_level) {
-        if (!(this_present_consistency_level && that_present_consistency_level))
-          return false;
-        if (!this.consistency_level.equals(that.consistency_level))
-          return false;
-      }
-
-      return true;
-    }
-
-    @Override
-    public int hashCode() {
-      HashCodeBuilder builder = new HashCodeBuilder();
-
-      boolean present_update_map = true && (isSetUpdate_map());
-      builder.append(present_update_map);
-      if (present_update_map)
-        builder.append(update_map);
-
-      boolean present_consistency_level = true && (isSetConsistency_level());
-      builder.append(present_consistency_level);
-      if (present_consistency_level)
-        builder.append(consistency_level.getValue());
-
-      return builder.toHashCode();
-    }
-
-    public int compareTo(batch_add_args other) {
-      if (!getClass().equals(other.getClass())) {
-        return getClass().getName().compareTo(other.getClass().getName());
-      }
-
-      int lastComparison = 0;
-      batch_add_args typedOther = (batch_add_args)other;
-
-      lastComparison = Boolean.valueOf(isSetUpdate_map()).compareTo(typedOther.isSetUpdate_map());
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-      if (isSetUpdate_map()) {
-        lastComparison = TBaseHelper.compareTo(this.update_map, typedOther.update_map);
-        if (lastComparison != 0) {
-          return lastComparison;
-        }
-      }
-      lastComparison = Boolean.valueOf(isSetConsistency_level()).compareTo(typedOther.isSetConsistency_level());
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-      if (isSetConsistency_level()) {
-        lastComparison = TBaseHelper.compareTo(this.consistency_level, typedOther.consistency_level);
-        if (lastComparison != 0) {
-          return lastComparison;
-        }
-      }
-      return 0;
-    }
-
-    public _Fields fieldForId(int fieldId) {
-      return _Fields.findByThriftId(fieldId);
-    }
-
-    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: // UPDATE_MAP
-            if (field.type == TType.MAP) {
-              {
-                TMap _map98 = iprot.readMapBegin();
-                this.update_map = new HashMap<ByteBuffer,Map<String,List<CounterMutation>>>(2*_map98.size);
-                for (int _i99 = 0; _i99 < _map98.size; ++_i99)
-                {
-                  ByteBuffer _key100;
-                  Map<String,List<CounterMutation>> _val101;
-                  _key100 = iprot.readBinary();
-                  {
-                    TMap _map102 = iprot.readMapBegin();
-                    _val101 = new HashMap<String,List<CounterMutation>>(2*_map102.size);
-                    for (int _i103 = 0; _i103 < _map102.size; ++_i103)
-                    {
-                      String _key104;
-                      List<CounterMutation> _val105;
-                      _key104 = iprot.readString();
-                      {
-                        TList _list106 = iprot.readListBegin();
-                        _val105 = new ArrayList<CounterMutation>(_list106.size);
-                        for (int _i107 = 0; _i107 < _list106.size; ++_i107)
-                        {
-                          CounterMutation _elem108;
-                          _elem108 = new CounterMutation();
-                          _elem108.read(iprot);
-                          _val105.add(_elem108);
-                        }
-                        iprot.readListEnd();
-                      }
-                      _val101.put(_key104, _val105);
-                    }
-                    iprot.readMapEnd();
-                  }
-                  this.update_map.put(_key100, _val101);
-                }
-                iprot.readMapEnd();
-              }
-            } else { 
-              TProtocolUtil.skip(iprot, field.type);
-            }
-            break;
-          case 2: // CONSISTENCY_LEVEL
-            if (field.type == TType.I32) {
-              this.consistency_level = ConsistencyLevel.findByValue(iprot.readI32());
-            } else { 
-              TProtocolUtil.skip(iprot, field.type);
-            }
-            break;
-          default:
-            TProtocolUtil.skip(iprot, field.type);
-        }
-        iprot.readFieldEnd();
-      }
-      iprot.readStructEnd();
-
-      // check for required fields of primitive type, which can't be checked in the validate method
-      validate();
-    }
-
-    public void write(TProtocol oprot) throws TException {
-      validate();
-
-      oprot.writeStructBegin(STRUCT_DESC);
-      if (this.update_map != null) {
-        oprot.writeFieldBegin(UPDATE_MAP_FIELD_DESC);
-        {
-          oprot.writeMapBegin(new TMap(TType.STRING, TType.MAP, this.update_map.size()));
-          for (Map.Entry<ByteBuffer, Map<String,List<CounterMutation>>> _iter109 : this.update_map.entrySet())
-          {
-            oprot.writeBinary(_iter109.getKey());
-            {
-              oprot.writeMapBegin(new TMap(TType.STRING, TType.LIST, _iter109.getValue().size()));
-              for (Map.Entry<String, List<CounterMutation>> _iter110 : _iter109.getValue().entrySet())
-              {
-                oprot.writeString(_iter110.getKey());
-                {
-                  oprot.writeListBegin(new TList(TType.STRUCT, _iter110.getValue().size()));
-                  for (CounterMutation _iter111 : _iter110.getValue())
-                  {
-                    _iter111.write(oprot);
-                  }
-                  oprot.writeListEnd();
-                }
-              }
-              oprot.writeMapEnd();
-            }
-          }
-          oprot.writeMapEnd();
-        }
-        oprot.writeFieldEnd();
-      }
-      if (this.consistency_level != null) {
-        oprot.writeFieldBegin(CONSISTENCY_LEVEL_FIELD_DESC);
-        oprot.writeI32(this.consistency_level.getValue());
-        oprot.writeFieldEnd();
-      }
-      oprot.writeFieldStop();
-      oprot.writeStructEnd();
-    }
-
-    @Override
-    public String toString() {
-      StringBuilder sb = new StringBuilder("batch_add_args(");
-      boolean first = true;
-
-      sb.append("update_map:");
-      if (this.update_map == null) {
-        sb.append("null");
-      } else {
-        sb.append(this.update_map);
-      }
-      first = false;
-      if (!first) sb.append(", ");
-      sb.append("consistency_level:");
-      if (this.consistency_level == null) {
-        sb.append("null");
-      } else {
-        sb.append(this.consistency_level);
-      }
-      first = false;
-      sb.append(")");
-      return sb.toString();
-    }
-
-    public void validate() throws TException {
-      // check for required fields
-      if (update_map == null) {
-        throw new TProtocolException("Required field 'update_map' was not present! Struct: " + toString());
-      }
-      if (consistency_level == null) {
-        throw new TProtocolException("Required field 'consistency_level' was not present! Struct: " + toString());
-      }
-    }
-
-  }
-
-  public static class batch_add_result implements TBase<batch_add_result, batch_add_result._Fields>, java.io.Serializable, Cloneable   {
-    private static final TStruct STRUCT_DESC = new TStruct("batch_add_result");
-
-    private static final TField IRE_FIELD_DESC = new TField("ire", TType.STRUCT, (short)1);
-    private static final TField UE_FIELD_DESC = new TField("ue", TType.STRUCT, (short)2);
-    private static final TField TE_FIELD_DESC = new TField("te", TType.STRUCT, (short)3);
-
-    public InvalidRequestException ire;
-    public UnavailableException ue;
-    public TimedOutException te;
-
-    /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
-    public enum _Fields implements TFieldIdEnum {
-      IRE((short)1, "ire"),
-      UE((short)2, "ue"),
-      TE((short)3, "te");
-
-      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: // IRE
-            return IRE;
-          case 2: // UE
-            return UE;
-          case 3: // TE
-            return TE;
-          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, FieldMetaData> metaDataMap;
-    static {
-      Map<_Fields, FieldMetaData> tmpMap = new EnumMap<_Fields, FieldMetaData>(_Fields.class);
-      tmpMap.put(_Fields.IRE, new FieldMetaData("ire", TFieldRequirementType.DEFAULT, 
-          new FieldValueMetaData(TType.STRUCT)));
-      tmpMap.put(_Fields.UE, new FieldMetaData("ue", TFieldRequirementType.DEFAULT, 
-          new FieldValueMetaData(TType.STRUCT)));
-      tmpMap.put(_Fields.TE, new FieldMetaData("te", TFieldRequirementType.DEFAULT, 
-          new FieldValueMetaData(TType.STRUCT)));
-      metaDataMap = Collections.unmodifiableMap(tmpMap);
-      FieldMetaData.addStructMetaDataMap(batch_add_result.class, metaDataMap);
-    }
-
-    public batch_add_result() {
-    }
-
-    public batch_add_result(
-      InvalidRequestException ire,
-      UnavailableException ue,
-      TimedOutException te)
-    {
-      this();
-      this.ire = ire;
-      this.ue = ue;
-      this.te = te;
-    }
-
-    /**
-     * Performs a deep copy on <i>other</i>.
-     */
-    public batch_add_result(batch_add_result other) {
-      if (other.isSetIre()) {
-        this.ire = new InvalidRequestException(other.ire);
-      }
-      if (other.isSetUe()) {
-        this.ue = new UnavailableException(other.ue);
-      }
-      if (other.isSetTe()) {
-        this.te = new TimedOutException(other.te);
-      }
-    }
-
-    public batch_add_result deepCopy() {
-      return new batch_add_result(this);
-    }
-
-    @Override
-    public void clear() {
-      this.ire = null;
-      this.ue = null;
-      this.te = null;
-    }
-
-    public InvalidRequestException getIre() {
-      return this.ire;
-    }
-
-    public batch_add_result setIre(InvalidRequestException ire) {
-      this.ire = ire;
-      return this;
-    }
-
-    public void unsetIre() {
-      this.ire = null;
-    }
-
-    /** Returns true if field ire is set (has been asigned a value) and false otherwise */
-    public boolean isSetIre() {
-      return this.ire != null;
-    }
-
-    public void setIreIsSet(boolean value) {
-      if (!value) {
-        this.ire = null;
-      }
-    }
-
-    public UnavailableException getUe() {
-      return this.ue;
-    }
-
-    public batch_add_result setUe(UnavailableException ue) {
-      this.ue = ue;
-      return this;
-    }
-
-    public void unsetUe() {
-      this.ue = null;
-    }
-
-    /** Returns true if field ue is set (has been asigned a value) and false otherwise */
-    public boolean isSetUe() {
-      return this.ue != null;
-    }
-
-    public void setUeIsSet(boolean value) {
-      if (!value) {
-        this.ue = null;
-      }
-    }
-
-    public TimedOutException getTe() {
-      return this.te;
-    }
-
-    public batch_add_result setTe(TimedOutException te) {
-      this.te = te;
-      return this;
-    }
-
-    public void unsetTe() {
-      this.te = null;
-    }
-
-    /** Returns true if field te is set (has been asigned a value) and false otherwise */
-    public boolean isSetTe() {
-      return this.te != null;
-    }
-
-    public void setTeIsSet(boolean value) {
-      if (!value) {
-        this.te = null;
-      }
-    }
-
-    public void setFieldValue(_Fields field, Object value) {
-      switch (field) {
-      case IRE:
-        if (value == null) {
-          unsetIre();
-        } else {
-          setIre((InvalidRequestException)value);
-        }
-        break;
-
-      case UE:
-        if (value == null) {
-          unsetUe();
-        } else {
-          setUe((UnavailableException)value);
-        }
-        break;
-
-      case TE:
-        if (value == null) {
-          unsetTe();
-        } else {
-          setTe((TimedOutException)value);
-        }
-        break;
-
-      }
-    }
-
-    public Object getFieldValue(_Fields field) {
-      switch (field) {
-      case IRE:
-        return getIre();
-
-      case UE:
-        return getUe();
-
-      case TE:
-        return getTe();
-
-      }
-      throw new IllegalStateException();
-    }
-
-    /** Returns true if field corresponding to fieldID is set (has been asigned a value) and false otherwise */
-    public boolean isSet(_Fields field) {
-      if (field == null) {
-        throw new IllegalArgumentException();
-      }
-
-      switch (field) {
-      case IRE:
-        return isSetIre();
-      case UE:
-        return isSetUe();
-      case TE:
-        return isSetTe();
-      }
-      throw new IllegalStateException();
-    }
-
-    @Override
-    public boolean equals(Object that) {
-      if (that == null)
-        return false;
-      if (that instanceof batch_add_result)
-        return this.equals((batch_add_result)that);
-      return false;
-    }
-
-    public boolean equals(batch_add_result that) {
-      if (that == null)
-        return false;
-
-      boolean this_present_ire = true && this.isSetIre();
-      boolean that_present_ire = true && that.isSetIre();
-      if (this_present_ire || that_present_ire) {
-        if (!(this_present_ire && that_present_ire))
-          return false;
-        if (!this.ire.equals(that.ire))
-          return false;
-      }
-
-      boolean this_present_ue = true && this.isSetUe();
-      boolean that_present_ue = true && that.isSetUe();
-      if (this_present_ue || that_present_ue) {
-        if (!(this_present_ue && that_present_ue))
-          return false;
-        if (!this.ue.equals(that.ue))
-          return false;
-      }
-
-      boolean this_present_te = true && this.isSetTe();
-      boolean that_present_te = true && that.isSetTe();
-      if (this_present_te || that_present_te) {
-        if (!(this_present_te && that_present_te))
-          return false;
-        if (!this.te.equals(that.te))
-          return false;
-      }
-
-      return true;
-    }
-
-    @Override
-    public int hashCode() {
-      HashCodeBuilder builder = new HashCodeBuilder();
-
-      boolean present_ire = true && (isSetIre());
-      builder.append(present_ire);
-      if (present_ire)
-        builder.append(ire);
-
-      boolean present_ue = true && (isSetUe());
-      builder.append(present_ue);
-      if (present_ue)
-        builder.append(ue);
-
-      boolean present_te = true && (isSetTe());
-      builder.append(present_te);
-      if (present_te)
-        builder.append(te);
-
-      return builder.toHashCode();
-    }
-
-    public int compareTo(batch_add_result other) {
-      if (!getClass().equals(other.getClass())) {
-        return getClass().getName().compareTo(other.getClass().getName());
-      }
-
-      int lastComparison = 0;
-      batch_add_result typedOther = (batch_add_result)other;
-
-      lastComparison = Boolean.valueOf(isSetIre()).compareTo(typedOther.isSetIre());
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-      if (isSetIre()) {
-        lastComparison = TBaseHelper.compareTo(this.ire, typedOther.ire);
-        if (lastComparison != 0) {
-          return lastComparison;
-        }
-      }
-      lastComparison = Boolean.valueOf(isSetUe()).compareTo(typedOther.isSetUe());
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-      if (isSetUe()) {
-        lastComparison = TBaseHelper.compareTo(this.ue, typedOther.ue);
-        if (lastComparison != 0) {
-          return lastComparison;
-        }
-      }
-      lastComparison = Boolean.valueOf(isSetTe()).compareTo(typedOther.isSetTe());
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-      if (isSetTe()) {
-        lastComparison = TBaseHelper.compareTo(this.te, typedOther.te);
-        if (lastComparison != 0) {
-          return lastComparison;
-        }
-      }
-      return 0;
-    }
-
-    public _Fields fieldForId(int fieldId) {
-      return _Fields.findByThriftId(fieldId);
-    }
-
-    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: // IRE
-            if (field.type == TType.STRUCT) {
-              this.ire = new InvalidRequestException();
-              this.ire.read(iprot);
-            } else { 
-              TProtocolUtil.skip(iprot, field.type);
-            }
-            break;
-          case 2: // UE
-            if (field.type == TType.STRUCT) {
-              this.ue = new UnavailableException();
-              this.ue.read(iprot);
-            } else { 
-              TProtocolUtil.skip(iprot, field.type);
-            }
-            break;
-          case 3: // TE
-            if (field.type == TType.STRUCT) {
-              this.te = new TimedOutException();
-              this.te.read(iprot);
-            } else { 
-              TProtocolUtil.skip(iprot, field.type);
-            }
-            break;
-          default:
-            TProtocolUtil.skip(iprot, field.type);
-        }
-        iprot.readFieldEnd();
-      }
-      iprot.readStructEnd();
-
-      // check for required fields of primitive type, which can't be checked in the validate method
-      validate();
-    }
-
-    public void write(TProtocol oprot) throws TException {
-      oprot.writeStructBegin(STRUCT_DESC);
-
-      if (this.isSetIre()) {
-        oprot.writeFieldBegin(IRE_FIELD_DESC);
-        this.ire.write(oprot);
-        oprot.writeFieldEnd();
-      } else if (this.isSetUe()) {
-        oprot.writeFieldBegin(UE_FIELD_DESC);
-        this.ue.write(oprot);
-        oprot.writeFieldEnd();
-      } else if (this.isSetTe()) {
-        oprot.writeFieldBegin(TE_FIELD_DESC);
-        this.te.write(oprot);
-        oprot.writeFieldEnd();
-      }
-      oprot.writeFieldStop();
-      oprot.writeStructEnd();
-    }
-
-    @Override
-    public String toString() {
-      StringBuilder sb = new StringBuilder("batch_add_result(");
-      boolean first = true;
-
-      sb.append("ire:");
-      if (this.ire == null) {
-        sb.append("null");
-      } else {
-        sb.append(this.ire);
-      }
-      first = false;
-      if (!first) sb.append(", ");
-      sb.append("ue:");
-      if (this.ue == null) {
-        sb.append("null");
-      } else {
-        sb.append(this.ue);
-      }
-      first = false;
-      if (!first) sb.append(", ");
-      sb.append("te:");
-      if (this.te == null) {
-        sb.append("null");
-      } else {
-        sb.append(this.te);
-      }
-      first = false;
-      sb.append(")");
-      return sb.toString();
-    }
-
-    public void validate() throws TException {
-      // check for required fields
-    }
-
-  }
-
   public static class get_counter_args implements TBase<get_counter_args, get_counter_args._Fields>, java.io.Serializable, Cloneable   {
     private static final TStruct STRUCT_DESC = new TStruct("get_counter_args");
 
@@ -22609,14 +21495,14 @@ public class Cassandra {
           case 0: // SUCCESS
             if (field.type == TType.LIST) {
               {
-                TList _list112 = iprot.readListBegin();
-                this.success = new ArrayList<Counter>(_list112.size);
-                for (int _i113 = 0; _i113 < _list112.size; ++_i113)
+                TList _list98 = iprot.readListBegin();
+                this.success = new ArrayList<Counter>(_list98.size);
+                for (int _i99 = 0; _i99 < _list98.size; ++_i99)
                 {
-                  Counter _elem114;
-                  _elem114 = new Counter();
-                  _elem114.read(iprot);
-                  this.success.add(_elem114);
+                  Counter _elem100;
+                  _elem100 = new Counter();
+                  _elem100.read(iprot);
+                  this.success.add(_elem100);
                 }
                 iprot.readListEnd();
               }
@@ -22666,9 +21552,9 @@ public class Cassandra {
         oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
         {
           oprot.writeListBegin(new TList(TType.STRUCT, this.success.size()));
-          for (Counter _iter115 : this.success)
+          for (Counter _iter101 : this.success)
           {
-            _iter115.write(oprot);
+            _iter101.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -23237,13 +22123,13 @@ public class Cassandra {
           case 1: // KEYS
             if (field.type == TType.LIST) {
               {
-                TList _list116 = iprot.readListBegin();
-                this.keys = new ArrayList<ByteBuffer>(_list116.size);
-                for (int _i117 = 0; _i117 < _list116.size; ++_i117)
+                TList _list102 = iprot.readListBegin();
+                this.keys = new ArrayList<ByteBuffer>(_list102.size);
+                for (int _i103 = 0; _i103 < _list102.size; ++_i103)
                 {
-                  ByteBuffer _elem118;
-                  _elem118 = iprot.readBinary();
-                  this.keys.add(_elem118);
+                  ByteBuffer _elem104;
+                  _elem104 = iprot.readBinary();
+                  this.keys.add(_elem104);
                 }
                 iprot.readListEnd();
               }
@@ -23293,9 +22179,9 @@ public class Cassandra {
         oprot.writeFieldBegin(KEYS_FIELD_DESC);
         {
           oprot.writeListBegin(new TList(TType.STRING, this.keys.size()));
-          for (ByteBuffer _iter119 : this.keys)
+          for (ByteBuffer _iter105 : this.keys)
           {
-            oprot.writeBinary(_iter119);
+            oprot.writeBinary(_iter105);
           }
           oprot.writeListEnd();
         }
@@ -23868,26 +22754,26 @@ public class Cassandra {
           case 0: // SUCCESS
             if (field.type == TType.MAP) {
               {
-                TMap _map120 = iprot.readMapBegin();
-                this.success = new HashMap<ByteBuffer,List<Counter>>(2*_map120.size);
-                for (int _i121 = 0; _i121 < _map120.size; ++_i121)
+                TMap _map106 = iprot.readMapBegin();
+                this.success = new HashMap<ByteBuffer,List<Counter>>(2*_map106.size);
+                for (int _i107 = 0; _i107 < _map106.size; ++_i107)
                 {
-                  ByteBuffer _key122;
-                  List<Counter> _val123;
-                  _key122 = iprot.readBinary();
+                  ByteBuffer _key108;
+                  List<Counter> _val109;
+                  _key108 = iprot.readBinary();
                   {
-                    TList _list124 = iprot.readListBegin();
-                    _val123 = new ArrayList<Counter>(_list124.size);
-                    for (int _i125 = 0; _i125 < _list124.size; ++_i125)
+                    TList _list110 = iprot.readListBegin();
+                    _val109 = new ArrayList<Counter>(_list110.size);
+                    for (int _i111 = 0; _i111 < _list110.size; ++_i111)
                     {
-                      Counter _elem126;
-                      _elem126 = new Counter();
-                      _elem126.read(iprot);
-                      _val123.add(_elem126);
+                      Counter _elem112;
+                      _elem112 = new Counter();
+                      _elem112.read(iprot);
+                      _val109.add(_elem112);
                     }
                     iprot.readListEnd();
                   }
-                  this.success.put(_key122, _val123);
+                  this.success.put(_key108, _val109);
                 }
                 iprot.readMapEnd();
               }
@@ -23937,14 +22823,14 @@ public class Cassandra {
         oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
         {
           oprot.writeMapBegin(new TMap(TType.STRING, TType.LIST, this.success.size()));
-          for (Map.Entry<ByteBuffer, List<Counter>> _iter127 : this.success.entrySet())
+          for (Map.Entry<ByteBuffer, List<Counter>> _iter113 : this.success.entrySet())
           {
-            oprot.writeBinary(_iter127.getKey());
+            oprot.writeBinary(_iter113.getKey());
             {
-              oprot.writeListBegin(new TList(TType.STRUCT, _iter127.getValue().size()));
-              for (Counter _iter128 : _iter127.getValue())
+              oprot.writeListBegin(new TList(TType.STRUCT, _iter113.getValue().size()));
+              for (Counter _iter114 : _iter113.getValue())
               {
-                _iter128.write(oprot);
+                _iter114.write(oprot);
               }
               oprot.writeListEnd();
             }
@@ -25536,25 +24422,25 @@ public class Cassandra {
           case 0: // SUCCESS
             if (field.type == TType.MAP) {
               {
-                TMap _map129 = iprot.readMapBegin();
-                this.success = new HashMap<String,List<String>>(2*_map129.size);
-                for (int _i130 = 0; _i130 < _map129.size; ++_i130)
+                TMap _map115 = iprot.readMapBegin();
+                this.success = new HashMap<String,List<String>>(2*_map115.size);
+                for (int _i116 = 0; _i116 < _map115.size; ++_i116)
                 {
-                  String _key131;
-                  List<String> _val132;
-                  _key131 = iprot.readString();
+                  String _key117;
+                  List<String> _val118;
+                  _key117 = iprot.readString();
                   {
-                    TList _list133 = iprot.readListBegin();
-                    _val132 = new ArrayList<String>(_list133.size);
-                    for (int _i134 = 0; _i134 < _list133.size; ++_i134)
+                    TList _list119 = iprot.readListBegin();
+                    _val118 = new ArrayList<String>(_list119.size);
+                    for (int _i120 = 0; _i120 < _list119.size; ++_i120)
                     {
-                      String _elem135;
-                      _elem135 = iprot.readString();
-                      _val132.add(_elem135);
+                      String _elem121;
+                      _elem121 = iprot.readString();
+                      _val118.add(_elem121);
                     }
                     iprot.readListEnd();
                   }
-                  this.success.put(_key131, _val132);
+                  this.success.put(_key117, _val118);
                 }
                 iprot.readMapEnd();
               }
@@ -25588,14 +24474,14 @@ public class Cassandra {
         oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
         {
           oprot.writeMapBegin(new TMap(TType.STRING, TType.LIST, this.success.size()));
-          for (Map.Entry<String, List<String>> _iter136 : this.success.entrySet())
+          for (Map.Entry<String, List<String>> _iter122 : this.success.entrySet())
           {
-            oprot.writeString(_iter136.getKey());
+            oprot.writeString(_iter122.getKey());
             {
-              oprot.writeListBegin(new TList(TType.STRING, _iter136.getValue().size()));
-              for (String _iter137 : _iter136.getValue())
+              oprot.writeListBegin(new TList(TType.STRING, _iter122.getValue().size()));
+              for (String _iter123 : _iter122.getValue())
               {
-                oprot.writeString(_iter137);
+                oprot.writeString(_iter123);
               }
               oprot.writeListEnd();
             }
@@ -26162,14 +25048,14 @@ public class Cassandra {
           case 0: // SUCCESS
             if (field.type == TType.LIST) {
               {
-                TList _list138 = iprot.readListBegin();
-                this.success = new ArrayList<KsDef>(_list138.size);
-                for (int _i139 = 0; _i139 < _list138.size; ++_i139)
+                TList _list124 = iprot.readListBegin();
+                this.success = new ArrayList<KsDef>(_list124.size);
+                for (int _i125 = 0; _i125 < _list124.size; ++_i125)
                 {
-                  KsDef _elem140;
-                  _elem140 = new KsDef();
-                  _elem140.read(iprot);
-                  this.success.add(_elem140);
+                  KsDef _elem126;
+                  _elem126 = new KsDef();
+                  _elem126.read(iprot);
+                  this.success.add(_elem126);
                 }
                 iprot.readListEnd();
               }
@@ -26203,9 +25089,9 @@ public class Cassandra {
         oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
         {
           oprot.writeListBegin(new TList(TType.STRUCT, this.success.size()));
-          for (KsDef _iter141 : this.success)
+          for (KsDef _iter127 : this.success)
           {
-            _iter141.write(oprot);
+            _iter127.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -27822,14 +26708,14 @@ public class Cassandra {
           case 0: // SUCCESS
             if (field.type == TType.LIST) {
               {
-                TList _list142 = iprot.readListBegin();
-                this.success = new ArrayList<TokenRange>(_list142.size);
-                for (int _i143 = 0; _i143 < _list142.size; ++_i143)
+                TList _list128 = iprot.readListBegin();
+                this.success = new ArrayList<TokenRange>(_list128.size);
+                for (int _i129 = 0; _i129 < _list128.size; ++_i129)
                 {
-                  TokenRange _elem144;
-                  _elem144 = new TokenRange();
-                  _elem144.read(iprot);
-                  this.success.add(_elem144);
+                  TokenRange _elem130;
+                  _elem130 = new TokenRange();
+                  _elem130.read(iprot);
+                  this.success.add(_elem130);
                 }
                 iprot.readListEnd();
               }
@@ -27863,9 +26749,9 @@ public class Cassandra {
         oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
         {
           oprot.writeListBegin(new TList(TType.STRUCT, this.success.size()));
-          for (TokenRange _iter145 : this.success)
+          for (TokenRange _iter131 : this.success)
           {
-            _iter145.write(oprot);
+            _iter131.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -30539,13 +29425,13 @@ public class Cassandra {
           case 0: // SUCCESS
             if (field.type == TType.LIST) {
               {
-                TList _list146 = iprot.readListBegin();
-                this.success = new ArrayList<String>(_list146.size);
-                for (int _i147 = 0; _i147 < _list146.size; ++_i147)
+                TList _list132 = iprot.readListBegin();
+                this.success = new ArrayList<String>(_list132.size);
+                for (int _i133 = 0; _i133 < _list132.size; ++_i133)
                 {
-                  String _elem148;
-                  _elem148 = iprot.readString();
-                  this.success.add(_elem148);
+                  String _elem134;
+                  _elem134 = iprot.readString();
+                  this.success.add(_elem134);
                 }
                 iprot.readListEnd();
               }
@@ -30579,9 +29465,9 @@ public class Cassandra {
         oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
         {
           oprot.writeListBegin(new TList(TType.STRING, this.success.size()));
-          for (String _iter149 : this.success)
+          for (String _iter135 : this.success)
           {
-            oprot.writeString(_iter149);
+            oprot.writeString(_iter135);
           }
           oprot.writeListEnd();
         }

Modified: cassandra/trunk/interface/thrift/gen-java/org/apache/cassandra/thrift/Deletion.java
URL: http://svn.apache.org/viewvc/cassandra/trunk/interface/thrift/gen-java/org/apache/cassandra/thrift/Deletion.java?rev=1089517&r1=1089516&r2=1089517&view=diff
==============================================================================
--- cassandra/trunk/interface/thrift/gen-java/org/apache/cassandra/thrift/Deletion.java (original)
+++ cassandra/trunk/interface/thrift/gen-java/org/apache/cassandra/thrift/Deletion.java Wed Apr  6 16:19:55 2011
@@ -48,6 +48,9 @@ import org.apache.thrift.meta_data.*;
 import org.apache.thrift.transport.*;
 import org.apache.thrift.protocol.*;
 
+/**
+ * Note that the timestamp is only optional in case of counter deletion.
+ */
 public class Deletion implements TBase<Deletion, Deletion._Fields>, java.io.Serializable, Cloneable {
   private static final TStruct STRUCT_DESC = new TStruct("Deletion");
 
@@ -130,7 +133,7 @@ public class Deletion implements TBase<D
   public static final Map<_Fields, FieldMetaData> metaDataMap;
   static {
     Map<_Fields, FieldMetaData> tmpMap = new EnumMap<_Fields, FieldMetaData>(_Fields.class);
-    tmpMap.put(_Fields.TIMESTAMP, new FieldMetaData("timestamp", TFieldRequirementType.REQUIRED, 
+    tmpMap.put(_Fields.TIMESTAMP, new FieldMetaData("timestamp", TFieldRequirementType.OPTIONAL, 
         new FieldValueMetaData(TType.I64)));
     tmpMap.put(_Fields.SUPER_COLUMN, new FieldMetaData("super_column", TFieldRequirementType.OPTIONAL, 
         new FieldValueMetaData(TType.STRING)));
@@ -143,14 +146,6 @@ public class Deletion implements TBase<D
   public Deletion() {
   }
 
-  public Deletion(
-    long timestamp)
-  {
-    this();
-    this.timestamp = timestamp;
-    setTimestampIsSet(true);
-  }
-
   /**
    * Performs a deep copy on <i>other</i>.
    */
@@ -334,8 +329,8 @@ public class Deletion implements TBase<D
     if (that == null)
       return false;
 
-    boolean this_present_timestamp = true;
-    boolean that_present_timestamp = true;
+    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;
@@ -368,7 +363,7 @@ public class Deletion implements TBase<D
   public int hashCode() {
     HashCodeBuilder builder = new HashCodeBuilder();
 
-    boolean present_timestamp = true;
+    boolean present_timestamp = true && (isSetTimestamp());
     builder.append(present_timestamp);
     if (present_timestamp)
       builder.append(timestamp);
@@ -472,9 +467,6 @@ public class Deletion implements TBase<D
     iprot.readStructEnd();
 
     // check for required fields of primitive type, which can't be checked in the validate method
-    if (!isSetTimestamp()) {
-      throw new TProtocolException("Required field 'timestamp' was not found in serialized data! Struct: " + toString());
-    }
     validate();
   }
 
@@ -482,9 +474,11 @@ public class Deletion implements TBase<D
     validate();
 
     oprot.writeStructBegin(STRUCT_DESC);
-    oprot.writeFieldBegin(TIMESTAMP_FIELD_DESC);
-    oprot.writeI64(this.timestamp);
-    oprot.writeFieldEnd();
+    if (isSetTimestamp()) {
+      oprot.writeFieldBegin(TIMESTAMP_FIELD_DESC);
+      oprot.writeI64(this.timestamp);
+      oprot.writeFieldEnd();
+    }
     if (this.super_column != null) {
       if (isSetSuper_column()) {
         oprot.writeFieldBegin(SUPER_COLUMN_FIELD_DESC);
@@ -508,9 +502,11 @@ public class Deletion implements TBase<D
     StringBuilder sb = new StringBuilder("Deletion(");
     boolean first = true;
 
-    sb.append("timestamp:");
-    sb.append(this.timestamp);
-    first = false;
+    if (isSetTimestamp()) {
+      sb.append("timestamp:");
+      sb.append(this.timestamp);
+      first = false;
+    }
     if (isSetSuper_column()) {
       if (!first) sb.append(", ");
       sb.append("super_column:");
@@ -537,7 +533,6 @@ public class Deletion implements TBase<D
 
   public void validate() throws TException {
     // check for required fields
-    // alas, we cannot check 'timestamp' because it's a primitive and you chose the non-beans generator.
   }
 
 }

Modified: cassandra/trunk/interface/thrift/gen-java/org/apache/cassandra/thrift/Mutation.java
URL: http://svn.apache.org/viewvc/cassandra/trunk/interface/thrift/gen-java/org/apache/cassandra/thrift/Mutation.java?rev=1089517&r1=1089516&r2=1089517&view=diff
==============================================================================
--- cassandra/trunk/interface/thrift/gen-java/org/apache/cassandra/thrift/Mutation.java (original)
+++ cassandra/trunk/interface/thrift/gen-java/org/apache/cassandra/thrift/Mutation.java Wed Apr  6 16:19:55 2011
@@ -49,7 +49,8 @@ import org.apache.thrift.transport.*;
 import org.apache.thrift.protocol.*;
 
 /**
- * A Mutation is either an insert, represented by filling column_or_supercolumn, or a deletion, represented by filling the deletion attribute.
+ * A Mutation is either an insert (represented by filling column_or_supercolumn), a deletion (represented by filling the deletion attribute),
+ * a counter addition (represented by filling counter), or a counter deletion (represented by filling counter_deletion).
  * @param column_or_supercolumn. An insert to a column or supercolumn
  * @param deletion. A deletion of a column or supercolumn
  */
@@ -58,14 +59,17 @@ public class Mutation implements TBase<M
 
   private static final TField COLUMN_OR_SUPERCOLUMN_FIELD_DESC = new TField("column_or_supercolumn", TType.STRUCT, (short)1);
   private static final TField DELETION_FIELD_DESC = new TField("deletion", TType.STRUCT, (short)2);
+  private static final TField COUNTER_FIELD_DESC = new TField("counter", TType.STRUCT, (short)3);
 
   public ColumnOrSuperColumn column_or_supercolumn;
   public Deletion deletion;
+  public Counter counter;
 
   /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
   public enum _Fields implements TFieldIdEnum {
     COLUMN_OR_SUPERCOLUMN((short)1, "column_or_supercolumn"),
-    DELETION((short)2, "deletion");
+    DELETION((short)2, "deletion"),
+    COUNTER((short)3, "counter");
 
     private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -84,6 +88,8 @@ public class Mutation implements TBase<M
           return COLUMN_OR_SUPERCOLUMN;
         case 2: // DELETION
           return DELETION;
+        case 3: // COUNTER
+          return COUNTER;
         default:
           return null;
       }
@@ -132,6 +138,8 @@ public class Mutation implements TBase<M
         new StructMetaData(TType.STRUCT, ColumnOrSuperColumn.class)));
     tmpMap.put(_Fields.DELETION, new FieldMetaData("deletion", TFieldRequirementType.OPTIONAL, 
         new StructMetaData(TType.STRUCT, Deletion.class)));
+    tmpMap.put(_Fields.COUNTER, new FieldMetaData("counter", TFieldRequirementType.OPTIONAL, 
+        new StructMetaData(TType.STRUCT, Counter.class)));
     metaDataMap = Collections.unmodifiableMap(tmpMap);
     FieldMetaData.addStructMetaDataMap(Mutation.class, metaDataMap);
   }
@@ -149,6 +157,9 @@ public class Mutation implements TBase<M
     if (other.isSetDeletion()) {
       this.deletion = new Deletion(other.deletion);
     }
+    if (other.isSetCounter()) {
+      this.counter = new Counter(other.counter);
+    }
   }
 
   public Mutation deepCopy() {
@@ -159,6 +170,7 @@ public class Mutation implements TBase<M
   public void clear() {
     this.column_or_supercolumn = null;
     this.deletion = null;
+    this.counter = null;
   }
 
   public ColumnOrSuperColumn getColumn_or_supercolumn() {
@@ -209,6 +221,30 @@ public class Mutation implements TBase<M
     }
   }
 
+  public Counter getCounter() {
+    return this.counter;
+  }
+
+  public Mutation setCounter(Counter counter) {
+    this.counter = counter;
+    return this;
+  }
+
+  public void unsetCounter() {
+    this.counter = null;
+  }
+
+  /** Returns true if field counter is set (has been asigned a value) and false otherwise */
+  public boolean isSetCounter() {
+    return this.counter != null;
+  }
+
+  public void setCounterIsSet(boolean value) {
+    if (!value) {
+      this.counter = null;
+    }
+  }
+
   public void setFieldValue(_Fields field, Object value) {
     switch (field) {
     case COLUMN_OR_SUPERCOLUMN:
@@ -227,6 +263,14 @@ public class Mutation implements TBase<M
       }
       break;
 
+    case COUNTER:
+      if (value == null) {
+        unsetCounter();
+      } else {
+        setCounter((Counter)value);
+      }
+      break;
+
     }
   }
 
@@ -238,6 +282,9 @@ public class Mutation implements TBase<M
     case DELETION:
       return getDeletion();
 
+    case COUNTER:
+      return getCounter();
+
     }
     throw new IllegalStateException();
   }
@@ -253,6 +300,8 @@ public class Mutation implements TBase<M
       return isSetColumn_or_supercolumn();
     case DELETION:
       return isSetDeletion();
+    case COUNTER:
+      return isSetCounter();
     }
     throw new IllegalStateException();
   }
@@ -288,6 +337,15 @@ public class Mutation implements TBase<M
         return false;
     }
 
+    boolean this_present_counter = true && this.isSetCounter();
+    boolean that_present_counter = true && that.isSetCounter();
+    if (this_present_counter || that_present_counter) {
+      if (!(this_present_counter && that_present_counter))
+        return false;
+      if (!this.counter.equals(that.counter))
+        return false;
+    }
+
     return true;
   }
 
@@ -305,6 +363,11 @@ public class Mutation implements TBase<M
     if (present_deletion)
       builder.append(deletion);
 
+    boolean present_counter = true && (isSetCounter());
+    builder.append(present_counter);
+    if (present_counter)
+      builder.append(counter);
+
     return builder.toHashCode();
   }
 
@@ -336,6 +399,16 @@ public class Mutation implements TBase<M
         return lastComparison;
       }
     }
+    lastComparison = Boolean.valueOf(isSetCounter()).compareTo(typedOther.isSetCounter());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetCounter()) {
+      lastComparison = TBaseHelper.compareTo(this.counter, typedOther.counter);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
     return 0;
   }
 
@@ -369,6 +442,14 @@ public class Mutation implements TBase<M
             TProtocolUtil.skip(iprot, field.type);
           }
           break;
+        case 3: // COUNTER
+          if (field.type == TType.STRUCT) {
+            this.counter = new Counter();
+            this.counter.read(iprot);
+          } else { 
+            TProtocolUtil.skip(iprot, field.type);
+          }
+          break;
         default:
           TProtocolUtil.skip(iprot, field.type);
       }
@@ -398,6 +479,13 @@ public class Mutation implements TBase<M
         oprot.writeFieldEnd();
       }
     }
+    if (this.counter != null) {
+      if (isSetCounter()) {
+        oprot.writeFieldBegin(COUNTER_FIELD_DESC);
+        this.counter.write(oprot);
+        oprot.writeFieldEnd();
+      }
+    }
     oprot.writeFieldStop();
     oprot.writeStructEnd();
   }
@@ -426,6 +514,16 @@ public class Mutation implements TBase<M
       }
       first = false;
     }
+    if (isSetCounter()) {
+      if (!first) sb.append(", ");
+      sb.append("counter:");
+      if (this.counter == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.counter);
+      }
+      first = false;
+    }
     sb.append(")");
     return sb.toString();
   }

Modified: cassandra/trunk/src/java/org/apache/cassandra/db/ColumnFamily.java
URL: http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/db/ColumnFamily.java?rev=1089517&r1=1089516&r2=1089517&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/db/ColumnFamily.java (original)
+++ cassandra/trunk/src/java/org/apache/cassandra/db/ColumnFamily.java Wed Apr  6 16:19:55 2011
@@ -158,22 +158,21 @@ public class ColumnFamily implements ICo
     public void addColumn(QueryPath path, ByteBuffer value, long timestamp, int timeToLive)
     {
         assert path.columnName != null : path;
+        assert !metadata().getDefaultValidator().isCommutative();
         Column column;
-        AbstractType defaultValidator = metadata().getDefaultValidator();
-        if (!defaultValidator.isCommutative())
-        {
-            if (timeToLive > 0)
-                column = new ExpiringColumn(path.columnName, value, timestamp, timeToLive);
-            else
-                column = new Column(path.columnName, value, timestamp);
-        }
+        if (timeToLive > 0)
+            column = new ExpiringColumn(path.columnName, value, timestamp, timeToLive);
         else
-        {
-            column = ((AbstractCommutativeType)defaultValidator).createColumn(path.columnName, value, timestamp);
-        }
+            column = new Column(path.columnName, value, timestamp);
         addColumn(path.superColumnName, column);
     }
 
+    public void addCounter(QueryPath path, long value)
+    {
+        assert path.columnName != null : path;
+        addColumn(path.superColumnName, new CounterUpdateColumn(path.columnName, value, System.currentTimeMillis()));
+    }
+
     public void addTombstone(QueryPath path, ByteBuffer localDeletionTime, long timestamp)
     {
         assert path.columnName != null : path;

Modified: cassandra/trunk/src/java/org/apache/cassandra/db/RowMutation.java
URL: http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/db/RowMutation.java?rev=1089517&r1=1089516&r2=1089517&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/db/RowMutation.java (original)
+++ cassandra/trunk/src/java/org/apache/cassandra/db/RowMutation.java Wed Apr  6 16:19:55 2011
@@ -33,8 +33,9 @@ import org.apache.cassandra.io.ICompactS
 import org.apache.cassandra.net.Message;
 import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.thrift.ColumnOrSuperColumn;
-import org.apache.cassandra.thrift.Deletion;
 import org.apache.cassandra.thrift.Mutation;
+import org.apache.cassandra.thrift.Counter;
+import org.apache.cassandra.thrift.SlicePredicate;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.FBUtilities;
 
@@ -146,6 +147,18 @@ public class RowMutation implements IMut
         columnFamily.addColumn(path, value, timestamp, timeToLive);
     }
 
+    public void addCounter(QueryPath path, long value)
+    {
+        Integer id = CFMetaData.getId(table_, path.columnFamilyName);
+        ColumnFamily columnFamily = modifications_.get(id);
+        if (columnFamily == null)
+        {
+            columnFamily = ColumnFamily.create(table_, path.columnFamilyName);
+            modifications_.put(id, columnFamily);
+        }
+        columnFamily.addCounter(path, value);
+    }
+
     public void add(QueryPath path, ByteBuffer value, long timestamp)
     {
         add(path, value, timestamp, 0);
@@ -223,12 +236,16 @@ public class RowMutation implements IMut
             {
                 if (mutation.deletion != null)
                 {
-                    deleteColumnOrSuperColumnToRowMutation(rm, cfName, mutation.deletion);
+                    deleteColumnOrSuperColumnToRowMutation(rm, cfName, mutation.deletion.predicate, mutation.deletion.super_column, mutation.deletion.timestamp);
                 }
-                else
+                if (mutation.column_or_supercolumn != null)
                 {
                     addColumnOrSuperColumnToRowMutation(rm, cfName, mutation.column_or_supercolumn);
                 }
+                if (mutation.counter != null)
+                {
+                    addCounterToRowMutation(rm, cfName, mutation.counter);
+                }
             }
         }
         return rm;
@@ -290,21 +307,33 @@ public class RowMutation implements IMut
         }
     }
 
-    private static void deleteColumnOrSuperColumnToRowMutation(RowMutation rm, String cfName, Deletion del)
+    private static void addCounterToRowMutation(RowMutation rm, String cfName, Counter counter)
+    {
+        if (counter.column == null)
+        {
+            for (org.apache.cassandra.thrift.CounterColumn column : counter.super_column.columns)
+            {
+                rm.addCounter(new QueryPath(cfName, counter.super_column.name, column.name), column.value);
+            }
+        }
+        else
+        {
+            rm.addCounter(new QueryPath(cfName, null, counter.column.name), counter.column.value);
+        }
+    }
+
+    private static void deleteColumnOrSuperColumnToRowMutation(RowMutation rm, String cfName, SlicePredicate predicate, ByteBuffer scName, long timestamp)
     {
-        if (del.predicate != null && del.predicate.column_names != null)
+        if (predicate != null && predicate.column_names != null)
         {
-            for(ByteBuffer c : del.predicate.column_names)
+            for (ByteBuffer c : predicate.column_names)
             {
-                if (del.super_column == null && DatabaseDescriptor.getColumnFamilyType(rm.table_, cfName) == ColumnFamilyType.Super)
-                    rm.delete(new QueryPath(cfName, c), del.timestamp);
-                else
-                    rm.delete(new QueryPath(cfName, del.super_column, c), del.timestamp);
+                rm.delete(new QueryPath(cfName, scName, c), timestamp);
             }
         }
         else
         {
-            rm.delete(new QueryPath(cfName, del.super_column), del.timestamp);
+            rm.delete(new QueryPath(cfName, scName), timestamp);
         }
     }
 

Modified: cassandra/trunk/src/java/org/apache/cassandra/hadoop/ColumnFamilyRecordWriter.java
URL: http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/hadoop/ColumnFamilyRecordWriter.java?rev=1089517&r1=1089516&r2=1089517&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/hadoop/ColumnFamilyRecordWriter.java (original)
+++ cassandra/trunk/src/java/org/apache/cassandra/hadoop/ColumnFamilyRecordWriter.java Wed Apr  6 16:19:55 2011
@@ -151,7 +151,7 @@ implements org.apache.hadoop.mapred.Reco
         {
             // deletion
             assert amut.deletion != null;
-            Deletion deletion = new Deletion(amut.deletion.timestamp);
+            Deletion deletion = new Deletion().setTimestamp(amut.deletion.timestamp);
             mutation.setDeletion(deletion);
 
             org.apache.cassandra.hadoop.avro.SlicePredicate apred = amut.deletion.predicate;

Modified: cassandra/trunk/src/java/org/apache/cassandra/thrift/CassandraServer.java
URL: http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/thrift/CassandraServer.java?rev=1089517&r1=1089516&r2=1089517&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/thrift/CassandraServer.java (original)
+++ cassandra/trunk/src/java/org/apache/cassandra/thrift/CassandraServer.java Wed Apr  6 16:19:55 2011
@@ -344,15 +344,12 @@ public class CassandraServer implements 
         return counts;
     }
 
-    private void internal_insert(ByteBuffer key, ColumnParent column_parent, Column column, ConsistencyLevel consistency_level, boolean isCommutativeOp)
+    private void internal_insert(ByteBuffer key, ColumnParent column_parent, Column column, ConsistencyLevel consistency_level)
     throws InvalidRequestException, UnavailableException, TimedOutException
     {
         state().hasColumnFamilyAccess(column_parent.column_family, Permission.WRITE);
 
-        CFMetaData metadata = ThriftValidation.validateColumnFamily(state().getKeyspace(), column_parent.column_family, isCommutativeOp);
-        ThriftValidation.validateKey(metadata, key);
-        if (isCommutativeOp)
-            ThriftValidation.validateCommutativeForWrite(metadata, consistency_level);
+        CFMetaData metadata = ThriftValidation.validateColumnFamily(state().getKeyspace(), column_parent.column_family, false);
         ThriftValidation.validateColumnNames(metadata, column_parent, Arrays.asList(column.name));
         ThriftValidation.validateColumnData(metadata, column);
 
@@ -373,15 +370,16 @@ public class CassandraServer implements 
     {
         logger.debug("insert");
 
-        internal_insert(key, column_parent, column, consistency_level, false);
+        internal_insert(key, column_parent, column, consistency_level);
     }
 
-    private void internal_batch_mutate(Map<ByteBuffer,Map<String,List<Mutation>>> mutation_map, ConsistencyLevel consistency_level, boolean isCommutativeOp)
+    private void internal_batch_mutate(Map<ByteBuffer,Map<String,List<Mutation>>> mutation_map, ConsistencyLevel consistency_level)
     throws InvalidRequestException, UnavailableException, TimedOutException
     {
         List<String> cfamsSeen = new ArrayList<String>();
-
         List<RowMutation> rowMutations = new ArrayList<RowMutation>();
+        String keyspace = state().getKeyspace();
+
         for (Map.Entry<ByteBuffer, Map<String, List<Mutation>>> mutationEntry: mutation_map.entrySet())
         {
             ByteBuffer key = mutationEntry.getKey();
@@ -390,8 +388,6 @@ public class CassandraServer implements 
             for (Map.Entry<String, List<Mutation>> columnFamilyMutations : columnFamilyToMutations.entrySet())
             {
                 String cfName = columnFamilyMutations.getKey();
-                CFMetaData metadata = ThriftValidation.validateColumnFamily(state().getKeyspace(), cfName, isCommutativeOp);
-                ThriftValidation.validateKey(metadata, key);
 
                 // Avoid unneeded authorizations
                 if (!(cfamsSeen.contains(cfName)))
@@ -400,7 +396,12 @@ public class CassandraServer implements 
                     cfamsSeen.add(cfName);
                 }
 
-                if (isCommutativeOp)
+                boolean isCommutativeOp = false;
+                boolean isOnlyDeletion = true;
+                CFMetaData metadata = ThriftValidation.validateColumnFamily(keyspace, cfName);
+                ThriftValidation.validateKey(metadata, key);
+
+                if (metadata.getDefaultValidator().isCommutative())
                     ThriftValidation.validateCommutativeForWrite(metadata, consistency_level);
 
                 for (Mutation mutation : columnFamilyMutations.getValue())
@@ -408,7 +409,7 @@ public class CassandraServer implements 
                     ThriftValidation.validateMutation(metadata, mutation);
                 }
             }
-            RowMutation rm = RowMutation.getRowMutationFromMutations(state().getKeyspace(), key, columnFamilyToMutations);
+            RowMutation rm = RowMutation.getRowMutationFromMutations(keyspace, key, columnFamilyToMutations);
             if (!rm.isEmpty())
                 rowMutations.add(rm);
         }
@@ -421,7 +422,7 @@ public class CassandraServer implements 
     {
         logger.debug("batch_mutate");
 
-        internal_batch_mutate(mutation_map, consistency_level, false);
+        internal_batch_mutate(mutation_map, consistency_level);
     }
 
     private void internal_remove(ByteBuffer key, ColumnPath column_path, long timestamp, ConsistencyLevel consistency_level, boolean isCommutativeOp)
@@ -999,86 +1000,30 @@ public class CassandraServer implements 
 
     // counter methods
 
-    private Column getCounterColumn(CounterColumn column)
-    {
-        return new Column(column.name, ByteBufferUtil.bytes(column.value), System.currentTimeMillis());
-    }
-
     public void add(ByteBuffer key, ColumnParent column_parent, CounterColumn column, ConsistencyLevel consistency_level)
             throws InvalidRequestException, UnavailableException, TimedOutException, TException
     {
         logger.debug("add");
 
-        internal_insert(key, column_parent, getCounterColumn(column), consistency_level, true);
-    }
-
-    private Mutation getMutation(CounterMutation counterMutation)
-    {
-        Mutation mutation = new Mutation();
-
-        if (counterMutation.isSetCounter())
-        {
-            Counter counter = counterMutation.counter;
-            ColumnOrSuperColumn cosc = new ColumnOrSuperColumn();
-            if (counter.isSetColumn())
-            {
-                Column c = new Column(counter.column.name, ByteBufferUtil.bytes(counter.column.value), System.currentTimeMillis());
-                cosc.setColumn(c);
-            }
+        state().hasColumnFamilyAccess(column_parent.column_family, Permission.WRITE);
+        String keyspace = state().getKeyspace();
 
-            if (counter.isSetSuper_column())
-            {
-                List<Column> subcolumns = new ArrayList<Column>(counter.super_column.columns.size());
-                for (CounterColumn subcol : counter.super_column.columns)
-                {
-                    subcolumns.add(new Column(subcol.name, ByteBufferUtil.bytes(subcol.value), System.currentTimeMillis()));
-                }
-                SuperColumn sc = new SuperColumn(counter.super_column.name, subcolumns);
-                cosc.setSuper_column(sc);
-            }
-            mutation.setColumn_or_supercolumn(cosc);
-        }
+        CFMetaData metadata = ThriftValidation.validateColumnFamily(keyspace, column_parent.column_family, true);
+        ThriftValidation.validateKey(metadata, key);
+        ThriftValidation.validateCommutativeForWrite(metadata, consistency_level);
+        ThriftValidation.validateColumnParent(metadata, column_parent);
+        ThriftValidation.validateColumnNames(metadata, column_parent, Arrays.asList(column.name));
 
-        if (counterMutation.isSetDeletion())
+        RowMutation rm = new RowMutation(keyspace, key);
+        try
         {
-            Deletion deletion = new Deletion(System.currentTimeMillis());
-            if (counterMutation.deletion.isSetSuper_column())
-                deletion.setSuper_column(counterMutation.deletion.super_column);
-            if (counterMutation.deletion.isSetPredicate())
-                deletion.setPredicate(counterMutation.deletion.predicate);
-            mutation.setDeletion(deletion);
+            rm.addCounter(new QueryPath(column_parent.column_family, column_parent.super_column, column.name), column.value);
         }
-
-        return mutation;
-    }
-
-    public void batch_add(Map<ByteBuffer, Map<String, List<CounterMutation>>> updateMap, ConsistencyLevel consistency_level)
-            throws InvalidRequestException, UnavailableException, TimedOutException, TException
-    {
-        logger.debug("batch_add");
-
-        String keyspace = state().getKeyspace();
-        
-        Map<ByteBuffer,Map<String,List<Mutation>>> mutation_map = new HashMap<ByteBuffer,Map<String,List<Mutation>>>();
-        
-        for (Entry<ByteBuffer, Map<String, List<CounterMutation>>> entry : updateMap.entrySet())
+        catch (MarshalException e)
         {
-            Map<String, List<Mutation>> valueMap = new HashMap<String, List<Mutation>>(entry.getValue().size());
-            
-            for (Entry<String, List<CounterMutation>> innerEntry : entry.getValue().entrySet())
-            {
-                List<Mutation> mutations = new ArrayList<Mutation>(innerEntry.getValue().size());
-                for (CounterMutation cm : innerEntry.getValue())
-                {
-                    mutations.add(getMutation(cm));
-                }
-                valueMap.put(innerEntry.getKey(), mutations);
-            }
-            
-            mutation_map.put(entry.getKey(), valueMap);
+            throw new InvalidRequestException(e.getMessage());
         }
-        
-        internal_batch_mutate(mutation_map, consistency_level, true);
+        doInsert(consistency_level, Arrays.asList(rm));
     }
 
     private Counter getCounter(ColumnOrSuperColumn cosc)