You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by jb...@apache.org on 2013/12/06 16:52:18 UTC

[01/16] add describe_local_ring for Hadoop's benefit patch by Piotr Kołaczkowski; reviewed by jbellis for CASSANDRA-6268

Updated Branches:
  refs/heads/cassandra-1.2 57613dc8c -> 86e949f32
  refs/heads/cassandra-2.0 f54f4e2c6 -> f2a82ee73
  refs/heads/trunk c00e05b65 -> e6170fef9


http://git-wip-us.apache.org/repos/asf/cassandra/blob/86e949f3/interface/thrift/gen-java/org/apache/cassandra/thrift/CfDef.java
----------------------------------------------------------------------
diff --git a/interface/thrift/gen-java/org/apache/cassandra/thrift/CfDef.java b/interface/thrift/gen-java/org/apache/cassandra/thrift/CfDef.java
index 60aa07f..059d630 100644
--- a/interface/thrift/gen-java/org/apache/cassandra/thrift/CfDef.java
+++ b/interface/thrift/gen-java/org/apache/cassandra/thrift/CfDef.java
@@ -32,10 +32,15 @@ import java.util.ArrayList;
 import java.util.Map;
 import java.util.HashMap;
 import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
 import java.util.EnumSet;
 import java.util.Collections;
 import java.util.BitSet;
 import java.nio.ByteBuffer;
+import java.util.Arrays;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 public class CfDef implements org.apache.thrift.TBase<CfDef, CfDef._Fields>, java.io.Serializable, Cloneable {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("CfDef");
@@ -62,6 +67,7 @@ public class CfDef implements org.apache.thrift.TBase<CfDef, CfDef._Fields>, jav
   private static final org.apache.thrift.protocol.TField BLOOM_FILTER_FP_CHANCE_FIELD_DESC = new org.apache.thrift.protocol.TField("bloom_filter_fp_chance", org.apache.thrift.protocol.TType.DOUBLE, (short)33);
   private static final org.apache.thrift.protocol.TField CACHING_FIELD_DESC = new org.apache.thrift.protocol.TField("caching", org.apache.thrift.protocol.TType.STRING, (short)34);
   private static final org.apache.thrift.protocol.TField DCLOCAL_READ_REPAIR_CHANCE_FIELD_DESC = new org.apache.thrift.protocol.TField("dclocal_read_repair_chance", org.apache.thrift.protocol.TType.DOUBLE, (short)37);
+  private static final org.apache.thrift.protocol.TField POPULATE_IO_CACHE_ON_FLUSH_FIELD_DESC = new org.apache.thrift.protocol.TField("populate_io_cache_on_flush", org.apache.thrift.protocol.TType.BOOL, (short)38);
   private static final org.apache.thrift.protocol.TField ROW_CACHE_SIZE_FIELD_DESC = new org.apache.thrift.protocol.TField("row_cache_size", org.apache.thrift.protocol.TType.DOUBLE, (short)9);
   private static final org.apache.thrift.protocol.TField KEY_CACHE_SIZE_FIELD_DESC = new org.apache.thrift.protocol.TField("key_cache_size", org.apache.thrift.protocol.TType.DOUBLE, (short)11);
   private static final org.apache.thrift.protocol.TField ROW_CACHE_SAVE_PERIOD_IN_SECONDS_FIELD_DESC = new org.apache.thrift.protocol.TField("row_cache_save_period_in_seconds", org.apache.thrift.protocol.TType.I32, (short)19);
@@ -72,7 +78,6 @@ public class CfDef implements org.apache.thrift.TBase<CfDef, CfDef._Fields>, jav
   private static final org.apache.thrift.protocol.TField MERGE_SHARDS_CHANCE_FIELD_DESC = new org.apache.thrift.protocol.TField("merge_shards_chance", org.apache.thrift.protocol.TType.DOUBLE, (short)25);
   private static final org.apache.thrift.protocol.TField ROW_CACHE_PROVIDER_FIELD_DESC = new org.apache.thrift.protocol.TField("row_cache_provider", org.apache.thrift.protocol.TType.STRING, (short)27);
   private static final org.apache.thrift.protocol.TField ROW_CACHE_KEYS_TO_SAVE_FIELD_DESC = new org.apache.thrift.protocol.TField("row_cache_keys_to_save", org.apache.thrift.protocol.TType.I32, (short)31);
-  private static final org.apache.thrift.protocol.TField POPULATE_IO_CACHE_ON_FLUSH_FIELD_DESC = new org.apache.thrift.protocol.TField("populate_io_cache_on_flush", org.apache.thrift.protocol.TType.BOOL, (short)38);
 
   public String keyspace; // required
   public String name; // required
@@ -331,16 +336,16 @@ public class CfDef implements org.apache.thrift.TBase<CfDef, CfDef._Fields>, jav
   private static final int __REPLICATE_ON_WRITE_ISSET_ID = 5;
   private static final int __BLOOM_FILTER_FP_CHANCE_ISSET_ID = 6;
   private static final int __DCLOCAL_READ_REPAIR_CHANCE_ISSET_ID = 7;
-  private static final int __ROW_CACHE_SIZE_ISSET_ID = 8;
-  private static final int __KEY_CACHE_SIZE_ISSET_ID = 9;
-  private static final int __ROW_CACHE_SAVE_PERIOD_IN_SECONDS_ISSET_ID = 10;
-  private static final int __KEY_CACHE_SAVE_PERIOD_IN_SECONDS_ISSET_ID = 11;
-  private static final int __MEMTABLE_FLUSH_AFTER_MINS_ISSET_ID = 12;
-  private static final int __MEMTABLE_THROUGHPUT_IN_MB_ISSET_ID = 13;
-  private static final int __MEMTABLE_OPERATIONS_IN_MILLIONS_ISSET_ID = 14;
-  private static final int __MERGE_SHARDS_CHANCE_ISSET_ID = 15;
-  private static final int __ROW_CACHE_KEYS_TO_SAVE_ISSET_ID = 16;
-  private static final int __POPULATE_IO_CACHE_ISSET_ID = 17;
+  private static final int __POPULATE_IO_CACHE_ON_FLUSH_ISSET_ID = 8;
+  private static final int __ROW_CACHE_SIZE_ISSET_ID = 9;
+  private static final int __KEY_CACHE_SIZE_ISSET_ID = 10;
+  private static final int __ROW_CACHE_SAVE_PERIOD_IN_SECONDS_ISSET_ID = 11;
+  private static final int __KEY_CACHE_SAVE_PERIOD_IN_SECONDS_ISSET_ID = 12;
+  private static final int __MEMTABLE_FLUSH_AFTER_MINS_ISSET_ID = 13;
+  private static final int __MEMTABLE_THROUGHPUT_IN_MB_ISSET_ID = 14;
+  private static final int __MEMTABLE_OPERATIONS_IN_MILLIONS_ISSET_ID = 15;
+  private static final int __MERGE_SHARDS_CHANCE_ISSET_ID = 16;
+  private static final int __ROW_CACHE_KEYS_TO_SAVE_ISSET_ID = 17;
   private BitSet __isset_bit_vector = new BitSet(18);
 
   public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
@@ -375,7 +380,7 @@ public class CfDef implements org.apache.thrift.TBase<CfDef, CfDef._Fields>, jav
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
     tmpMap.put(_Fields.REPLICATE_ON_WRITE, new org.apache.thrift.meta_data.FieldMetaData("replicate_on_write", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL)));
-    tmpMap.put(_Fields.KEY_VALIDATION_CLASS, new org.apache.thrift.meta_data.FieldMetaData("key_validation_class", org.apache.thrift.TFieldRequirementType.OPTIONAL,
+    tmpMap.put(_Fields.KEY_VALIDATION_CLASS, new org.apache.thrift.meta_data.FieldMetaData("key_validation_class", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
     tmpMap.put(_Fields.KEY_ALIAS, new org.apache.thrift.meta_data.FieldMetaData("key_alias", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING        , true)));
@@ -395,9 +400,9 @@ public class CfDef implements org.apache.thrift.TBase<CfDef, CfDef._Fields>, jav
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
     tmpMap.put(_Fields.DCLOCAL_READ_REPAIR_CHANCE, new org.apache.thrift.meta_data.FieldMetaData("dclocal_read_repair_chance", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.DOUBLE)));
-    tmpMap.put(_Fields.POPULATE_IO_CACHE_ON_FLUSH, new org.apache.thrift.meta_data.FieldMetaData("populate_io_cache_on_flush", org.apache.thrift.TFieldRequirementType.OPTIONAL,
+    tmpMap.put(_Fields.POPULATE_IO_CACHE_ON_FLUSH, new org.apache.thrift.meta_data.FieldMetaData("populate_io_cache_on_flush", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL)));
-    tmpMap.put(_Fields.ROW_CACHE_SIZE, new org.apache.thrift.meta_data.FieldMetaData("row_cache_size", org.apache.thrift.TFieldRequirementType.OPTIONAL,
+    tmpMap.put(_Fields.ROW_CACHE_SIZE, new org.apache.thrift.meta_data.FieldMetaData("row_cache_size", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.DOUBLE)));
     tmpMap.put(_Fields.KEY_CACHE_SIZE, new org.apache.thrift.meta_data.FieldMetaData("key_cache_size", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.DOUBLE)));
@@ -429,6 +434,7 @@ public class CfDef implements org.apache.thrift.TBase<CfDef, CfDef._Fields>, jav
     this.caching = "keys_only";
 
     this.dclocal_read_repair_chance = 0;
+
   }
 
   public CfDef(
@@ -485,6 +491,7 @@ public class CfDef implements org.apache.thrift.TBase<CfDef, CfDef._Fields>, jav
     }
     if (other.isSetKey_alias()) {
       this.key_alias = org.apache.thrift.TBaseHelper.copyBinary(other.key_alias);
+;
     }
     if (other.isSetCompaction_strategy()) {
       this.compaction_strategy = other.compaction_strategy;
@@ -575,10 +582,11 @@ public class CfDef implements org.apache.thrift.TBase<CfDef, CfDef._Fields>, jav
     setBloom_filter_fp_chanceIsSet(false);
     this.bloom_filter_fp_chance = 0.0;
     this.caching = "keys_only";
+
     this.dclocal_read_repair_chance = 0;
+
     setPopulate_io_cache_on_flushIsSet(false);
     this.populate_io_cache_on_flush = false;
-
     setRow_cache_sizeIsSet(false);
     this.row_cache_size = 0.0;
     setKey_cache_sizeIsSet(false);
@@ -1154,16 +1162,6 @@ public class CfDef implements org.apache.thrift.TBase<CfDef, CfDef._Fields>, jav
     return this;
   }
 
-  public boolean getPopulate_io_cache_on_flush() {
-    return this.populate_io_cache_on_flush;
-  }
-
-  public CfDef setPopulate_io_cache_on_flush(boolean populate_io_cache_on_flush) {
-    this.populate_io_cache_on_flush = populate_io_cache_on_flush;
-    setPopulate_io_cache_on_flushIsSet(true);
-    return this;
-  }
-
   public void unsetDclocal_read_repair_chance() {
     __isset_bit_vector.clear(__DCLOCAL_READ_REPAIR_CHANCE_ISSET_ID);
   }
@@ -1177,17 +1175,27 @@ public class CfDef implements org.apache.thrift.TBase<CfDef, CfDef._Fields>, jav
     __isset_bit_vector.set(__DCLOCAL_READ_REPAIR_CHANCE_ISSET_ID, value);
   }
 
+  public boolean isPopulate_io_cache_on_flush() {
+    return this.populate_io_cache_on_flush;
+  }
+
+  public CfDef setPopulate_io_cache_on_flush(boolean populate_io_cache_on_flush) {
+    this.populate_io_cache_on_flush = populate_io_cache_on_flush;
+    setPopulate_io_cache_on_flushIsSet(true);
+    return this;
+  }
+
   public void unsetPopulate_io_cache_on_flush() {
-    __isset_bit_vector.clear(__POPULATE_IO_CACHE_ISSET_ID);
+    __isset_bit_vector.clear(__POPULATE_IO_CACHE_ON_FLUSH_ISSET_ID);
   }
 
   /** Returns true if field populate_io_cache_on_flush is set (has been assigned a value) and false otherwise */
   public boolean isSetPopulate_io_cache_on_flush() {
-    return __isset_bit_vector.get(__POPULATE_IO_CACHE_ISSET_ID);
+    return __isset_bit_vector.get(__POPULATE_IO_CACHE_ON_FLUSH_ISSET_ID);
   }
 
   public void setPopulate_io_cache_on_flushIsSet(boolean value) {
-    __isset_bit_vector.set(__POPULATE_IO_CACHE_ISSET_ID, value);
+    __isset_bit_vector.set(__POPULATE_IO_CACHE_ON_FLUSH_ISSET_ID, value);
   }
 
   /**
@@ -1663,7 +1671,7 @@ public class CfDef implements org.apache.thrift.TBase<CfDef, CfDef._Fields>, jav
       if (value == null) {
         unsetPopulate_io_cache_on_flush();
       } else {
-        setPopulate_io_cache_on_flush((Boolean) value);
+        setPopulate_io_cache_on_flush((Boolean)value);
       }
       break;
 
@@ -1819,7 +1827,7 @@ public class CfDef implements org.apache.thrift.TBase<CfDef, CfDef._Fields>, jav
       return Double.valueOf(getDclocal_read_repair_chance());
 
     case POPULATE_IO_CACHE_ON_FLUSH:
-      return Boolean.valueOf(getPopulate_io_cache_on_flush());
+      return Boolean.valueOf(isPopulate_io_cache_on_flush());
 
     case ROW_CACHE_SIZE:
       return Double.valueOf(getRow_cache_size());
@@ -2146,10 +2154,10 @@ public class CfDef implements org.apache.thrift.TBase<CfDef, CfDef._Fields>, jav
     boolean this_present_populate_io_cache_on_flush = true && this.isSetPopulate_io_cache_on_flush();
     boolean that_present_populate_io_cache_on_flush = true && that.isSetPopulate_io_cache_on_flush();
     if (this_present_populate_io_cache_on_flush || that_present_populate_io_cache_on_flush) {
-        if (!(this_present_populate_io_cache_on_flush && that_present_populate_io_cache_on_flush))
-            return false;
-        if (this.populate_io_cache_on_flush != that.populate_io_cache_on_flush)
-            return false;
+      if (!(this_present_populate_io_cache_on_flush && that_present_populate_io_cache_on_flush))
+        return false;
+      if (this.populate_io_cache_on_flush != that.populate_io_cache_on_flush)
+        return false;
     }
 
     boolean this_present_row_cache_size = true && this.isSetRow_cache_size();
@@ -2362,7 +2370,7 @@ public class CfDef implements org.apache.thrift.TBase<CfDef, CfDef._Fields>, jav
     boolean present_populate_io_cache_on_flush = true && (isSetPopulate_io_cache_on_flush());
     builder.append(present_populate_io_cache_on_flush);
     if (present_populate_io_cache_on_flush)
-        builder.append(populate_io_cache_on_flush);
+      builder.append(populate_io_cache_on_flush);
 
     boolean present_row_cache_size = true && (isSetRow_cache_size());
     builder.append(present_row_cache_size);
@@ -2973,10 +2981,10 @@ public class CfDef implements org.apache.thrift.TBase<CfDef, CfDef._Fields>, jav
           if (field.type == org.apache.thrift.protocol.TType.BOOL) {
             this.populate_io_cache_on_flush = iprot.readBool();
             setPopulate_io_cache_on_flushIsSet(true);
-          } else {
+          } else { 
             org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
           }
-          break;  
+          break;
         case 9: // ROW_CACHE_SIZE
           if (field.type == org.apache.thrift.protocol.TType.DOUBLE) {
             this.row_cache_size = iprot.readDouble();
@@ -3479,9 +3487,9 @@ public class CfDef implements org.apache.thrift.TBase<CfDef, CfDef._Fields>, jav
     }
     if (isSetPopulate_io_cache_on_flush()) {
       if (!first) sb.append(", ");
-        sb.append("populate_io_cache_on_flush:");
-        sb.append(this.populate_io_cache_on_flush);
-        first = false;
+      sb.append("populate_io_cache_on_flush:");
+      sb.append(this.populate_io_cache_on_flush);
+      first = false;
     }
     if (isSetRow_cache_size()) {
       if (!first) sb.append(", ");

http://git-wip-us.apache.org/repos/asf/cassandra/blob/86e949f3/interface/thrift/gen-java/org/apache/cassandra/thrift/Constants.java
----------------------------------------------------------------------
diff --git a/interface/thrift/gen-java/org/apache/cassandra/thrift/Constants.java b/interface/thrift/gen-java/org/apache/cassandra/thrift/Constants.java
index 1315a5b..98a7ce9 100644
--- a/interface/thrift/gen-java/org/apache/cassandra/thrift/Constants.java
+++ b/interface/thrift/gen-java/org/apache/cassandra/thrift/Constants.java
@@ -44,6 +44,6 @@ import org.slf4j.LoggerFactory;
 
 public class Constants {
 
-  public static final String VERSION = "19.36.1";
+  public static final String VERSION = "19.36.2";
 
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/86e949f3/src/java/org/apache/cassandra/hadoop/AbstractColumnFamilyInputFormat.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/hadoop/AbstractColumnFamilyInputFormat.java b/src/java/org/apache/cassandra/hadoop/AbstractColumnFamilyInputFormat.java
index 8fd66ab..cca2734 100644
--- a/src/java/org/apache/cassandra/hadoop/AbstractColumnFamilyInputFormat.java
+++ b/src/java/org/apache/cassandra/hadoop/AbstractColumnFamilyInputFormat.java
@@ -19,12 +19,7 @@ package org.apache.cassandra.hadoop;
 
 import java.io.IOException;
 import java.net.InetAddress;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Random;
+import java.util.*;
 import java.util.concurrent.Callable;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Future;
@@ -130,6 +125,7 @@ public abstract class AbstractColumnFamilyInputFormat<K, Y> extends InputFormat<
         partitioner = ConfigHelper.getInputPartitioner(context.getConfiguration());
         logger.debug("partitioner is " + partitioner);
 
+
         // cannonical ranges, split into pieces, fetching the splits in parallel
         ExecutorService executor = new ThreadPoolExecutor(0, 128, 60L, TimeUnit.SECONDS, new LinkedBlockingQueue<Runnable>());
         List<InputSplit> splits = new ArrayList<InputSplit>();
@@ -326,7 +322,7 @@ public abstract class AbstractColumnFamilyInputFormat<K, Y> extends InputFormat<
         List<TokenRange> map;
         try
         {
-            map = client.describe_ring(ConfigHelper.getInputKeyspace(conf));
+            map = client.describe_local_ring(ConfigHelper.getInputKeyspace(conf));
         }
         catch (InvalidRequestException e)
         {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/86e949f3/src/java/org/apache/cassandra/service/StorageService.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/StorageService.java b/src/java/org/apache/cassandra/service/StorageService.java
index c9326ae..b85f478 100644
--- a/src/java/org/apache/cassandra/service/StorageService.java
+++ b/src/java/org/apache/cassandra/service/StorageService.java
@@ -34,6 +34,7 @@ import javax.management.Notification;
 import javax.management.NotificationBroadcasterSupport;
 import javax.management.ObjectName;
 
+import com.google.common.base.Predicate;
 import com.google.common.collect.*;
 
 import com.google.common.util.concurrent.AtomicDouble;
@@ -1080,15 +1081,61 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
 
     public Map<Range<Token>, List<InetAddress>> getRangeToAddressMap(String keyspace)
     {
+        return getRangeToAddressMap(keyspace, tokenMetadata.sortedTokens());
+    }
+
+    public Map<Range<Token>, List<InetAddress>> getRangeToAddressMapInLocalDC(String keyspace)
+    {
+        Predicate<InetAddress> isLocalDC = new Predicate<InetAddress>()
+        {
+            public boolean apply(InetAddress address)
+            {
+                return isLocalDC(address);
+            }
+        };
+
+        Map<Range<Token>, List<InetAddress>> origMap = getRangeToAddressMap(keyspace, getTokensInLocalDC());
+        Map<Range<Token>, List<InetAddress>> filteredMap = Maps.newHashMap();
+        for (Map.Entry<Range<Token>, List<InetAddress>> entry : origMap.entrySet())
+        {
+            List<InetAddress> endpointsInLocalDC = Lists.newArrayList(Collections2.filter(entry.getValue(), isLocalDC));
+            filteredMap.put(entry.getKey(), endpointsInLocalDC);
+        }
+
+        return filteredMap;
+    }
+
+    private List<Token> getTokensInLocalDC()
+    {
+        List<Token> filteredTokens = Lists.newArrayList();
+        for (Token token : tokenMetadata.sortedTokens())
+        {
+            InetAddress endpoint = tokenMetadata.getEndpoint(token);
+            if (isLocalDC(endpoint))
+                filteredTokens.add(token);
+        }
+        return filteredTokens;
+    }
+
+    private boolean isLocalDC(InetAddress targetHost)
+    {
+        String remoteDC = DatabaseDescriptor.getEndpointSnitch().getDatacenter(targetHost);
+        String localDC = DatabaseDescriptor.getEndpointSnitch().getDatacenter(FBUtilities.getBroadcastAddress());
+        return remoteDC.equals(localDC);
+    }
+
+    private Map<Range<Token>, List<InetAddress>> getRangeToAddressMap(String keyspace, List<Token> sortedTokens)
+    {
         // some people just want to get a visual representation of things. Allow null and set it to the first
         // non-system table.
         if (keyspace == null)
             keyspace = Schema.instance.getNonSystemTables().get(0);
 
-        List<Range<Token>> ranges = getAllRanges(tokenMetadata.sortedTokens());
+        List<Range<Token>> ranges = getAllRanges(sortedTokens);
         return constructRangeToEndpointMap(keyspace, ranges);
     }
 
+
     /**
      * The same as {@code describeRing(String)} but converts TokenRange to the String for JMX compatibility
      *
@@ -1126,13 +1173,31 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
      */
     public List<TokenRange> describeRing(String keyspace) throws InvalidRequestException
     {
+        return describeRing(keyspace, false);
+    }
+
+    /**
+     * The same as {@code describeRing(String)} but considers only the part of the ring formed by nodes in the local DC.
+     */
+    public List<TokenRange> describeLocalRing(String keyspace) throws InvalidRequestException
+    {
+        return describeRing(keyspace, true);
+    }
+
+    private List<TokenRange> describeRing(String keyspace, boolean includeOnlyLocalDC) throws InvalidRequestException
+    {
         if (keyspace == null || Table.open(keyspace).getReplicationStrategy() instanceof LocalStrategy)
             throw new InvalidRequestException("There is no ring for the keyspace: " + keyspace);
 
         List<TokenRange> ranges = new ArrayList<TokenRange>();
         Token.TokenFactory tf = getPartitioner().getTokenFactory();
 
-        for (Map.Entry<Range<Token>, List<InetAddress>> entry : getRangeToAddressMap(keyspace).entrySet())
+        Map<Range<Token>, List<InetAddress>> rangeToAddressMap =
+                includeOnlyLocalDC
+                        ? getRangeToAddressMapInLocalDC(keyspace)
+                        : getRangeToAddressMap(keyspace);
+
+        for (Map.Entry<Range<Token>, List<InetAddress>> entry : rangeToAddressMap.entrySet())
         {
             Range range = entry.getKey();
             List<InetAddress> addresses = entry.getValue();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/86e949f3/src/java/org/apache/cassandra/thrift/CassandraServer.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/thrift/CassandraServer.java b/src/java/org/apache/cassandra/thrift/CassandraServer.java
index 883ab5a..6a35285 100644
--- a/src/java/org/apache/cassandra/thrift/CassandraServer.java
+++ b/src/java/org/apache/cassandra/thrift/CassandraServer.java
@@ -1183,6 +1183,19 @@ public class CassandraServer implements Cassandra.Iface
         }
     }
 
+    @Override
+    public List<TokenRange> describe_local_ring(String keyspace) throws InvalidRequestException, TException
+    {
+        try
+        {
+            return StorageService.instance.describeLocalRing(keyspace);
+        }
+        catch (RequestValidationException e)
+        {
+            throw ThriftConversion.toThrift(e);
+        }
+    }
+
     public Map<String, String> describe_token_map() throws InvalidRequestException
     {
         return StorageService.instance.getTokenToEndpointMap();


[04/16] add describe_local_ring for Hadoop's benefit patch by Piotr Kołaczkowski; reviewed by jbellis for CASSANDRA-6268

Posted by jb...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/86e949f3/interface/thrift/gen-java/org/apache/cassandra/thrift/CfDef.java
----------------------------------------------------------------------
diff --git a/interface/thrift/gen-java/org/apache/cassandra/thrift/CfDef.java b/interface/thrift/gen-java/org/apache/cassandra/thrift/CfDef.java
index 60aa07f..059d630 100644
--- a/interface/thrift/gen-java/org/apache/cassandra/thrift/CfDef.java
+++ b/interface/thrift/gen-java/org/apache/cassandra/thrift/CfDef.java
@@ -32,10 +32,15 @@ import java.util.ArrayList;
 import java.util.Map;
 import java.util.HashMap;
 import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
 import java.util.EnumSet;
 import java.util.Collections;
 import java.util.BitSet;
 import java.nio.ByteBuffer;
+import java.util.Arrays;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 public class CfDef implements org.apache.thrift.TBase<CfDef, CfDef._Fields>, java.io.Serializable, Cloneable {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("CfDef");
@@ -62,6 +67,7 @@ public class CfDef implements org.apache.thrift.TBase<CfDef, CfDef._Fields>, jav
   private static final org.apache.thrift.protocol.TField BLOOM_FILTER_FP_CHANCE_FIELD_DESC = new org.apache.thrift.protocol.TField("bloom_filter_fp_chance", org.apache.thrift.protocol.TType.DOUBLE, (short)33);
   private static final org.apache.thrift.protocol.TField CACHING_FIELD_DESC = new org.apache.thrift.protocol.TField("caching", org.apache.thrift.protocol.TType.STRING, (short)34);
   private static final org.apache.thrift.protocol.TField DCLOCAL_READ_REPAIR_CHANCE_FIELD_DESC = new org.apache.thrift.protocol.TField("dclocal_read_repair_chance", org.apache.thrift.protocol.TType.DOUBLE, (short)37);
+  private static final org.apache.thrift.protocol.TField POPULATE_IO_CACHE_ON_FLUSH_FIELD_DESC = new org.apache.thrift.protocol.TField("populate_io_cache_on_flush", org.apache.thrift.protocol.TType.BOOL, (short)38);
   private static final org.apache.thrift.protocol.TField ROW_CACHE_SIZE_FIELD_DESC = new org.apache.thrift.protocol.TField("row_cache_size", org.apache.thrift.protocol.TType.DOUBLE, (short)9);
   private static final org.apache.thrift.protocol.TField KEY_CACHE_SIZE_FIELD_DESC = new org.apache.thrift.protocol.TField("key_cache_size", org.apache.thrift.protocol.TType.DOUBLE, (short)11);
   private static final org.apache.thrift.protocol.TField ROW_CACHE_SAVE_PERIOD_IN_SECONDS_FIELD_DESC = new org.apache.thrift.protocol.TField("row_cache_save_period_in_seconds", org.apache.thrift.protocol.TType.I32, (short)19);
@@ -72,7 +78,6 @@ public class CfDef implements org.apache.thrift.TBase<CfDef, CfDef._Fields>, jav
   private static final org.apache.thrift.protocol.TField MERGE_SHARDS_CHANCE_FIELD_DESC = new org.apache.thrift.protocol.TField("merge_shards_chance", org.apache.thrift.protocol.TType.DOUBLE, (short)25);
   private static final org.apache.thrift.protocol.TField ROW_CACHE_PROVIDER_FIELD_DESC = new org.apache.thrift.protocol.TField("row_cache_provider", org.apache.thrift.protocol.TType.STRING, (short)27);
   private static final org.apache.thrift.protocol.TField ROW_CACHE_KEYS_TO_SAVE_FIELD_DESC = new org.apache.thrift.protocol.TField("row_cache_keys_to_save", org.apache.thrift.protocol.TType.I32, (short)31);
-  private static final org.apache.thrift.protocol.TField POPULATE_IO_CACHE_ON_FLUSH_FIELD_DESC = new org.apache.thrift.protocol.TField("populate_io_cache_on_flush", org.apache.thrift.protocol.TType.BOOL, (short)38);
 
   public String keyspace; // required
   public String name; // required
@@ -331,16 +336,16 @@ public class CfDef implements org.apache.thrift.TBase<CfDef, CfDef._Fields>, jav
   private static final int __REPLICATE_ON_WRITE_ISSET_ID = 5;
   private static final int __BLOOM_FILTER_FP_CHANCE_ISSET_ID = 6;
   private static final int __DCLOCAL_READ_REPAIR_CHANCE_ISSET_ID = 7;
-  private static final int __ROW_CACHE_SIZE_ISSET_ID = 8;
-  private static final int __KEY_CACHE_SIZE_ISSET_ID = 9;
-  private static final int __ROW_CACHE_SAVE_PERIOD_IN_SECONDS_ISSET_ID = 10;
-  private static final int __KEY_CACHE_SAVE_PERIOD_IN_SECONDS_ISSET_ID = 11;
-  private static final int __MEMTABLE_FLUSH_AFTER_MINS_ISSET_ID = 12;
-  private static final int __MEMTABLE_THROUGHPUT_IN_MB_ISSET_ID = 13;
-  private static final int __MEMTABLE_OPERATIONS_IN_MILLIONS_ISSET_ID = 14;
-  private static final int __MERGE_SHARDS_CHANCE_ISSET_ID = 15;
-  private static final int __ROW_CACHE_KEYS_TO_SAVE_ISSET_ID = 16;
-  private static final int __POPULATE_IO_CACHE_ISSET_ID = 17;
+  private static final int __POPULATE_IO_CACHE_ON_FLUSH_ISSET_ID = 8;
+  private static final int __ROW_CACHE_SIZE_ISSET_ID = 9;
+  private static final int __KEY_CACHE_SIZE_ISSET_ID = 10;
+  private static final int __ROW_CACHE_SAVE_PERIOD_IN_SECONDS_ISSET_ID = 11;
+  private static final int __KEY_CACHE_SAVE_PERIOD_IN_SECONDS_ISSET_ID = 12;
+  private static final int __MEMTABLE_FLUSH_AFTER_MINS_ISSET_ID = 13;
+  private static final int __MEMTABLE_THROUGHPUT_IN_MB_ISSET_ID = 14;
+  private static final int __MEMTABLE_OPERATIONS_IN_MILLIONS_ISSET_ID = 15;
+  private static final int __MERGE_SHARDS_CHANCE_ISSET_ID = 16;
+  private static final int __ROW_CACHE_KEYS_TO_SAVE_ISSET_ID = 17;
   private BitSet __isset_bit_vector = new BitSet(18);
 
   public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
@@ -375,7 +380,7 @@ public class CfDef implements org.apache.thrift.TBase<CfDef, CfDef._Fields>, jav
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
     tmpMap.put(_Fields.REPLICATE_ON_WRITE, new org.apache.thrift.meta_data.FieldMetaData("replicate_on_write", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL)));
-    tmpMap.put(_Fields.KEY_VALIDATION_CLASS, new org.apache.thrift.meta_data.FieldMetaData("key_validation_class", org.apache.thrift.TFieldRequirementType.OPTIONAL,
+    tmpMap.put(_Fields.KEY_VALIDATION_CLASS, new org.apache.thrift.meta_data.FieldMetaData("key_validation_class", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
     tmpMap.put(_Fields.KEY_ALIAS, new org.apache.thrift.meta_data.FieldMetaData("key_alias", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING        , true)));
@@ -395,9 +400,9 @@ public class CfDef implements org.apache.thrift.TBase<CfDef, CfDef._Fields>, jav
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
     tmpMap.put(_Fields.DCLOCAL_READ_REPAIR_CHANCE, new org.apache.thrift.meta_data.FieldMetaData("dclocal_read_repair_chance", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.DOUBLE)));
-    tmpMap.put(_Fields.POPULATE_IO_CACHE_ON_FLUSH, new org.apache.thrift.meta_data.FieldMetaData("populate_io_cache_on_flush", org.apache.thrift.TFieldRequirementType.OPTIONAL,
+    tmpMap.put(_Fields.POPULATE_IO_CACHE_ON_FLUSH, new org.apache.thrift.meta_data.FieldMetaData("populate_io_cache_on_flush", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL)));
-    tmpMap.put(_Fields.ROW_CACHE_SIZE, new org.apache.thrift.meta_data.FieldMetaData("row_cache_size", org.apache.thrift.TFieldRequirementType.OPTIONAL,
+    tmpMap.put(_Fields.ROW_CACHE_SIZE, new org.apache.thrift.meta_data.FieldMetaData("row_cache_size", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.DOUBLE)));
     tmpMap.put(_Fields.KEY_CACHE_SIZE, new org.apache.thrift.meta_data.FieldMetaData("key_cache_size", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.DOUBLE)));
@@ -429,6 +434,7 @@ public class CfDef implements org.apache.thrift.TBase<CfDef, CfDef._Fields>, jav
     this.caching = "keys_only";
 
     this.dclocal_read_repair_chance = 0;
+
   }
 
   public CfDef(
@@ -485,6 +491,7 @@ public class CfDef implements org.apache.thrift.TBase<CfDef, CfDef._Fields>, jav
     }
     if (other.isSetKey_alias()) {
       this.key_alias = org.apache.thrift.TBaseHelper.copyBinary(other.key_alias);
+;
     }
     if (other.isSetCompaction_strategy()) {
       this.compaction_strategy = other.compaction_strategy;
@@ -575,10 +582,11 @@ public class CfDef implements org.apache.thrift.TBase<CfDef, CfDef._Fields>, jav
     setBloom_filter_fp_chanceIsSet(false);
     this.bloom_filter_fp_chance = 0.0;
     this.caching = "keys_only";
+
     this.dclocal_read_repair_chance = 0;
+
     setPopulate_io_cache_on_flushIsSet(false);
     this.populate_io_cache_on_flush = false;
-
     setRow_cache_sizeIsSet(false);
     this.row_cache_size = 0.0;
     setKey_cache_sizeIsSet(false);
@@ -1154,16 +1162,6 @@ public class CfDef implements org.apache.thrift.TBase<CfDef, CfDef._Fields>, jav
     return this;
   }
 
-  public boolean getPopulate_io_cache_on_flush() {
-    return this.populate_io_cache_on_flush;
-  }
-
-  public CfDef setPopulate_io_cache_on_flush(boolean populate_io_cache_on_flush) {
-    this.populate_io_cache_on_flush = populate_io_cache_on_flush;
-    setPopulate_io_cache_on_flushIsSet(true);
-    return this;
-  }
-
   public void unsetDclocal_read_repair_chance() {
     __isset_bit_vector.clear(__DCLOCAL_READ_REPAIR_CHANCE_ISSET_ID);
   }
@@ -1177,17 +1175,27 @@ public class CfDef implements org.apache.thrift.TBase<CfDef, CfDef._Fields>, jav
     __isset_bit_vector.set(__DCLOCAL_READ_REPAIR_CHANCE_ISSET_ID, value);
   }
 
+  public boolean isPopulate_io_cache_on_flush() {
+    return this.populate_io_cache_on_flush;
+  }
+
+  public CfDef setPopulate_io_cache_on_flush(boolean populate_io_cache_on_flush) {
+    this.populate_io_cache_on_flush = populate_io_cache_on_flush;
+    setPopulate_io_cache_on_flushIsSet(true);
+    return this;
+  }
+
   public void unsetPopulate_io_cache_on_flush() {
-    __isset_bit_vector.clear(__POPULATE_IO_CACHE_ISSET_ID);
+    __isset_bit_vector.clear(__POPULATE_IO_CACHE_ON_FLUSH_ISSET_ID);
   }
 
   /** Returns true if field populate_io_cache_on_flush is set (has been assigned a value) and false otherwise */
   public boolean isSetPopulate_io_cache_on_flush() {
-    return __isset_bit_vector.get(__POPULATE_IO_CACHE_ISSET_ID);
+    return __isset_bit_vector.get(__POPULATE_IO_CACHE_ON_FLUSH_ISSET_ID);
   }
 
   public void setPopulate_io_cache_on_flushIsSet(boolean value) {
-    __isset_bit_vector.set(__POPULATE_IO_CACHE_ISSET_ID, value);
+    __isset_bit_vector.set(__POPULATE_IO_CACHE_ON_FLUSH_ISSET_ID, value);
   }
 
   /**
@@ -1663,7 +1671,7 @@ public class CfDef implements org.apache.thrift.TBase<CfDef, CfDef._Fields>, jav
       if (value == null) {
         unsetPopulate_io_cache_on_flush();
       } else {
-        setPopulate_io_cache_on_flush((Boolean) value);
+        setPopulate_io_cache_on_flush((Boolean)value);
       }
       break;
 
@@ -1819,7 +1827,7 @@ public class CfDef implements org.apache.thrift.TBase<CfDef, CfDef._Fields>, jav
       return Double.valueOf(getDclocal_read_repair_chance());
 
     case POPULATE_IO_CACHE_ON_FLUSH:
-      return Boolean.valueOf(getPopulate_io_cache_on_flush());
+      return Boolean.valueOf(isPopulate_io_cache_on_flush());
 
     case ROW_CACHE_SIZE:
       return Double.valueOf(getRow_cache_size());
@@ -2146,10 +2154,10 @@ public class CfDef implements org.apache.thrift.TBase<CfDef, CfDef._Fields>, jav
     boolean this_present_populate_io_cache_on_flush = true && this.isSetPopulate_io_cache_on_flush();
     boolean that_present_populate_io_cache_on_flush = true && that.isSetPopulate_io_cache_on_flush();
     if (this_present_populate_io_cache_on_flush || that_present_populate_io_cache_on_flush) {
-        if (!(this_present_populate_io_cache_on_flush && that_present_populate_io_cache_on_flush))
-            return false;
-        if (this.populate_io_cache_on_flush != that.populate_io_cache_on_flush)
-            return false;
+      if (!(this_present_populate_io_cache_on_flush && that_present_populate_io_cache_on_flush))
+        return false;
+      if (this.populate_io_cache_on_flush != that.populate_io_cache_on_flush)
+        return false;
     }
 
     boolean this_present_row_cache_size = true && this.isSetRow_cache_size();
@@ -2362,7 +2370,7 @@ public class CfDef implements org.apache.thrift.TBase<CfDef, CfDef._Fields>, jav
     boolean present_populate_io_cache_on_flush = true && (isSetPopulate_io_cache_on_flush());
     builder.append(present_populate_io_cache_on_flush);
     if (present_populate_io_cache_on_flush)
-        builder.append(populate_io_cache_on_flush);
+      builder.append(populate_io_cache_on_flush);
 
     boolean present_row_cache_size = true && (isSetRow_cache_size());
     builder.append(present_row_cache_size);
@@ -2973,10 +2981,10 @@ public class CfDef implements org.apache.thrift.TBase<CfDef, CfDef._Fields>, jav
           if (field.type == org.apache.thrift.protocol.TType.BOOL) {
             this.populate_io_cache_on_flush = iprot.readBool();
             setPopulate_io_cache_on_flushIsSet(true);
-          } else {
+          } else { 
             org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
           }
-          break;  
+          break;
         case 9: // ROW_CACHE_SIZE
           if (field.type == org.apache.thrift.protocol.TType.DOUBLE) {
             this.row_cache_size = iprot.readDouble();
@@ -3479,9 +3487,9 @@ public class CfDef implements org.apache.thrift.TBase<CfDef, CfDef._Fields>, jav
     }
     if (isSetPopulate_io_cache_on_flush()) {
       if (!first) sb.append(", ");
-        sb.append("populate_io_cache_on_flush:");
-        sb.append(this.populate_io_cache_on_flush);
-        first = false;
+      sb.append("populate_io_cache_on_flush:");
+      sb.append(this.populate_io_cache_on_flush);
+      first = false;
     }
     if (isSetRow_cache_size()) {
       if (!first) sb.append(", ");

http://git-wip-us.apache.org/repos/asf/cassandra/blob/86e949f3/interface/thrift/gen-java/org/apache/cassandra/thrift/Constants.java
----------------------------------------------------------------------
diff --git a/interface/thrift/gen-java/org/apache/cassandra/thrift/Constants.java b/interface/thrift/gen-java/org/apache/cassandra/thrift/Constants.java
index 1315a5b..98a7ce9 100644
--- a/interface/thrift/gen-java/org/apache/cassandra/thrift/Constants.java
+++ b/interface/thrift/gen-java/org/apache/cassandra/thrift/Constants.java
@@ -44,6 +44,6 @@ import org.slf4j.LoggerFactory;
 
 public class Constants {
 
-  public static final String VERSION = "19.36.1";
+  public static final String VERSION = "19.36.2";
 
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/86e949f3/src/java/org/apache/cassandra/hadoop/AbstractColumnFamilyInputFormat.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/hadoop/AbstractColumnFamilyInputFormat.java b/src/java/org/apache/cassandra/hadoop/AbstractColumnFamilyInputFormat.java
index 8fd66ab..cca2734 100644
--- a/src/java/org/apache/cassandra/hadoop/AbstractColumnFamilyInputFormat.java
+++ b/src/java/org/apache/cassandra/hadoop/AbstractColumnFamilyInputFormat.java
@@ -19,12 +19,7 @@ package org.apache.cassandra.hadoop;
 
 import java.io.IOException;
 import java.net.InetAddress;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Random;
+import java.util.*;
 import java.util.concurrent.Callable;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Future;
@@ -130,6 +125,7 @@ public abstract class AbstractColumnFamilyInputFormat<K, Y> extends InputFormat<
         partitioner = ConfigHelper.getInputPartitioner(context.getConfiguration());
         logger.debug("partitioner is " + partitioner);
 
+
         // cannonical ranges, split into pieces, fetching the splits in parallel
         ExecutorService executor = new ThreadPoolExecutor(0, 128, 60L, TimeUnit.SECONDS, new LinkedBlockingQueue<Runnable>());
         List<InputSplit> splits = new ArrayList<InputSplit>();
@@ -326,7 +322,7 @@ public abstract class AbstractColumnFamilyInputFormat<K, Y> extends InputFormat<
         List<TokenRange> map;
         try
         {
-            map = client.describe_ring(ConfigHelper.getInputKeyspace(conf));
+            map = client.describe_local_ring(ConfigHelper.getInputKeyspace(conf));
         }
         catch (InvalidRequestException e)
         {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/86e949f3/src/java/org/apache/cassandra/service/StorageService.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/StorageService.java b/src/java/org/apache/cassandra/service/StorageService.java
index c9326ae..b85f478 100644
--- a/src/java/org/apache/cassandra/service/StorageService.java
+++ b/src/java/org/apache/cassandra/service/StorageService.java
@@ -34,6 +34,7 @@ import javax.management.Notification;
 import javax.management.NotificationBroadcasterSupport;
 import javax.management.ObjectName;
 
+import com.google.common.base.Predicate;
 import com.google.common.collect.*;
 
 import com.google.common.util.concurrent.AtomicDouble;
@@ -1080,15 +1081,61 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
 
     public Map<Range<Token>, List<InetAddress>> getRangeToAddressMap(String keyspace)
     {
+        return getRangeToAddressMap(keyspace, tokenMetadata.sortedTokens());
+    }
+
+    public Map<Range<Token>, List<InetAddress>> getRangeToAddressMapInLocalDC(String keyspace)
+    {
+        Predicate<InetAddress> isLocalDC = new Predicate<InetAddress>()
+        {
+            public boolean apply(InetAddress address)
+            {
+                return isLocalDC(address);
+            }
+        };
+
+        Map<Range<Token>, List<InetAddress>> origMap = getRangeToAddressMap(keyspace, getTokensInLocalDC());
+        Map<Range<Token>, List<InetAddress>> filteredMap = Maps.newHashMap();
+        for (Map.Entry<Range<Token>, List<InetAddress>> entry : origMap.entrySet())
+        {
+            List<InetAddress> endpointsInLocalDC = Lists.newArrayList(Collections2.filter(entry.getValue(), isLocalDC));
+            filteredMap.put(entry.getKey(), endpointsInLocalDC);
+        }
+
+        return filteredMap;
+    }
+
+    private List<Token> getTokensInLocalDC()
+    {
+        List<Token> filteredTokens = Lists.newArrayList();
+        for (Token token : tokenMetadata.sortedTokens())
+        {
+            InetAddress endpoint = tokenMetadata.getEndpoint(token);
+            if (isLocalDC(endpoint))
+                filteredTokens.add(token);
+        }
+        return filteredTokens;
+    }
+
+    private boolean isLocalDC(InetAddress targetHost)
+    {
+        String remoteDC = DatabaseDescriptor.getEndpointSnitch().getDatacenter(targetHost);
+        String localDC = DatabaseDescriptor.getEndpointSnitch().getDatacenter(FBUtilities.getBroadcastAddress());
+        return remoteDC.equals(localDC);
+    }
+
+    private Map<Range<Token>, List<InetAddress>> getRangeToAddressMap(String keyspace, List<Token> sortedTokens)
+    {
         // some people just want to get a visual representation of things. Allow null and set it to the first
         // non-system table.
         if (keyspace == null)
             keyspace = Schema.instance.getNonSystemTables().get(0);
 
-        List<Range<Token>> ranges = getAllRanges(tokenMetadata.sortedTokens());
+        List<Range<Token>> ranges = getAllRanges(sortedTokens);
         return constructRangeToEndpointMap(keyspace, ranges);
     }
 
+
     /**
      * The same as {@code describeRing(String)} but converts TokenRange to the String for JMX compatibility
      *
@@ -1126,13 +1173,31 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
      */
     public List<TokenRange> describeRing(String keyspace) throws InvalidRequestException
     {
+        return describeRing(keyspace, false);
+    }
+
+    /**
+     * The same as {@code describeRing(String)} but considers only the part of the ring formed by nodes in the local DC.
+     */
+    public List<TokenRange> describeLocalRing(String keyspace) throws InvalidRequestException
+    {
+        return describeRing(keyspace, true);
+    }
+
+    private List<TokenRange> describeRing(String keyspace, boolean includeOnlyLocalDC) throws InvalidRequestException
+    {
         if (keyspace == null || Table.open(keyspace).getReplicationStrategy() instanceof LocalStrategy)
             throw new InvalidRequestException("There is no ring for the keyspace: " + keyspace);
 
         List<TokenRange> ranges = new ArrayList<TokenRange>();
         Token.TokenFactory tf = getPartitioner().getTokenFactory();
 
-        for (Map.Entry<Range<Token>, List<InetAddress>> entry : getRangeToAddressMap(keyspace).entrySet())
+        Map<Range<Token>, List<InetAddress>> rangeToAddressMap =
+                includeOnlyLocalDC
+                        ? getRangeToAddressMapInLocalDC(keyspace)
+                        : getRangeToAddressMap(keyspace);
+
+        for (Map.Entry<Range<Token>, List<InetAddress>> entry : rangeToAddressMap.entrySet())
         {
             Range range = entry.getKey();
             List<InetAddress> addresses = entry.getValue();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/86e949f3/src/java/org/apache/cassandra/thrift/CassandraServer.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/thrift/CassandraServer.java b/src/java/org/apache/cassandra/thrift/CassandraServer.java
index 883ab5a..6a35285 100644
--- a/src/java/org/apache/cassandra/thrift/CassandraServer.java
+++ b/src/java/org/apache/cassandra/thrift/CassandraServer.java
@@ -1183,6 +1183,19 @@ public class CassandraServer implements Cassandra.Iface
         }
     }
 
+    @Override
+    public List<TokenRange> describe_local_ring(String keyspace) throws InvalidRequestException, TException
+    {
+        try
+        {
+            return StorageService.instance.describeLocalRing(keyspace);
+        }
+        catch (RequestValidationException e)
+        {
+            throw ThriftConversion.toThrift(e);
+        }
+    }
+
     public Map<String, String> describe_token_map() throws InvalidRequestException
     {
         return StorageService.instance.getTokenToEndpointMap();


[08/16] add describe_local_ring for Hadoop's benefit patch by Piotr Kołaczkowski; reviewed by jbellis for CASSANDRA-6268

Posted by jb...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/86e949f3/interface/thrift/gen-java/org/apache/cassandra/thrift/Cassandra.java
----------------------------------------------------------------------
diff --git a/interface/thrift/gen-java/org/apache/cassandra/thrift/Cassandra.java b/interface/thrift/gen-java/org/apache/cassandra/thrift/Cassandra.java
index 7edbbd0..5c8d286 100644
--- a/interface/thrift/gen-java/org/apache/cassandra/thrift/Cassandra.java
+++ b/interface/thrift/gen-java/org/apache/cassandra/thrift/Cassandra.java
@@ -5,7 +5,7 @@
  */
 package org.apache.cassandra.thrift;
 /*
- *
+ * 
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -13,16 +13,16 @@ package org.apache.cassandra.thrift;
  * to you under the Apache License, Version 2.0 (the
  * "License"); you may not use this file except in compliance
  * with the License.  You may obtain a copy of the License at
- *
+ * 
  *   http://www.apache.org/licenses/LICENSE-2.0
- *
+ * 
  * Unless required by applicable law or agreed to in writing,
  * software distributed under the License is distributed on an
  * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
  * KIND, either express or implied.  See the License for the
  * specific language governing permissions and limitations
  * under the License.
- *
+ * 
  */
 
 
@@ -53,7 +53,7 @@ public class Cassandra {
     /**
      * Get the Column or SuperColumn at the given column_path. If no value is present, NotFoundException is thrown. (This is
      * the only method that can throw an exception under non-failure conditions.)
-     *
+     * 
      * @param key
      * @param column_path
      * @param consistency_level
@@ -63,7 +63,7 @@ public class Cassandra {
     /**
      * Get the group of columns contained by column_parent (either a ColumnFamily name or a ColumnFamily/SuperColumn name
      * pair) specified by the given SlicePredicate. If no matching values are found, an empty list is returned.
-     *
+     * 
      * @param key
      * @param column_parent
      * @param predicate
@@ -74,7 +74,7 @@ public class Cassandra {
     /**
      * returns the number of columns matching <code>predicate</code> for a particular <code>key</code>,
      * <code>ColumnFamily</code> and optionally <code>SuperColumn</code>.
-     *
+     * 
      * @param key
      * @param column_parent
      * @param predicate
@@ -84,7 +84,7 @@ public class Cassandra {
 
     /**
      * Performs a get_slice for column_parent and predicate for the given keys in parallel.
-     *
+     * 
      * @param keys
      * @param column_parent
      * @param predicate
@@ -94,7 +94,7 @@ public class Cassandra {
 
     /**
      * Perform a get_count in parallel on the given list<binary> keys. The return value maps keys to the count found.
-     *
+     * 
      * @param keys
      * @param column_parent
      * @param predicate
@@ -104,7 +104,7 @@ public class Cassandra {
 
     /**
      * returns a subset of columns for a contiguous range of keys.
-     *
+     * 
      * @param column_parent
      * @param predicate
      * @param range
@@ -114,7 +114,7 @@ public class Cassandra {
 
     /**
      * returns a range of columns, wrapping to the next rows if necessary to collect max_results.
-     *
+     * 
      * @param column_family
      * @param range
      * @param start_column
@@ -124,8 +124,8 @@ public class Cassandra {
 
     /**
      * Returns the subset of columns specified in SlicePredicate for the rows matching the IndexClause
-     * @deprecated; use get_range_slices instead with range.row_filter specified
-     *
+     * @deprecated use get_range_slices instead with range.row_filter specified
+     * 
      * @param column_parent
      * @param index_clause
      * @param column_predicate
@@ -135,7 +135,7 @@ public class Cassandra {
 
     /**
      * Insert a Column at the given column_parent.column_family and optional column_parent.super_column.
-     *
+     * 
      * @param key
      * @param column_parent
      * @param column
@@ -145,7 +145,7 @@ public class Cassandra {
 
     /**
      * Increment or decrement a counter.
-     *
+     * 
      * @param key
      * @param column_parent
      * @param column
@@ -157,7 +157,7 @@ public class Cassandra {
      * Remove data from the row specified by key at the granularity specified by column_path, and the given timestamp. Note
      * that all the values in column_path besides column_path.column_family are truly optional: you can remove the entire
      * row by just specifying the ColumnFamily, or you can remove a SuperColumn or a single Column by specifying those levels too.
-     *
+     * 
      * @param key
      * @param column_path
      * @param timestamp
@@ -169,7 +169,7 @@ public class Cassandra {
      * Remove a counter at the specified location.
      * Note that counters have limited support for deletes: if you remove a counter, you must wait to issue any following update
      * until the delete has reached all the nodes and all of them have been fully compacted.
-     *
+     * 
      * @param key
      * @param path
      * @param consistency_level
@@ -178,10 +178,10 @@ public class Cassandra {
 
     /**
      *   Mutate many columns or super columns for many row keys. See also: Mutation.
-     *
+     * 
      *   mutation_map maps key to column family to a list of Mutation objects to take place at that scope.
      * *
-     *
+     * 
      * @param mutation_map
      * @param consistency_level
      */
@@ -189,10 +189,10 @@ public class Cassandra {
 
     /**
      *   Atomically mutate many columns or super columns for many row keys. See also: Mutation.
-     *
+     * 
      *   mutation_map maps key to column family to a list of Mutation objects to take place at that scope.
      * *
-     *
+     * 
      * @param mutation_map
      * @param consistency_level
      */
@@ -205,7 +205,7 @@ public class Cassandra {
      * only marks the data as deleted.
      * The operation succeeds only if all hosts in the cluster at available and will throw an UnavailableException if
      * some hosts are down.
-     *
+     * 
      * @param cfname
      */
     public void truncate(String cfname) throws InvalidRequestException, UnavailableException, TimedOutException, org.apache.thrift.TException;
@@ -238,15 +238,22 @@ public class Cassandra {
      * to list of endpoints, because you can't use Thrift structs as
      * map keys:
      * https://issues.apache.org/jira/browse/THRIFT-162
-     *
+     * 
      * for the same reason, we can't return a set here, even though
      * order is neither important nor predictable.
-     *
+     * 
      * @param keyspace
      */
     public List<TokenRange> describe_ring(String keyspace) throws InvalidRequestException, org.apache.thrift.TException;
 
     /**
+     * same as describe_ring, but considers only nodes in the local DC
+     * 
+     * @param keyspace
+     */
+    public List<TokenRange> describe_local_ring(String keyspace) throws InvalidRequestException, org.apache.thrift.TException;
+
+    /**
      * get the mapping between token->node ip
      * without taking replication into consideration
      * https://issues.apache.org/jira/browse/CASSANDRA-4092
@@ -265,7 +272,7 @@ public class Cassandra {
 
     /**
      * describe specified keyspace
-     *
+     * 
      * @param keyspace
      */
     public KsDef describe_keyspace(String keyspace) throws NotFoundException, InvalidRequestException, org.apache.thrift.TException;
@@ -273,10 +280,10 @@ public class Cassandra {
     /**
      * experimental API for hadoop/parallel query support.
      * may change violently and without warning.
-     *
+     * 
      * returns list of token strings such that first subrange is (list[0], list[1]],
      * next is (list[1], list[2]], etc.
-     *
+     * 
      * @param cfName
      * @param start_token
      * @param end_token
@@ -294,42 +301,42 @@ public class Cassandra {
 
     /**
      * adds a column family. returns the new schema id.
-     *
+     * 
      * @param cf_def
      */
     public String system_add_column_family(CfDef cf_def) throws InvalidRequestException, SchemaDisagreementException, org.apache.thrift.TException;
 
     /**
      * drops a column family. returns the new schema id.
-     *
+     * 
      * @param column_family
      */
     public String system_drop_column_family(String column_family) throws InvalidRequestException, SchemaDisagreementException, org.apache.thrift.TException;
 
     /**
      * adds a keyspace and any column families that are part of it. returns the new schema id.
-     *
+     * 
      * @param ks_def
      */
     public String system_add_keyspace(KsDef ks_def) throws InvalidRequestException, SchemaDisagreementException, org.apache.thrift.TException;
 
     /**
      * drops a keyspace and any column families that are part of it. returns the new schema id.
-     *
+     * 
      * @param keyspace
      */
     public String system_drop_keyspace(String keyspace) throws InvalidRequestException, SchemaDisagreementException, org.apache.thrift.TException;
 
     /**
      * updates properties of a keyspace. returns the new schema id.
-     *
+     * 
      * @param ks_def
      */
     public String system_update_keyspace(KsDef ks_def) throws InvalidRequestException, SchemaDisagreementException, org.apache.thrift.TException;
 
     /**
      * updates properties of a column family. returns the new schema id.
-     *
+     * 
      * @param cf_def
      */
     public String system_update_column_family(CfDef cf_def) throws InvalidRequestException, SchemaDisagreementException, org.apache.thrift.TException;
@@ -337,7 +344,7 @@ public class Cassandra {
     /**
      * Executes a CQL (Cassandra Query Language) statement and returns a
      * CqlResult containing the results.
-     *
+     * 
      * @param query
      * @param compression
      */
@@ -350,7 +357,7 @@ public class Cassandra {
      * - the type of CQL statement
      * - an id token of the compiled CQL stored on the server side.
      * - a count of the discovered bound markers in the statement
-     *
+     * 
      * @param query
      * @param compression
      */
@@ -361,7 +368,7 @@ public class Cassandra {
     /**
      * Executes a prepared CQL (Cassandra Query Language) statement by passing an id token and  a list of variables
      * to bind and returns a CqlResult containing the results.
-     *
+     * 
      * @param itemId
      * @param values
      */
@@ -371,7 +378,7 @@ public class Cassandra {
 
     /**
      * @deprecated This is now a no-op. Please use the CQL3 specific methods instead.
-     *
+     * 
      * @param version
      */
     public void set_cql_version(String version) throws InvalidRequestException, org.apache.thrift.TException;
@@ -424,6 +431,8 @@ public class Cassandra {
 
     public void describe_ring(String keyspace, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.describe_ring_call> resultHandler) throws org.apache.thrift.TException;
 
+    public void describe_local_ring(String keyspace, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.describe_local_ring_call> resultHandler) throws org.apache.thrift.TException;
+
     public void describe_token_map(org.apache.thrift.async.AsyncMethodCallback<AsyncClient.describe_token_map_call> resultHandler) throws org.apache.thrift.TException;
 
     public void describe_partitioner(org.apache.thrift.async.AsyncMethodCallback<AsyncClient.describe_partitioner_call> resultHandler) throws org.apache.thrift.TException;
@@ -1153,6 +1162,32 @@ public class Cassandra {
       throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "describe_ring failed: unknown result");
     }
 
+    public List<TokenRange> describe_local_ring(String keyspace) throws InvalidRequestException, org.apache.thrift.TException
+    {
+      send_describe_local_ring(keyspace);
+      return recv_describe_local_ring();
+    }
+
+    public void send_describe_local_ring(String keyspace) throws org.apache.thrift.TException
+    {
+      describe_local_ring_args args = new describe_local_ring_args();
+      args.setKeyspace(keyspace);
+      sendBase("describe_local_ring", args);
+    }
+
+    public List<TokenRange> recv_describe_local_ring() throws InvalidRequestException, org.apache.thrift.TException
+    {
+      describe_local_ring_result result = new describe_local_ring_result();
+      receiveBase(result, "describe_local_ring");
+      if (result.isSetSuccess()) {
+        return result.success;
+      }
+      if (result.ire != null) {
+        throw result.ire;
+      }
+      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "describe_local_ring failed: unknown result");
+    }
+
     public Map<String,String> describe_token_map() throws InvalidRequestException, org.apache.thrift.TException
     {
       send_describe_token_map();
@@ -2546,6 +2581,38 @@ public class Cassandra {
       }
     }
 
+    public void describe_local_ring(String keyspace, org.apache.thrift.async.AsyncMethodCallback<describe_local_ring_call> resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      describe_local_ring_call method_call = new describe_local_ring_call(keyspace, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    public static class describe_local_ring_call extends org.apache.thrift.async.TAsyncMethodCall {
+      private String keyspace;
+      public describe_local_ring_call(String keyspace, org.apache.thrift.async.AsyncMethodCallback<describe_local_ring_call> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+        super(client, protocolFactory, transport, resultHandler, false);
+        this.keyspace = keyspace;
+      }
+
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("describe_local_ring", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        describe_local_ring_args args = new describe_local_ring_args();
+        args.setKeyspace(keyspace);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      public List<TokenRange> getResult() throws InvalidRequestException, org.apache.thrift.TException {
+        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+          throw new IllegalStateException("Method call not finished!");
+        }
+        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+        return (new Client(prot)).recv_describe_local_ring();
+      }
+    }
+
     public void describe_token_map(org.apache.thrift.async.AsyncMethodCallback<describe_token_map_call> resultHandler) throws org.apache.thrift.TException {
       checkReady();
       describe_token_map_call method_call = new describe_token_map_call(resultHandler, this, ___protocolFactory, ___transport);
@@ -3251,6 +3318,7 @@ public class Cassandra {
       processMap.put("describe_cluster_name", new describe_cluster_name());
       processMap.put("describe_version", new describe_version());
       processMap.put("describe_ring", new describe_ring());
+      processMap.put("describe_local_ring", new describe_local_ring());
       processMap.put("describe_token_map", new describe_token_map());
       processMap.put("describe_partitioner", new describe_partitioner());
       processMap.put("describe_snitch", new describe_snitch());
@@ -3771,6 +3839,26 @@ public class Cassandra {
       }
     }
 
+    private static class describe_local_ring<I extends Iface> extends org.apache.thrift.ProcessFunction<I, describe_local_ring_args> {
+      public describe_local_ring() {
+        super("describe_local_ring");
+      }
+
+      protected describe_local_ring_args getEmptyArgsInstance() {
+        return new describe_local_ring_args();
+      }
+
+      protected describe_local_ring_result getResult(I iface, describe_local_ring_args args) throws org.apache.thrift.TException {
+        describe_local_ring_result result = new describe_local_ring_result();
+        try {
+          result.success = iface.describe_local_ring(args.keyspace);
+        } catch (InvalidRequestException ire) {
+          result.ire = ire;
+        }
+        return result;
+      }
+    }
+
     private static class describe_token_map<I extends Iface> extends org.apache.thrift.ProcessFunction<I, describe_token_map_args> {
       public describe_token_map() {
         super("describe_token_map");
@@ -4269,7 +4357,7 @@ public class Cassandra {
     public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
     static {
       Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
-      tmpMap.put(_Fields.AUTH_REQUEST, new org.apache.thrift.meta_data.FieldMetaData("auth_request", org.apache.thrift.TFieldRequirementType.REQUIRED,
+      tmpMap.put(_Fields.AUTH_REQUEST, new org.apache.thrift.meta_data.FieldMetaData("auth_request", org.apache.thrift.TFieldRequirementType.REQUIRED, 
           new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, AuthenticationRequest.class)));
       metaDataMap = Collections.unmodifiableMap(tmpMap);
       org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(login_args.class, metaDataMap);
@@ -4430,7 +4518,7 @@ public class Cassandra {
       while (true)
       {
         field = iprot.readFieldBegin();
-        if (field.type == org.apache.thrift.protocol.TType.STOP) {
+        if (field.type == org.apache.thrift.protocol.TType.STOP) { 
           break;
         }
         switch (field.id) {
@@ -4438,7 +4526,7 @@ public class Cassandra {
             if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
               this.auth_request = new AuthenticationRequest();
               this.auth_request.read(iprot);
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -4582,9 +4670,9 @@ public class Cassandra {
     public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
     static {
       Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
-      tmpMap.put(_Fields.AUTHNX, new org.apache.thrift.meta_data.FieldMetaData("authnx", org.apache.thrift.TFieldRequirementType.DEFAULT,
+      tmpMap.put(_Fields.AUTHNX, new org.apache.thrift.meta_data.FieldMetaData("authnx", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT)));
-      tmpMap.put(_Fields.AUTHZX, new org.apache.thrift.meta_data.FieldMetaData("authzx", org.apache.thrift.TFieldRequirementType.DEFAULT,
+      tmpMap.put(_Fields.AUTHZX, new org.apache.thrift.meta_data.FieldMetaData("authzx", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT)));
       metaDataMap = Collections.unmodifiableMap(tmpMap);
       org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(login_result.class, metaDataMap);
@@ -4812,7 +4900,7 @@ public class Cassandra {
       while (true)
       {
         field = iprot.readFieldBegin();
-        if (field.type == org.apache.thrift.protocol.TType.STOP) {
+        if (field.type == org.apache.thrift.protocol.TType.STOP) { 
           break;
         }
         switch (field.id) {
@@ -4820,7 +4908,7 @@ public class Cassandra {
             if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
               this.authnx = new AuthenticationException();
               this.authnx.read(iprot);
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -4828,7 +4916,7 @@ public class Cassandra {
             if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
               this.authzx = new AuthorizationException();
               this.authzx.read(iprot);
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -4975,7 +5063,7 @@ public class Cassandra {
     public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
     static {
       Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
-      tmpMap.put(_Fields.KEYSPACE, new org.apache.thrift.meta_data.FieldMetaData("keyspace", org.apache.thrift.TFieldRequirementType.REQUIRED,
+      tmpMap.put(_Fields.KEYSPACE, new org.apache.thrift.meta_data.FieldMetaData("keyspace", org.apache.thrift.TFieldRequirementType.REQUIRED, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
       metaDataMap = Collections.unmodifiableMap(tmpMap);
       org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(set_keyspace_args.class, metaDataMap);
@@ -5136,14 +5224,14 @@ public class Cassandra {
       while (true)
       {
         field = iprot.readFieldBegin();
-        if (field.type == org.apache.thrift.protocol.TType.STOP) {
+        if (field.type == org.apache.thrift.protocol.TType.STOP) { 
           break;
         }
         switch (field.id) {
           case 1: // KEYSPACE
             if (field.type == org.apache.thrift.protocol.TType.STRING) {
               this.keyspace = iprot.readString();
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -5282,7 +5370,7 @@ public class Cassandra {
     public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
     static {
       Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
-      tmpMap.put(_Fields.IRE, new org.apache.thrift.meta_data.FieldMetaData("ire", org.apache.thrift.TFieldRequirementType.DEFAULT,
+      tmpMap.put(_Fields.IRE, new org.apache.thrift.meta_data.FieldMetaData("ire", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT)));
       metaDataMap = Collections.unmodifiableMap(tmpMap);
       org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(set_keyspace_result.class, metaDataMap);
@@ -5443,7 +5531,7 @@ public class Cassandra {
       while (true)
       {
         field = iprot.readFieldBegin();
-        if (field.type == org.apache.thrift.protocol.TType.STOP) {
+        if (field.type == org.apache.thrift.protocol.TType.STOP) { 
           break;
         }
         switch (field.id) {
@@ -5451,7 +5539,7 @@ public class Cassandra {
             if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
               this.ire = new InvalidRequestException();
               this.ire.read(iprot);
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -5526,7 +5614,7 @@ public class Cassandra {
     public ByteBuffer key; // required
     public ColumnPath column_path; // required
     /**
-     *
+     * 
      * @see ConsistencyLevel
      */
     public ConsistencyLevel consistency_level; // required
@@ -5536,7 +5624,7 @@ public class Cassandra {
       KEY((short)1, "key"),
       COLUMN_PATH((short)2, "column_path"),
       /**
-       *
+       * 
        * @see ConsistencyLevel
        */
       CONSISTENCY_LEVEL((short)3, "consistency_level");
@@ -5604,11 +5692,11 @@ public class Cassandra {
     public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
     static {
       Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
-      tmpMap.put(_Fields.KEY, new org.apache.thrift.meta_data.FieldMetaData("key", org.apache.thrift.TFieldRequirementType.REQUIRED,
+      tmpMap.put(_Fields.KEY, new org.apache.thrift.meta_data.FieldMetaData("key", org.apache.thrift.TFieldRequirementType.REQUIRED, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING          , true)));
-      tmpMap.put(_Fields.COLUMN_PATH, new org.apache.thrift.meta_data.FieldMetaData("column_path", org.apache.thrift.TFieldRequirementType.REQUIRED,
+      tmpMap.put(_Fields.COLUMN_PATH, new org.apache.thrift.meta_data.FieldMetaData("column_path", org.apache.thrift.TFieldRequirementType.REQUIRED, 
           new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ColumnPath.class)));
-      tmpMap.put(_Fields.CONSISTENCY_LEVEL, new org.apache.thrift.meta_data.FieldMetaData("consistency_level", org.apache.thrift.TFieldRequirementType.REQUIRED,
+      tmpMap.put(_Fields.CONSISTENCY_LEVEL, new org.apache.thrift.meta_data.FieldMetaData("consistency_level", org.apache.thrift.TFieldRequirementType.REQUIRED, 
           new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, ConsistencyLevel.class)));
       metaDataMap = Collections.unmodifiableMap(tmpMap);
       org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(get_args.class, metaDataMap);
@@ -5717,7 +5805,7 @@ public class Cassandra {
     }
 
     /**
-     *
+     * 
      * @see ConsistencyLevel
      */
     public ConsistencyLevel getConsistency_level() {
@@ -5725,7 +5813,7 @@ public class Cassandra {
     }
 
     /**
-     *
+     * 
      * @see ConsistencyLevel
      */
     public get_args setConsistency_level(ConsistencyLevel consistency_level) {
@@ -5925,14 +6013,14 @@ public class Cassandra {
       while (true)
       {
         field = iprot.readFieldBegin();
-        if (field.type == org.apache.thrift.protocol.TType.STOP) {
+        if (field.type == org.apache.thrift.protocol.TType.STOP) { 
           break;
         }
         switch (field.id) {
           case 1: // KEY
             if (field.type == org.apache.thrift.protocol.TType.STRING) {
               this.key = iprot.readBinary();
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -5940,14 +6028,14 @@ public class Cassandra {
             if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
               this.column_path = new ColumnPath();
               this.column_path.read(iprot);
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
           case 3: // CONSISTENCY_LEVEL
             if (field.type == org.apache.thrift.protocol.TType.I32) {
               this.consistency_level = ConsistencyLevel.findByValue(iprot.readI32());
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -6138,15 +6226,15 @@ public class Cassandra {
     public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
     static {
       Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
-      tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT,
+      tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ColumnOrSuperColumn.class)));
-      tmpMap.put(_Fields.IRE, new org.apache.thrift.meta_data.FieldMetaData("ire", org.apache.thrift.TFieldRequirementType.DEFAULT,
+      tmpMap.put(_Fields.IRE, new org.apache.thrift.meta_data.FieldMetaData("ire", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT)));
-      tmpMap.put(_Fields.NFE, new org.apache.thrift.meta_data.FieldMetaData("nfe", org.apache.thrift.TFieldRequirementType.DEFAULT,
+      tmpMap.put(_Fields.NFE, new org.apache.thrift.meta_data.FieldMetaData("nfe", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT)));
-      tmpMap.put(_Fields.UE, new org.apache.thrift.meta_data.FieldMetaData("ue", org.apache.thrift.TFieldRequirementType.DEFAULT,
+      tmpMap.put(_Fields.UE, new org.apache.thrift.meta_data.FieldMetaData("ue", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT)));
-      tmpMap.put(_Fields.TE, new org.apache.thrift.meta_data.FieldMetaData("te", org.apache.thrift.TFieldRequirementType.DEFAULT,
+      tmpMap.put(_Fields.TE, new org.apache.thrift.meta_data.FieldMetaData("te", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT)));
       metaDataMap = Collections.unmodifiableMap(tmpMap);
       org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(get_result.class, metaDataMap);
@@ -6575,7 +6663,7 @@ public class Cassandra {
       while (true)
       {
         field = iprot.readFieldBegin();
-        if (field.type == org.apache.thrift.protocol.TType.STOP) {
+        if (field.type == org.apache.thrift.protocol.TType.STOP) { 
           break;
         }
         switch (field.id) {
@@ -6583,7 +6671,7 @@ public class Cassandra {
             if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
               this.success = new ColumnOrSuperColumn();
               this.success.read(iprot);
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -6591,7 +6679,7 @@ public class Cassandra {
             if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
               this.ire = new InvalidRequestException();
               this.ire.read(iprot);
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -6599,7 +6687,7 @@ public class Cassandra {
             if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
               this.nfe = new NotFoundException();
               this.nfe.read(iprot);
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -6607,7 +6695,7 @@ public class Cassandra {
             if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
               this.ue = new UnavailableException();
               this.ue.read(iprot);
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -6615,7 +6703,7 @@ public class Cassandra {
             if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
               this.te = new TimedOutException();
               this.te.read(iprot);
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -6740,7 +6828,7 @@ public class Cassandra {
     public ColumnParent column_parent; // required
     public SlicePredicate predicate; // required
     /**
-     *
+     * 
      * @see ConsistencyLevel
      */
     public ConsistencyLevel consistency_level; // required
@@ -6751,7 +6839,7 @@ public class Cassandra {
       COLUMN_PARENT((short)2, "column_parent"),
       PREDICATE((short)3, "predicate"),
       /**
-       *
+       * 
        * @see ConsistencyLevel
        */
       CONSISTENCY_LEVEL((short)4, "consistency_level");
@@ -6821,13 +6909,13 @@ public class Cassandra {
     public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
     static {
       Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
-      tmpMap.put(_Fields.KEY, new org.apache.thrift.meta_data.FieldMetaData("key", org.apache.thrift.TFieldRequirementType.REQUIRED,
+      tmpMap.put(_Fields.KEY, new org.apache.thrift.meta_data.FieldMetaData("key", org.apache.thrift.TFieldRequirementType.REQUIRED, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING          , true)));
-      tmpMap.put(_Fields.COLUMN_PARENT, new org.apache.thrift.meta_data.FieldMetaData("column_parent", org.apache.thrift.TFieldRequirementType.REQUIRED,
+      tmpMap.put(_Fields.COLUMN_PARENT, new org.apache.thrift.meta_data.FieldMetaData("column_parent", org.apache.thrift.TFieldRequirementType.REQUIRED, 
           new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ColumnParent.class)));
-      tmpMap.put(_Fields.PREDICATE, new org.apache.thrift.meta_data.FieldMetaData("predicate", org.apache.thrift.TFieldRequirementType.REQUIRED,
+      tmpMap.put(_Fields.PREDICATE, new org.apache.thrift.meta_data.FieldMetaData("predicate", org.apache.thrift.TFieldRequirementType.REQUIRED, 
           new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, SlicePredicate.class)));
-      tmpMap.put(_Fields.CONSISTENCY_LEVEL, new org.apache.thrift.meta_data.FieldMetaData("consistency_level", org.apache.thrift.TFieldRequirementType.REQUIRED,
+      tmpMap.put(_Fields.CONSISTENCY_LEVEL, new org.apache.thrift.meta_data.FieldMetaData("consistency_level", org.apache.thrift.TFieldRequirementType.REQUIRED, 
           new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, ConsistencyLevel.class)));
       metaDataMap = Collections.unmodifiableMap(tmpMap);
       org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(get_slice_args.class, metaDataMap);
@@ -6966,7 +7054,7 @@ public class Cassandra {
     }
 
     /**
-     *
+     * 
      * @see ConsistencyLevel
      */
     public ConsistencyLevel getConsistency_level() {
@@ -6974,7 +7062,7 @@ public class Cassandra {
     }
 
     /**
-     *
+     * 
      * @see ConsistencyLevel
      */
     public get_slice_args setConsistency_level(ConsistencyLevel consistency_level) {
@@ -7211,14 +7299,14 @@ public class Cassandra {
       while (true)
       {
         field = iprot.readFieldBegin();
-        if (field.type == org.apache.thrift.protocol.TType.STOP) {
+        if (field.type == org.apache.thrift.protocol.TType.STOP) { 
           break;
         }
         switch (field.id) {
           case 1: // KEY
             if (field.type == org.apache.thrift.protocol.TType.STRING) {
               this.key = iprot.readBinary();
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -7226,7 +7314,7 @@ public class Cassandra {
             if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
               this.column_parent = new ColumnParent();
               this.column_parent.read(iprot);
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -7234,14 +7322,14 @@ public class Cassandra {
             if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
               this.predicate = new SlicePredicate();
               this.predicate.read(iprot);
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
           case 4: // CONSISTENCY_LEVEL
             if (field.type == org.apache.thrift.protocol.TType.I32) {
               this.consistency_level = ConsistencyLevel.findByValue(iprot.readI32());
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -7443,14 +7531,14 @@ public class Cassandra {
     public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
     static {
       Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
-      tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT,
-          new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST,
+      tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
               new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ColumnOrSuperColumn.class))));
-      tmpMap.put(_Fields.IRE, new org.apache.thrift.meta_data.FieldMetaData("ire", org.apache.thrift.TFieldRequirementType.DEFAULT,
+      tmpMap.put(_Fields.IRE, new org.apache.thrift.meta_data.FieldMetaData("ire", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT)));
-      tmpMap.put(_Fields.UE, new org.apache.thrift.meta_data.FieldMetaData("ue", org.apache.thrift.TFieldRequirementType.DEFAULT,
+      tmpMap.put(_Fields.UE, new org.apache.thrift.meta_data.FieldMetaData("ue", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT)));
-      tmpMap.put(_Fields.TE, new org.apache.thrift.meta_data.FieldMetaData("te", org.apache.thrift.TFieldRequirementType.DEFAULT,
+      tmpMap.put(_Fields.TE, new org.apache.thrift.meta_data.FieldMetaData("te", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT)));
       metaDataMap = Collections.unmodifiableMap(tmpMap);
       org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(get_slice_result.class, metaDataMap);
@@ -7831,7 +7919,7 @@ public class Cassandra {
       while (true)
       {
         field = iprot.readFieldBegin();
-        if (field.type == org.apache.thrift.protocol.TType.STOP) {
+        if (field.type == org.apache.thrift.protocol.TType.STOP) { 
           break;
         }
         switch (field.id) {
@@ -7849,7 +7937,7 @@ public class Cassandra {
                 }
                 iprot.readListEnd();
               }
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -7857,7 +7945,7 @@ public class Cassandra {
             if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
               this.ire = new InvalidRequestException();
               this.ire.read(iprot);
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -7865,7 +7953,7 @@ public class Cassandra {
             if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
               this.ue = new UnavailableException();
               this.ue.read(iprot);
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -7873,7 +7961,7 @@ public class Cassandra {
             if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
               this.te = new TimedOutException();
               this.te.read(iprot);
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -7993,7 +8081,7 @@ public class Cassandra {
     public ColumnParent column_parent; // required
     public SlicePredicate predicate; // required
     /**
-     *
+     * 
      * @see ConsistencyLevel
      */
     public ConsistencyLevel consistency_level; // required
@@ -8004,7 +8092,7 @@ public class Cassandra {
       COLUMN_PARENT((short)2, "column_parent"),
       PREDICATE((short)3, "predicate"),
       /**
-       *
+       * 
        * @see ConsistencyLevel
        */
       CONSISTENCY_LEVEL((short)4, "consistency_level");
@@ -8074,13 +8162,13 @@ public class Cassandra {
     public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
     static {
       Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
-      tmpMap.put(_Fields.KEY, new org.apache.thrift.meta_data.FieldMetaData("key", org.apache.thrift.TFieldRequirementType.REQUIRED,
+      tmpMap.put(_Fields.KEY, new org.apache.thrift.meta_data.FieldMetaData("key", org.apache.thrift.TFieldRequirementType.REQUIRED, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING          , true)));
-      tmpMap.put(_Fields.COLUMN_PARENT, new org.apache.thrift.meta_data.FieldMetaData("column_parent", org.apache.thrift.TFieldRequirementType.REQUIRED,
+      tmpMap.put(_Fields.COLUMN_PARENT, new org.apache.thrift.meta_data.FieldMetaData("column_parent", org.apache.thrift.TFieldRequirementType.REQUIRED, 
           new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ColumnParent.class)));
-      tmpMap.put(_Fields.PREDICATE, new org.apache.thrift.meta_data.FieldMetaData("predicate", org.apache.thrift.TFieldRequirementType.REQUIRED,
+      tmpMap.put(_Fields.PREDICATE, new org.apache.thrift.meta_data.FieldMetaData("predicate", org.apache.thrift.TFieldRequirementType.REQUIRED, 
           new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, SlicePredicate.class)));
-      tmpMap.put(_Fields.CONSISTENCY_LEVEL, new org.apache.thrift.meta_data.FieldMetaData("consistency_level", org.apache.thrift.TFieldRequirementType.REQUIRED,
+      tmpMap.put(_Fields.CONSISTENCY_LEVEL, new org.apache.thrift.meta_data.FieldMetaData("consistency_level", org.apache.thrift.TFieldRequirementType.REQUIRED, 
           new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, ConsistencyLevel.class)));
       metaDataMap = Collections.unmodifiableMap(tmpMap);
       org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(get_count_args.class, metaDataMap);
@@ -8219,7 +8307,7 @@ public class Cassandra {
     }
 
     /**
-     *
+     * 
      * @see ConsistencyLevel
      */
     public ConsistencyLevel getConsistency_level() {
@@ -8227,7 +8315,7 @@ public class Cassandra {
     }
 
     /**
-     *
+     * 
      * @see ConsistencyLevel
      */
     public get_count_args setConsistency_level(ConsistencyLevel consistency_level) {
@@ -8464,14 +8552,14 @@ public class Cassandra {
       while (true)
       {
         field = iprot.readFieldBegin();
-        if (field.type == org.apache.thrift.protocol.TType.STOP) {
+        if (field.type == org.apache.thrift.protocol.TType.STOP) { 
           break;
         }
         switch (field.id) {
           case 1: // KEY
             if (field.type == org.apache.thrift.protocol.TType.STRING) {
               this.key = iprot.readBinary();
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -8479,7 +8567,7 @@ public class Cassandra {
             if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
               this.column_parent = new ColumnParent();
               this.column_parent.read(iprot);
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -8487,14 +8575,14 @@ public class Cassandra {
             if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
               this.predicate = new SlicePredicate();
               this.predicate.read(iprot);
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
           case 4: // CONSISTENCY_LEVEL
             if (field.type == org.apache.thrift.protocol.TType.I32) {
               this.consistency_level = ConsistencyLevel.findByValue(iprot.readI32());
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -8698,13 +8786,13 @@ public class Cassandra {
     public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
     static {
       Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
-      tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT,
+      tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
-      tmpMap.put(_Fields.IRE, new org.apache.thrift.meta_data.FieldMetaData("ire", org.apache.thrift.TFieldRequirementType.DEFAULT,
+      tmpMap.put(_Fields.IRE, new org.apache.thrift.meta_data.FieldMetaData("ire", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT)));
-      tmpMap.put(_Fields.UE, new org.apache.thrift.meta_data.FieldMetaData("ue", org.apache.thrift.TFieldRequirementType.DEFAULT,
+      tmpMap.put(_Fields.UE, new org.apache.thrift.meta_data.FieldMetaData("ue", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT)));
-      tmpMap.put(_Fields.TE, new org.apache.thrift.meta_data.FieldMetaData("te", org.apache.thrift.TFieldRequirementType.DEFAULT,
+      tmpMap.put(_Fields.TE, new org.apache.thrift.meta_data.FieldMetaData("te", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT)));
       metaDataMap = Collections.unmodifiableMap(tmpMap);
       org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(get_count_result.class, metaDataMap);
@@ -9067,7 +9155,7 @@ public class Cassandra {
       while (true)
       {
         field = iprot.readFieldBegin();
-        if (field.type == org.apache.thrift.protocol.TType.STOP) {
+        if (field.type == org.apache.thrift.protocol.TType.STOP) { 
           break;
         }
         switch (field.id) {
@@ -9075,7 +9163,7 @@ public class Cassandra {
             if (field.type == org.apache.thrift.protocol.TType.I32) {
               this.success = iprot.readI32();
               setSuccessIsSet(true);
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -9083,7 +9171,7 @@ public class Cassandra {
             if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
               this.ire = new InvalidRequestException();
               this.ire.read(iprot);
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -9091,7 +9179,7 @@ public class Cassandra {
             if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
               this.ue = new UnavailableException();
               this.ue.read(iprot);
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -9099,7 +9187,7 @@ public class Cassandra {
             if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
               this.te = new TimedOutException();
               this.te.read(iprot);
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -9208,7 +9296,7 @@ public class Cassandra {
     public ColumnParent column_parent; // required
     public SlicePredicate predicate; // required
     /**
-     *
+     * 
      * @see ConsistencyLevel
      */
     public ConsistencyLevel consistency_level; // required
@@ -9219,7 +9307,7 @@ public class Cassandra {
       COLUMN_PARENT((short)2, "column_parent"),
       PREDICATE((short)3, "predicate"),
       /**
-       *
+       * 
        * @see ConsistencyLevel
        */
       CONSISTENCY_LEVEL((short)4, "consistency_level");
@@ -9289,14 +9377,14 @@ public class Cassandra {
     public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
     static {
       Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
-      tmpMap.put(_Fields.KEYS, new org.apache.thrift.meta_data.FieldMetaData("keys", org.apache.thrift.TFieldRequirementType.REQUIRED,
-          new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST,
+      tmpMap.put(_Fields.KEYS, new org.apache.thrift.meta_data.FieldMetaData("keys", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+          new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
               new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING              , true))));
-      tmpMap.put(_Fields.COLUMN_PARENT, new org.apache.thrift.meta_data.FieldMetaData("column_parent", org.apache.thrift.TFieldRequirementType.REQUIRED,
+      tmpMap.put(_Fields.COLUMN_PARENT, new org.apache.thrift.meta_data.FieldMetaData("column_parent", org.apache.thrift.TFieldRequirementType.REQUIRED, 
           new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ColumnParent.class)));
-      tmpMap.put(_Fields.PREDICATE, new org.apache.thrift.meta_data.FieldMetaData("predicate", org.apache.thrift.TFieldRequirementType.REQUIRED,
+      tmpMap.put(_Fields.PREDICATE, new org.apache.thrift.meta_data.FieldMetaData("predicate", org.apache.thrift.TFieldRequirementType.REQUIRED, 
           new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, SlicePredicate.class)));
-      tmpMap.put(_Fields.CONSISTENCY_LEVEL, new org.apache.thrift.meta_data.FieldMetaData("consistency_level", org.apache.thrift.TFieldRequirementType.REQUIRED,
+      tmpMap.put(_Fields.CONSISTENCY_LEVEL, new org.apache.thrift.meta_data.FieldMetaData("consistency_level", org.apache.thrift.TFieldRequirementType.REQUIRED, 
           new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, ConsistencyLevel.class)));
       metaDataMap = Collections.unmodifiableMap(tmpMap);
       org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(multiget_slice_args.class, metaDataMap);
@@ -9445,7 +9533,7 @@ public class Cassandra {
     }
 
     /**
-     *
+     * 
      * @see ConsistencyLevel
      */
     public ConsistencyLevel getConsistency_level() {
@@ -9453,7 +9541,7 @@ public class Cassandra {
     }
 
     /**
-     *
+     * 
      * @see ConsistencyLevel
      */
     public multiget_slice_args setConsistency_level(ConsistencyLevel consistency_level) {
@@ -9690,7 +9778,7 @@ public class Cassandra {
       while (true)
       {
         field = iprot.readFieldBegin();
-        if (field.type == org.apache.thrift.protocol.TType.STOP) {
+        if (field.type == org.apache.thrift.protocol.TType.STOP) { 
           break;
         }
         switch (field.id) {
@@ -9707,7 +9795,7 @@ public class Cassandra {
                 }
                 iprot.readListEnd();
               }
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -9715,7 +9803,7 @@ public class Cassandra {
             if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
               this.column_parent = new ColumnParent();
               this.column_parent.read(iprot);
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -9723,14 +9811,14 @@ public class Cassandra {
             if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
               this.predicate = new SlicePredicate();
               this.predicate.read(iprot);
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
           case 4: // CONSISTENCY_LEVEL
             if (field.type == org.apache.thrift.protocol.TType.I32) {
               this.consistency_level = ConsistencyLevel.findByValue(iprot.readI32());
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -9939,16 +10027,16 @@ public class Cassandra {
     public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
     static {
       Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
-      tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT,
-          new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP,
-              new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING              , true),
-              new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST,
+      tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, 
+              new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING              , true), 
+              new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
                   new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ColumnOrSuperColumn.class)))));
-      tmpMap.put(_Fields.IRE, new org.apache.thrift.meta_data.FieldMetaData("ire", org.apache.thrift.TFieldRequirementType.DEFAULT,
+      tmpMap.put(_Fields.IRE, new org.apache.thrift.meta_data.FieldMetaData("ire", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT)));
-      tmpMap.put(_Fields.UE, new org.apache.thrift.meta_data.FieldMetaData("ue", org.apache.thrift.TFieldRequirementType.DEFAULT,
+      tmpMap.put(_Fields.UE, new org.apache.thrift.meta_data.FieldMetaData("ue", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT)));
-      tmpMap.put(_Fields.TE, new org.apache.thrift.meta_data.FieldMetaData("te", org.apache.thrift.TFieldRequirementType.DEFAULT,
+      tmpMap.put(_Fields.TE, new org.apache.thrift.meta_data.FieldMetaData("te", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT)));
       metaDataMap = Collections.unmodifiableMap(tmpMap);
       org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(multiget_slice_result.class, metaDataMap);
@@ -10337,7 +10425,7 @@ public class Cassandra {
       while (true)
       {
         field = iprot.readFieldBegin();
-        if (field.type == org.apache.thrift.protocol.TType.STOP) {
+        if (field.type == org.apache.thrift.protocol.TType.STOP) { 
           break;
         }
         switch (field.id) {
@@ -10367,7 +10455,7 @@ public class Cassandra {
                 }
                 iprot.readMapEnd();
               }
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -10375,7 +10463,7 @@ public class Cassandra {
             if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
               this.ire = new InvalidRequestException();
               this.ire.read(iprot);
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -10383,7 +10471,7 @@ public class Cassandra {
             if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
               this.ue = new UnavailableException();
               this.ue.read(iprot);
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -10391,7 +10479,7 @@ public class Cassandra {
             if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
               this.te = new TimedOutException();
               this.te.read(iprot);
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -10519,7 +10607,7 @@ public class Cassandra {
     public ColumnParent column_parent; // required
     public SlicePredicate predicate; // required
     /**
-     *
+     * 
      * @see ConsistencyLevel
      */
     public ConsistencyLevel consistency_level; // required
@@ -10530,7 +10618,7 @@ public class Cassandra {
       COLUMN_PARENT((short)2, "column_parent"),
       PREDICATE((short)3, "predicate"),
       /**
-       *
+       * 
        * @see ConsistencyLevel
        */
       CONSISTENCY_LEVEL((short)4, "consistency_level");
@@ -10600,14 +10688,14 @@ public class Cassandra {
     public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
     static {
       Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
-      tmpMap.put(_Fields.KEYS, new org.apache.thrift.meta_data.FieldMetaData("keys", org.apache.thrift.TFieldRequirementType.REQUIRED,
-          new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST,
+      tmpMap.put(_Fields.KEYS, new org.apache.thrift.meta_data.FieldMetaData("keys", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+          new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
               new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING              , true))));
-      tmpMap.put(_Fields.COLUMN_PARENT, new org.apache.thrift.meta_data.FieldMetaData("column_parent", org.apache.thrift.TFieldRequirementType.REQUIRED,
+      tmpMap.put(_Fields.COLUMN_PARENT, new org.apache.thrift.meta_data.FieldMetaData("column_parent", org.apache.thrift.TFieldRequirementType.REQUIRED, 
           new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ColumnParent.class)));
-      tmpMap.put(_Fields.PREDICATE, new org.apache.thrift.meta_data.FieldMetaData("predicate", org.apache.thrift.TFieldRequirementType.REQUIRED,
+      tmpMap.put(_Fields.PREDICATE, new org.apache.thrift.meta_data.FieldMetaData("predicate", org.apache.thrift.TFieldRequirementType.REQUIRED, 
           new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, SlicePredicate.class)));
-      tmpMap.put(_Fields.CONSISTENCY_LEVEL, new org.apache.thrift.meta_data.FieldMetaData("consistency_level", org.apache.thrift.TFieldRequirementType.REQUIRED,
+      tmpMap.put(_Fields.CONSISTENCY_LEVEL, new org.apache.thrift.meta_data.FieldMetaData("consistency_level", org.apache.thrift.TFieldRequirementType.REQUIRED, 
           new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, ConsistencyLevel.class)));
       metaDataMap = Collections.unmodifiableMap(tmpMap);
       org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(multiget_count_args.class, metaDataMap);
@@ -10756,7 +10844,7 @@ public class Cassandra {
     }
 
     /**
-     *
+     * 
      * @see ConsistencyLevel
      */
     public ConsistencyLevel getConsistency_level() {
@@ -10764,7 +10852,7 @@ public class Cassandra {
     }
 
     /**
-     *
+     * 
      * @see ConsistencyLevel
      */
     public multiget_count_args setConsistency_level(ConsistencyLevel consistency_level) {
@@ -11001,7 +11089,7 @@ public class Cassandra {
       while (true)
       {
         field = iprot.readFieldBegin();
-        if (field.type == org.apache.thrift.protocol.TType.STOP) {
+        if (field.type == org.apache.thrift.protocol.TType.STOP) { 
           break;
         }
         switch (field.id) {
@@ -11018,7 +11106,7 @@ public class Cassandra {
                 }
                 iprot.readListEnd();
               }
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -11026,7 +11114,7 @@ public class Cassandra {
             if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
               this.column_parent = new ColumnParent();
               this.column_parent.read(iprot);
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -11034,14 +11122,14 @@ public class Cassandra {
             if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
               this.predicate = new SlicePredicate();
               this.predicate.read(iprot);
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
           case 4: // CONSISTENCY_LEVEL
             if (field.type == org.apache.thrift.protocol.TType.I32) {
               this.consistency_level = ConsistencyLevel.findByValue(iprot.readI32());
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -11250,15 +11338,15 @@ public class Cassandra {
     public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
     static {
       Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
-      tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT,
-          new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP,
-              new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING              , true),
+      tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, 
+              new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING              , true), 
               new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32))));
-      tmpMap.put(_Fields.IRE, new org.apache.thrift.meta_data.FieldMetaData("ire", org.apache.thrift.TFieldRequirementType.DEFAULT,
+      tmpMap.put(_Fields.IRE, new org.apache.thrift.meta_data.FieldMetaData("ire", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT)));
-      tmpMap.put(_Fields.UE, new org.apache.thrift.meta_data.FieldMetaData("ue", org.apache.thrift.TFieldRequirementType.DEFAULT,
+      tmpMap.put(_Fields.UE, new org.apache.thrift.meta_data.FieldMetaData("ue", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT)));
-      tmpMap.put(_Fields.TE, new org.apache.thrift.meta_data.FieldMetaData("te", org.apache.thrift.TFieldRequirementType.DEFAULT,
+      tmpMap.put(_Fields.TE, new org.apache.thrift.meta_data.FieldMetaData("te", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT)));
       metaDataMap = Collections.unmodifiableMap(tmpMap);
       org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(multiget_count_result.class, metaDataMap);
@@ -11644,7 +11732,7 @@ public class Cassandra {
       while (true)
       {
         field = iprot.readFieldBegin();
-        if (field.type == org.apache.thrift.protocol.TType.STOP) {
+        if (field.type == org.apache.thrift.protocol.TType.STOP) { 
           break;
         }
         switch (field.id) {
@@ -11663,7 +11751,7 @@ public class Cassandra {
                 }
                 iprot.readMapEnd();
               }
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -11671,7 +11759,7 @@ public class Cassandra {
             if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
               this.ire = new InvalidRequestException();
               this.ire.read(iprot);
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -11679,7 +11767,7 @@ public class Cassandra {
             if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
               this.ue = new UnavailableException();
               this.ue.read(iprot);
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -11687,7 +11775,7 @@ public class Cassandra {
             if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
               this.te = new TimedOutException();
               this.te.read(iprot);
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -11808,7 +11896,7 @@ public class Cassandra {
     public SlicePredicate predicate; // required
     public KeyRange range; // required
     /**
-     *
+     * 
      * @see ConsistencyLevel
      */
     public ConsistencyLevel consistency_level; // required
@@ -11819,7 +11907,7 @@ public class Cassandra {
       PREDICATE((short)2, "predicate"),
       RANGE((short)3, "range"),
       /**
-       *
+       * 
        * @see ConsistencyLevel
        */
       CONSISTENCY_LEVEL((short)4, "consistency_level");
@@ -11889,13 +11977,13 @@ public class Cassandra {
     public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
     static {
       Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
-      tmpMap.put(_Fields.COLUMN_PARENT, new org.apache.thrift.meta_data.FieldMetaData("column_parent", org.apache.thrift.TFieldRequirementType.REQUIRED,
+      tmpMap.put(_Fields.COLUMN_PARENT, new org.apache.thrift.meta_data.FieldMetaData("column_parent", org.apache.thrift.TFieldRequirementType.REQUIRED, 
           new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ColumnParent.class)));
-      tmpMap.put(_Fields.PREDICATE, new org.apache.thrift.meta_data.FieldMetaData("predicate", org.apache.thrift.TFieldRequirementType.REQUIRED,
+      tmpMap.put(_Fields.PREDICATE, new org.apache.thrift.meta_data.FieldMetaData("predicate", org.apache.thrift.TFieldRequirementType.REQUIRED, 
           new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, SlicePredicate.class)));
-      tmpMap.put(_Fields.RANGE, new org.apache.thrift.meta_data.FieldMetaData("range", org.apache.thrift.TFieldRequirementType.REQUIRED,
+      tmpMap.put(_Fields.RANGE, new org.apache.thrift.meta_data.FieldMetaData("range", org.apache.thrift.TFieldRequirementType.REQUIRED, 
           new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, KeyRange.class)));
-      tmpMap.put(_Fields.CONSISTENCY_LEVEL, new org.apache.thrift.meta_data.FieldMetaData("consistency_level", org.apache.thrift.TFieldRequirementType.REQUIRED,
+      tmpMap.put(_Fields.CONSISTENCY_LEVEL, new org.apache.thrift.meta_data.FieldMetaData("consistency_level", org.apache.thrift.TFieldRequirementType.REQUIRED, 
           new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, ConsistencyLevel.class)));
       metaDataMap = Collections.unmodifiableMap(tmpMap);
       org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(get_range_slices_args.class, metaDataMap);
@@ -12023,7 +12111,7 @@ public class Cassandra {
     }
 
     /**
-     *
+     * 
      * @see ConsistencyLevel
      */
     public ConsistencyLevel getConsistency_level() {
@@ -12031,7 +12119,7 @@ public class Cassandra {
     }
 
     /**
-     *
+     * 
      * @see ConsistencyLevel
      */
     public get_range_slices_args setConsistency_level(ConsistencyLevel consistency_level) {
@@ -12268,7 +12356,7 @@ public class Cassandra {
       while (true)
       {
         field = iprot.readFieldBegin();
-        if (field.type == org.apache.thrift.protocol.TType.STOP) {
+        if (field.type == org.apache.thrift.protocol.TType.STOP) { 
           break;
         }
         switch (field.id) {
@@ -12276,7 +12364,7 @@ public class Cassandra {
             if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
               this.column_parent = new ColumnParent();
               this.column_parent.read(iprot);
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -12284,7 +12372,7 @@ public class Cassandra {
             if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
               this.predicate = new SlicePredicate();
               this.predicate.read(iprot);
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -12292,14 +12380,14 @@ public class Cassandra {
             if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
               this.range = new KeyRange();
               this.range.read(iprot);
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
           case 4: // CONSISTENCY_LEVEL
             if (field.type == org.apache.thrift.protocol.TType.I32) {
               this.consistency_level = ConsistencyLevel.findByValue(iprot.readI32());
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -12501,14 +12589,14 @@ public class Cassandra {
     public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
     static {
       Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
-      tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT,
-          new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST,
+      tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
               new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, KeySlice.class))));
-      tmpMap.put(_Fields.IRE, new org.apache.thrift.meta_data.FieldMetaData("ire", org.apache.thrift.TFieldRequirementType.DEFAULT,
+      tmpMap.put(_Fields.IRE, new org.apache.thrift.meta_data.FieldMetaData("ire", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT)));
-      tmpMap.put(_Fields.UE, new org.apache.thrift.meta_data.FieldMetaData("ue", org.apache.thrift.TFieldRequirementType.DEFAULT,
+      tmpMap.put(_Fields.UE, new org.apache.thrift.meta_data.FieldMetaData("ue", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT)));
-      tmpMap.put(_Fields.TE, new org.apache.thrift.meta_data.FieldMetaData("te", org.apache.thrift.TFieldRequirementType.DEFAULT,
+      tmpMap.put(_Fields.TE, new org.apache.thrift.meta_data.FieldMetaData("te", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT)));
       metaDataMap = Collections.unmodifiableMap(tmpMap);
       org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(get_range_slices_result.class, metaDataMap);
@@ -12889,7 +12977,7 @@ public class Cassandra {
       while (true)
       {
         field = iprot.readFieldBegin();
-        if (field.type == org.apache.thrift.protocol.TType.STOP) {
+        if (field.type == org.apache.thrift.protocol.TType.STOP) { 
           break;
         }
         switch (field.id) {
@@ -12907,7 +12995,7 @@ public class Cassandra {
                 }
                 iprot.readListEnd();
               }
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -12915,7 +13003,7 @@ public class Cassandra {
             if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
               this.ire = new InvalidRequestException();
               this.ire.read(iprot);
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -12923,7 +13011,7 @@ public class Cassandra {
             if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
               this.ue = new UnavailableException();
               this.ue.read(iprot);
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -12931,7 +13019,7 @@ public class Cassandra {
             if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
               this.te = new TimedOutException();
               this.te.read(iprot);
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -13051,7 +13139,7 @@ public class Cassandra {
     public KeyRange range; // required
     public ByteBuffer start_column; // required
     /**
-     *
+     * 
      * @see ConsistencyLevel
      */
     public ConsistencyLevel consistency_level; // required
@@ -13062,7 +13150,7 @@ public class Cassandra {
       RANGE((short)2, "range"),
       START_COLUMN((short)3, "start_column"),
       /**
-       *
+       * 
        * @see ConsistencyLevel
        */
       CONSISTENCY_LEVEL((short)4, "consistency_level");
@@ -13132,13 +13220,13 @@ public class Cassandra {
     public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
     static {
       Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
-      tmpMap.put(_Fields.COLUMN_FAMILY, new org.apache.thrift.meta_data.FieldMetaData("column_family", org.apache.thrift.TFieldRequirementType.REQUIRED,
+      tmpMap.put(_Fields.COLUMN_FAMILY, new org.apache.thrift.meta_data.FieldMetaData("column_family", org.apache.thrift.TFieldRequirementType.REQUIRED, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
-      tmpMap.put(_Fields.RANGE, new org.apache.thrift.meta_data.FieldMetaData("range", org.apache.thrift.TFieldRequirementType.REQUIRED,
+      tmpMap.put(_Fields.RANGE, new org.apache.thrift.meta_data.FieldMetaData("range", org.apache.thrift.TFieldRequirementType.REQUIRED, 
           new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, KeyRange.class)));
-      tmpMap.put(_Fields.START_COLUMN, new org.apache.thrift.meta_data.FieldMetaData("start_column", org.apache.thrift.TFieldRequirementType.REQUIRED,
+      tmpMap.put(_Fields.START_COLUMN, new org.apache.thrift.meta_data.FieldMetaData("start_column", org.apache.thrift.TFieldRequirementType.REQUIRED, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING          , true)));
-      tmpMap.put(_Fields.CONSISTENCY_LEVEL, new org.apache.thrift.meta_data.FieldMetaData("consistency_level", org.apache.thrift.TFieldRequirementType.REQUIRED,
+      tmpMap.put(_Fields.CONSISTENCY_LEVEL, new org.apache.thrift.meta_data.FieldMetaData("consistency_level", org.apache.thrift.TFieldRequirementType.REQUIRED, 
           new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, ConsistencyLevel.class)));
       metaDataMap = Collections.unmodifiableMap(tmpMap);
       org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(get_paged_slice_args.class, metaDataMap);
@@ -13277,7 +13365,7 @@ public class Cassandra {
     }
 
     /**
-     *
+     * 
      * @see ConsistencyLevel
      */
     public ConsistencyLevel getConsistency_level() {
@@ -13285,7 +13373,7 @@ public class Cassandra {
     }
 
     /**
-     *
+     * 
      * @see ConsistencyLevel
      */
     public get_paged_slice_args setConsistency_level(ConsistencyLevel consistency_level) {
@@ -13522,14 +13610,14 @@ public class Cassandra {
       while (true)
       {
         field = iprot.readFieldBegin();
-        if (field.type == org.apache.thrift.protocol.TType.STOP) {
+        if (field.type == org.apache.thrift.protocol.TType.STOP) { 
           break;
         }
         switch (field.id) {
           case 1: // COLUMN_FAMILY
             if (field.type == org.apache.thrift.protocol.TType.STRING) {
               this.column_family = iprot.readString();
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -13537,21 +13625,21 @@ public class Cassandra {
             if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
               this.range = new KeyRange();
               this.range.read(iprot);
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
           case 3: // START_COLUMN
             if (field.type == org.apache.thrift.protocol.TType.STRING) {
               this.start_column = iprot.readBinary();
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
           case 4: // CONSISTENCY_LEVEL
             if (field.type == org.apache.thrift.protocol.TType.I32) {
               this.consistency_level = ConsistencyLevel.findByValue(iprot.readI32());
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -13753,14 +13841,14 @@ public class Cassandra {
     public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
     static {
       Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
-      tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT,
-          new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST,
+      tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
               new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, KeySlice.class))));
-      tmpMap.put(_Fields.IRE, new org.apache.thrift.meta_data.FieldMetaData("ire", org.apache.thrift.TFieldRequirementType.DEFAULT,
+      tmpMap.put(_Fields.IRE, new org.apache.thrift.meta_data.FieldMetaData("ire", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT)));
-      tmpMap.put(_Fields.UE, new org.apache.thrift.meta_data.FieldMetaData("ue", org.apache.thrift.TFieldRequirementType.DEFAULT,
+      tmpMap.put(_Fields.UE, new org.apache.thrift.meta_data.FieldMetaData("ue", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT)));
-      tmpMap.put(_Fields.TE, new org.apache.thrift.meta_data.FieldMetaData("te", org.apache.thrift.TFieldRequirementType.DEFAULT,
+      tmpMap.put(_Fields.TE, new org.apache.thrift.meta_data.FieldMetaData("te", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT)));
       metaDataMap = Collections.unmodifiableMap(tmpMap);
       org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(get_paged_slice_result.class, metaDataMap);
@@ -14141,7 +14229,7 @@ public class Cassandra {
       while (true)
       {
         field = iprot.readFieldBegin();
-        if (field.type == org.apache.thrift.protocol.TType.STOP) {
+        if (field.type == org.apache.thrift.protocol.TType.STOP) { 
           break;
         }
         switch (field.id) {
@@ -14159,7 +14247,7 @@ public class Cassandra {
                 }
                 iprot.readListEnd();
               }
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -14167,7 +14255,7 @@ public class Cassandra {
             if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
               this.ire = new InvalidRequestException();
               this.ire.read(iprot);
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -14175,7 +14263,7 @@ public class Cassandra {
             if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
               this.ue = new UnavailableException();
               this.ue.read(iprot);
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -14183,7 +14271,7 @@ public class Cassandra {
             if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
               this.te = new TimedOutException();
               this.te.read(iprot);
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -14303,7 +14391,7 @@ public class Cassandra {
     public IndexClause index_clause; // required
     public SlicePredicate column_predicate; // required
     /**
-     *
+     * 
      * @see ConsistencyLevel
      */
     public ConsistencyLevel consistency_level; // required
@@ -14314,7 +14402,7 @@ public class Cassandra {
       INDEX_CLAUSE((short)2, "index_clause"),
       COLUMN_PREDICATE((short)3, "column_predicate"),
       /**
-       *
+       * 
        * @see ConsistencyLevel
        */
       CONSISTENCY_LEVEL((short)4, "consistency_level");
@@ -14384,13 +14472,13 @@ public class Cassandra {
     public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
     static {
       Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
-      tmpMap.put(_Fields.COLUMN_PARENT, new org.apache.thrift.meta_data.FieldMetaData("column_parent", org.apache.thrift.TFieldRequirementType.REQUIRED,
+      tmpMap.put(_Fields.COLUMN_PARENT, new org.apache.thrift.meta_data.FieldMetaData("column_parent", org.apache.thrift.TFieldRequirementType.REQUIRED, 
           new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ColumnParent.class)));
-      tmpMap.put(_Fields.INDEX_CLAUSE, new org.apache.thrift.meta_data.FieldMetaData("index_clause", org.apache.thrift.TFieldRequirementType.REQUIRED,
+      tmpMap.put(_Fields.INDEX_CLAUSE, new org.apache.thrift.meta_data.FieldMetaData("index_clause", org.apache.thrift.TFieldRequirementType.REQUIRED, 
           new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, IndexClause.class)));
-      tmpMap.put(_Fields.COLUMN_PREDICATE, new org.apache.thrift.meta_data.FieldMetaData("column_predicate", org.apache.thrift.TFieldRequirementType.REQUIRED,
+      tmpMap.put(_Fields.COLUMN_PREDICATE, new org.apache.thrift.meta_data.FieldMetaData("column_predicate", org.apache.thrift.TFieldRequirementType.REQUIRED, 
           new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, SlicePredicate.class)));
-      tmpMap.put(_Fields.CONSISTENCY_LEVEL, new org.apache.thrift.meta_data.FieldMetaData("consistency_level", org.apache.thrift.TFieldRequirementType.REQUIRED,
+      tmpMap.put(_Fields.CONSISTENCY_LEVEL, new org.apache.thrift.meta_data.FieldMetaData("consistency_level", org.apache.thrift.TFieldRequirementType.REQUIRED, 
           new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, ConsistencyLevel.class)));
       metaDataMap = Collections.unmodifiableMap(tmpMap);
       org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(get_indexed_slices_args.class, metaDataMap);
@@ -14518,7 +14606,7 @@ public class Cassandra {
     }
 
     /**
-     *
+     * 
      * @see ConsistencyLevel
      */
     public ConsistencyLevel getConsistency_level() {
@@ -14526,7 +14614,7 @@ public class Cassandra {
     }
 
     /**
-     *
+     * 
      * @see ConsistencyLevel
      */
     public get_indexed_slices_args setConsistency_level(ConsistencyLevel consistency_level) {
@@ -14763,7 +14851,7 @@ public class Cassandra {
       while (true)
       {
         field = iprot.readFieldBegin();
-        if (field.type == org.apache.thrift.protocol.TType.STOP) {
+        if (field.type == org.apache.thrift.protocol.TType.STOP) { 

<TRUNCATED>

[11/16] Merge branch 'cassandra-1.2' into cassandra-2.0

Posted by jb...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/f2a82ee7/interface/thrift/gen-java/org/apache/cassandra/thrift/Cassandra.java
----------------------------------------------------------------------
diff --cc interface/thrift/gen-java/org/apache/cassandra/thrift/Cassandra.java
index 837acfc,5c8d286..15b99fa
--- a/interface/thrift/gen-java/org/apache/cassandra/thrift/Cassandra.java
+++ b/interface/thrift/gen-java/org/apache/cassandra/thrift/Cassandra.java
@@@ -437,91 -387,91 +444,93 @@@ public class Cassandra 
  
    public interface AsyncIface {
  
 -    public void login(AuthenticationRequest auth_request, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.login_call> resultHandler) throws org.apache.thrift.TException;
 +    public void login(AuthenticationRequest auth_request, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
 -    public void set_keyspace(String keyspace, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.set_keyspace_call> resultHandler) throws org.apache.thrift.TException;
 +    public void set_keyspace(String keyspace, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
 -    public void get(ByteBuffer key, ColumnPath column_path, ConsistencyLevel consistency_level, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.get_call> resultHandler) throws org.apache.thrift.TException;
 +    public void get(ByteBuffer key, ColumnPath column_path, ConsistencyLevel consistency_level, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
 -    public void get_slice(ByteBuffer key, ColumnParent column_parent, SlicePredicate predicate, ConsistencyLevel consistency_level, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.get_slice_call> resultHandler) throws org.apache.thrift.TException;
 +    public void get_slice(ByteBuffer key, ColumnParent column_parent, SlicePredicate predicate, ConsistencyLevel consistency_level, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
 -    public void get_count(ByteBuffer key, ColumnParent column_parent, SlicePredicate predicate, ConsistencyLevel consistency_level, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.get_count_call> resultHandler) throws org.apache.thrift.TException;
 +    public void get_count(ByteBuffer key, ColumnParent column_parent, SlicePredicate predicate, ConsistencyLevel consistency_level, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
 -    public void multiget_slice(List<ByteBuffer> keys, ColumnParent column_parent, SlicePredicate predicate, ConsistencyLevel consistency_level, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.multiget_slice_call> resultHandler) throws org.apache.thrift.TException;
 +    public void multiget_slice(List<ByteBuffer> keys, ColumnParent column_parent, SlicePredicate predicate, ConsistencyLevel consistency_level, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
 -    public void multiget_count(List<ByteBuffer> keys, ColumnParent column_parent, SlicePredicate predicate, ConsistencyLevel consistency_level, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.multiget_count_call> resultHandler) throws org.apache.thrift.TException;
 +    public void multiget_count(List<ByteBuffer> keys, ColumnParent column_parent, SlicePredicate predicate, ConsistencyLevel consistency_level, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
 -    public void get_range_slices(ColumnParent column_parent, SlicePredicate predicate, KeyRange range, ConsistencyLevel consistency_level, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.get_range_slices_call> resultHandler) throws org.apache.thrift.TException;
 +    public void get_range_slices(ColumnParent column_parent, SlicePredicate predicate, KeyRange range, ConsistencyLevel consistency_level, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
 -    public void get_paged_slice(String column_family, KeyRange range, ByteBuffer start_column, ConsistencyLevel consistency_level, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.get_paged_slice_call> resultHandler) throws org.apache.thrift.TException;
 +    public void get_paged_slice(String column_family, KeyRange range, ByteBuffer start_column, ConsistencyLevel consistency_level, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
 -    public void get_indexed_slices(ColumnParent column_parent, IndexClause index_clause, SlicePredicate column_predicate, ConsistencyLevel consistency_level, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.get_indexed_slices_call> resultHandler) throws org.apache.thrift.TException;
 +    public void get_indexed_slices(ColumnParent column_parent, IndexClause index_clause, SlicePredicate column_predicate, ConsistencyLevel consistency_level, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
 -    public void insert(ByteBuffer key, ColumnParent column_parent, Column column, ConsistencyLevel consistency_level, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.insert_call> resultHandler) throws org.apache.thrift.TException;
 +    public void insert(ByteBuffer key, ColumnParent column_parent, Column column, ConsistencyLevel consistency_level, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
 -    public void add(ByteBuffer key, ColumnParent column_parent, CounterColumn column, ConsistencyLevel consistency_level, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.add_call> resultHandler) throws org.apache.thrift.TException;
 +    public void add(ByteBuffer key, ColumnParent column_parent, CounterColumn column, ConsistencyLevel consistency_level, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
 -    public void remove(ByteBuffer key, ColumnPath column_path, long timestamp, ConsistencyLevel consistency_level, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.remove_call> resultHandler) throws org.apache.thrift.TException;
 +    public void cas(ByteBuffer key, String column_family, List<Column> expected, List<Column> updates, ConsistencyLevel serial_consistency_level, ConsistencyLevel commit_consistency_level, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
 -    public void remove_counter(ByteBuffer key, ColumnPath path, ConsistencyLevel consistency_level, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.remove_counter_call> resultHandler) throws org.apache.thrift.TException;
 +    public void remove(ByteBuffer key, ColumnPath column_path, long timestamp, ConsistencyLevel consistency_level, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
 -    public void batch_mutate(Map<ByteBuffer,Map<String,List<Mutation>>> mutation_map, ConsistencyLevel consistency_level, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.batch_mutate_call> resultHandler) throws org.apache.thrift.TException;
 +    public void remove_counter(ByteBuffer key, ColumnPath path, ConsistencyLevel consistency_level, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
 -    public void atomic_batch_mutate(Map<ByteBuffer,Map<String,List<Mutation>>> mutation_map, ConsistencyLevel consistency_level, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.atomic_batch_mutate_call> resultHandler) throws org.apache.thrift.TException;
 +    public void batch_mutate(Map<ByteBuffer,Map<String,List<Mutation>>> mutation_map, ConsistencyLevel consistency_level, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
 -    public void truncate(String cfname, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.truncate_call> resultHandler) throws org.apache.thrift.TException;
 +    public void atomic_batch_mutate(Map<ByteBuffer,Map<String,List<Mutation>>> mutation_map, ConsistencyLevel consistency_level, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
 -    public void describe_schema_versions(org.apache.thrift.async.AsyncMethodCallback<AsyncClient.describe_schema_versions_call> resultHandler) throws org.apache.thrift.TException;
 +    public void truncate(String cfname, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
 -    public void describe_keyspaces(org.apache.thrift.async.AsyncMethodCallback<AsyncClient.describe_keyspaces_call> resultHandler) throws org.apache.thrift.TException;
 +    public void describe_schema_versions(org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
 -    public void describe_cluster_name(org.apache.thrift.async.AsyncMethodCallback<AsyncClient.describe_cluster_name_call> resultHandler) throws org.apache.thrift.TException;
 +    public void describe_keyspaces(org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
 -    public void describe_version(org.apache.thrift.async.AsyncMethodCallback<AsyncClient.describe_version_call> resultHandler) throws org.apache.thrift.TException;
 +    public void describe_cluster_name(org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
 -    public void describe_ring(String keyspace, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.describe_ring_call> resultHandler) throws org.apache.thrift.TException;
 +    public void describe_version(org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
 -    public void describe_local_ring(String keyspace, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.describe_local_ring_call> resultHandler) throws org.apache.thrift.TException;
 +    public void describe_ring(String keyspace, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
 -    public void describe_token_map(org.apache.thrift.async.AsyncMethodCallback<AsyncClient.describe_token_map_call> resultHandler) throws org.apache.thrift.TException;
++    public void describe_local_ring(String keyspace, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+ 
 -    public void describe_partitioner(org.apache.thrift.async.AsyncMethodCallback<AsyncClient.describe_partitioner_call> resultHandler) throws org.apache.thrift.TException;
 +    public void describe_token_map(org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
 -    public void describe_snitch(org.apache.thrift.async.AsyncMethodCallback<AsyncClient.describe_snitch_call> resultHandler) throws org.apache.thrift.TException;
 +    public void describe_partitioner(org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
 -    public void describe_keyspace(String keyspace, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.describe_keyspace_call> resultHandler) throws org.apache.thrift.TException;
 +    public void describe_snitch(org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
 -    public void describe_splits(String cfName, String start_token, String end_token, int keys_per_split, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.describe_splits_call> resultHandler) throws org.apache.thrift.TException;
 +    public void describe_keyspace(String keyspace, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
 -    public void trace_next_query(org.apache.thrift.async.AsyncMethodCallback<AsyncClient.trace_next_query_call> resultHandler) throws org.apache.thrift.TException;
 +    public void describe_splits(String cfName, String start_token, String end_token, int keys_per_split, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
 -    public void describe_splits_ex(String cfName, String start_token, String end_token, int keys_per_split, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.describe_splits_ex_call> resultHandler) throws org.apache.thrift.TException;
 +    public void trace_next_query(org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
 -    public void system_add_column_family(CfDef cf_def, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.system_add_column_family_call> resultHandler) throws org.apache.thrift.TException;
 +    public void describe_splits_ex(String cfName, String start_token, String end_token, int keys_per_split, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
 -    public void system_drop_column_family(String column_family, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.system_drop_column_family_call> resultHandler) throws org.apache.thrift.TException;
 +    public void system_add_column_family(CfDef cf_def, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
 -    public void system_add_keyspace(KsDef ks_def, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.system_add_keyspace_call> resultHandler) throws org.apache.thrift.TException;
 +    public void system_drop_column_family(String column_family, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
 -    public void system_drop_keyspace(String keyspace, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.system_drop_keyspace_call> resultHandler) throws org.apache.thrift.TException;
 +    public void system_add_keyspace(KsDef ks_def, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
 -    public void system_update_keyspace(KsDef ks_def, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.system_update_keyspace_call> resultHandler) throws org.apache.thrift.TException;
 +    public void system_drop_keyspace(String keyspace, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
 -    public void system_update_column_family(CfDef cf_def, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.system_update_column_family_call> resultHandler) throws org.apache.thrift.TException;
 +    public void system_update_keyspace(KsDef ks_def, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
 -    public void execute_cql_query(ByteBuffer query, Compression compression, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.execute_cql_query_call> resultHandler) throws org.apache.thrift.TException;
 +    public void system_update_column_family(CfDef cf_def, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
 -    public void execute_cql3_query(ByteBuffer query, Compression compression, ConsistencyLevel consistency, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.execute_cql3_query_call> resultHandler) throws org.apache.thrift.TException;
 +    public void execute_cql_query(ByteBuffer query, Compression compression, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
 -    public void prepare_cql_query(ByteBuffer query, Compression compression, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.prepare_cql_query_call> resultHandler) throws org.apache.thrift.TException;
 +    public void execute_cql3_query(ByteBuffer query, Compression compression, ConsistencyLevel consistency, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
 -    public void prepare_cql3_query(ByteBuffer query, Compression compression, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.prepare_cql3_query_call> resultHandler) throws org.apache.thrift.TException;
 +    public void prepare_cql_query(ByteBuffer query, Compression compression, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
 -    public void execute_prepared_cql_query(int itemId, List<ByteBuffer> values, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.execute_prepared_cql_query_call> resultHandler) throws org.apache.thrift.TException;
 +    public void prepare_cql3_query(ByteBuffer query, Compression compression, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
 -    public void execute_prepared_cql3_query(int itemId, List<ByteBuffer> values, ConsistencyLevel consistency, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.execute_prepared_cql3_query_call> resultHandler) throws org.apache.thrift.TException;
 +    public void execute_prepared_cql_query(int itemId, List<ByteBuffer> values, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
 -    public void set_cql_version(String version, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.set_cql_version_call> resultHandler) throws org.apache.thrift.TException;
 +    public void execute_prepared_cql3_query(int itemId, List<ByteBuffer> values, ConsistencyLevel consistency, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
 +
 +    public void set_cql_version(String version, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
    }
  
@@@ -2689,7 -2581,39 +2724,39 @@@
        }
      }
  
 -    public void describe_local_ring(String keyspace, org.apache.thrift.async.AsyncMethodCallback<describe_local_ring_call> resultHandler) throws org.apache.thrift.TException {
++    public void describe_local_ring(String keyspace, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+       checkReady();
+       describe_local_ring_call method_call = new describe_local_ring_call(keyspace, resultHandler, this, ___protocolFactory, ___transport);
+       this.___currentMethod = method_call;
+       ___manager.call(method_call);
+     }
+ 
+     public static class describe_local_ring_call extends org.apache.thrift.async.TAsyncMethodCall {
+       private String keyspace;
 -      public describe_local_ring_call(String keyspace, org.apache.thrift.async.AsyncMethodCallback<describe_local_ring_call> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
++      public describe_local_ring_call(String keyspace, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+         super(client, protocolFactory, transport, resultHandler, false);
+         this.keyspace = keyspace;
+       }
+ 
+       public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+         prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("describe_local_ring", org.apache.thrift.protocol.TMessageType.CALL, 0));
+         describe_local_ring_args args = new describe_local_ring_args();
+         args.setKeyspace(keyspace);
+         args.write(prot);
+         prot.writeMessageEnd();
+       }
+ 
+       public List<TokenRange> getResult() throws InvalidRequestException, org.apache.thrift.TException {
+         if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+           throw new IllegalStateException("Method call not finished!");
+         }
+         org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+         org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+         return (new Client(prot)).recv_describe_local_ring();
+       }
+     }
+ 
 -    public void describe_token_map(org.apache.thrift.async.AsyncMethodCallback<describe_token_map_call> resultHandler) throws org.apache.thrift.TException {
 +    public void describe_token_map(org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
        checkReady();
        describe_token_map_call method_call = new describe_token_map_call(resultHandler, this, ___protocolFactory, ___transport);
        this.___currentMethod = method_call;
@@@ -4031,7 -3839,27 +4099,31 @@@
        }
      }
  
 -    private static class describe_local_ring<I extends Iface> extends org.apache.thrift.ProcessFunction<I, describe_local_ring_args> {
++    public static class describe_local_ring<I extends Iface> extends org.apache.thrift.ProcessFunction<I, describe_local_ring_args> {
+       public describe_local_ring() {
+         super("describe_local_ring");
+       }
+ 
 -      protected describe_local_ring_args getEmptyArgsInstance() {
++      public describe_local_ring_args getEmptyArgsInstance() {
+         return new describe_local_ring_args();
+       }
+ 
 -      protected describe_local_ring_result getResult(I iface, describe_local_ring_args args) throws org.apache.thrift.TException {
++      protected boolean isOneway() {
++        return false;
++      }
++
++      public describe_local_ring_result getResult(I iface, describe_local_ring_args args) throws org.apache.thrift.TException {
+         describe_local_ring_result result = new describe_local_ring_result();
+         try {
+           result.success = iface.describe_local_ring(args.keyspace);
+         } catch (InvalidRequestException ire) {
+           result.ire = ire;
+         }
+         return result;
+       }
+     }
+ 
 -    private static class describe_token_map<I extends Iface> extends org.apache.thrift.ProcessFunction<I, describe_token_map_args> {
 +    public static class describe_token_map<I extends Iface> extends org.apache.thrift.ProcessFunction<I, describe_token_map_args> {
        public describe_token_map() {
          super("describe_token_map");
        }
@@@ -4539,2942 -4287,201 +4631,3000 @@@
  
    }
  
 -  public static class login_args implements org.apache.thrift.TBase<login_args, login_args._Fields>, java.io.Serializable, Cloneable   {
 -    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("login_args");
 +  public static class AsyncProcessor<I extends AsyncIface> extends org.apache.thrift.TBaseAsyncProcessor<I> {
 +    private static final Logger LOGGER = LoggerFactory.getLogger(AsyncProcessor.class.getName());
 +    public AsyncProcessor(I iface) {
 +      super(iface, getProcessMap(new HashMap<String, org.apache.thrift.AsyncProcessFunction<I, ? extends org.apache.thrift.TBase, ?>>()));
 +    }
  
 -    private static final org.apache.thrift.protocol.TField AUTH_REQUEST_FIELD_DESC = new org.apache.thrift.protocol.TField("auth_request", org.apache.thrift.protocol.TType.STRUCT, (short)1);
 +    protected AsyncProcessor(I iface, Map<String,  org.apache.thrift.AsyncProcessFunction<I, ? extends  org.apache.thrift.TBase, ?>> processMap) {
 +      super(iface, getProcessMap(processMap));
 +    }
  
 -    public AuthenticationRequest auth_request; // required
 +    private static <I extends AsyncIface> Map<String,  org.apache.thrift.AsyncProcessFunction<I, ? extends  org.apache.thrift.TBase,?>> getProcessMap(Map<String,  org.apache.thrift.AsyncProcessFunction<I, ? extends  org.apache.thrift.TBase, ?>> processMap) {
 +      processMap.put("login", new login());
 +      processMap.put("set_keyspace", new set_keyspace());
 +      processMap.put("get", new get());
 +      processMap.put("get_slice", new get_slice());
 +      processMap.put("get_count", new get_count());
 +      processMap.put("multiget_slice", new multiget_slice());
 +      processMap.put("multiget_count", new multiget_count());
 +      processMap.put("get_range_slices", new get_range_slices());
 +      processMap.put("get_paged_slice", new get_paged_slice());
 +      processMap.put("get_indexed_slices", new get_indexed_slices());
 +      processMap.put("insert", new insert());
 +      processMap.put("add", new add());
 +      processMap.put("cas", new cas());
 +      processMap.put("remove", new remove());
 +      processMap.put("remove_counter", new remove_counter());
 +      processMap.put("batch_mutate", new batch_mutate());
 +      processMap.put("atomic_batch_mutate", new atomic_batch_mutate());
 +      processMap.put("truncate", new truncate());
 +      processMap.put("describe_schema_versions", new describe_schema_versions());
 +      processMap.put("describe_keyspaces", new describe_keyspaces());
 +      processMap.put("describe_cluster_name", new describe_cluster_name());
 +      processMap.put("describe_version", new describe_version());
 +      processMap.put("describe_ring", new describe_ring());
++      processMap.put("describe_local_ring", new describe_local_ring());
 +      processMap.put("describe_token_map", new describe_token_map());
 +      processMap.put("describe_partitioner", new describe_partitioner());
 +      processMap.put("describe_snitch", new describe_snitch());
 +      processMap.put("describe_keyspace", new describe_keyspace());
 +      processMap.put("describe_splits", new describe_splits());
 +      processMap.put("trace_next_query", new trace_next_query());
 +      processMap.put("describe_splits_ex", new describe_splits_ex());
 +      processMap.put("system_add_column_family", new system_add_column_family());
 +      processMap.put("system_drop_column_family", new system_drop_column_family());
 +      processMap.put("system_add_keyspace", new system_add_keyspace());
 +      processMap.put("system_drop_keyspace", new system_drop_keyspace());
 +      processMap.put("system_update_keyspace", new system_update_keyspace());
 +      processMap.put("system_update_column_family", new system_update_column_family());
 +      processMap.put("execute_cql_query", new execute_cql_query());
 +      processMap.put("execute_cql3_query", new execute_cql3_query());
 +      processMap.put("prepare_cql_query", new prepare_cql_query());
 +      processMap.put("prepare_cql3_query", new prepare_cql3_query());
 +      processMap.put("execute_prepared_cql_query", new execute_prepared_cql_query());
 +      processMap.put("execute_prepared_cql3_query", new execute_prepared_cql3_query());
 +      processMap.put("set_cql_version", new set_cql_version());
 +      return processMap;
 +    }
  
 -    /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
 -    public enum _Fields implements org.apache.thrift.TFieldIdEnum {
 -      AUTH_REQUEST((short)1, "auth_request");
 +    public static class login<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, login_args, Void> {
 +      public login() {
 +        super("login");
 +      }
  
 -      private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 +      public login_args getEmptyArgsInstance() {
 +        return new login_args();
 +      }
  
 -      static {
 -        for (_Fields field : EnumSet.allOf(_Fields.class)) {
 -          byName.put(field.getFieldName(), field);
 -        }
 +      public AsyncMethodCallback<Void> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
 +        final org.apache.thrift.AsyncProcessFunction fcall = this;
 +        return new AsyncMethodCallback<Void>() { 
 +          public void onComplete(Void o) {
 +            login_result result = new login_result();
 +            try {
 +              fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
 +              return;
 +            } catch (Exception e) {
 +              LOGGER.error("Exception writing to internal frame buffer", e);
 +            }
 +            fb.close();
 +          }
 +          public void onError(Exception e) {
 +            byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
 +            org.apache.thrift.TBase msg;
 +            login_result result = new login_result();
 +            if (e instanceof AuthenticationException) {
 +                        result.authnx = (AuthenticationException) e;
 +                        result.setAuthnxIsSet(true);
 +                        msg = result;
 +            }
 +            else             if (e instanceof AuthorizationException) {
 +                        result.authzx = (AuthorizationException) e;
 +                        result.setAuthzxIsSet(true);
 +                        msg = result;
 +            }
 +             else 
 +            {
 +              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
 +              msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
 +            }
 +            try {
 +              fcall.sendResponse(fb,msg,msgType,seqid);
 +              return;
 +            } catch (Exception ex) {
 +              LOGGER.error("Exception writing to internal frame buffer", ex);
 +            }
 +            fb.close();
 +          }
 +        };
        }
  
 -      /**
 -       * Find the _Fields constant that matches fieldId, or null if its not found.
 -       */
 -      public static _Fields findByThriftId(int fieldId) {
 -        switch(fieldId) {
 -          case 1: // AUTH_REQUEST
 -            return AUTH_REQUEST;
 -          default:
 -            return null;
 -        }
 +      protected boolean isOneway() {
 +        return false;
        }
  
 -      /**
 -       * 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;
 +      public void start(I iface, login_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws TException {
 +        iface.login(args.auth_request,resultHandler);
        }
 +    }
  
 -      /**
 -       * Find the _Fields constant that matches name, or null if its not found.
 -       */
 -      public static _Fields findByName(String name) {
 -        return byName.get(name);
 +    public static class set_keyspace<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, set_keyspace_args, Void> {
 +      public set_keyspace() {
 +        super("set_keyspace");
        }
  
 -      private final short _thriftId;
 -      private final String _fieldName;
 +      public set_keyspace_args getEmptyArgsInstance() {
 +        return new set_keyspace_args();
 +      }
  
 -      _Fields(short thriftId, String fieldName) {
 -        _thriftId = thriftId;
 -        _fieldName = fieldName;
 +      public AsyncMethodCallback<Void> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
 +        final org.apache.thrift.AsyncProcessFunction fcall = this;
 +        return new AsyncMethodCallback<Void>() { 
 +          public void onComplete(Void o) {
 +            set_keyspace_result result = new set_keyspace_result();
 +            try {
 +              fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
 +              return;
 +            } catch (Exception e) {
 +              LOGGER.error("Exception writing to internal frame buffer", e);
 +            }
 +            fb.close();
 +          }
 +          public void onError(Exception e) {
 +            byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
 +            org.apache.thrift.TBase msg;
 +            set_keyspace_result result = new set_keyspace_result();
 +            if (e instanceof InvalidRequestException) {
 +                        result.ire = (InvalidRequestException) e;
 +                        result.setIreIsSet(true);
 +                        msg = result;
 +            }
 +             else 
 +            {
 +              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
 +              msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
 +            }
 +            try {
 +              fcall.sendResponse(fb,msg,msgType,seqid);
 +              return;
 +            } catch (Exception ex) {
 +              LOGGER.error("Exception writing to internal frame buffer", ex);
 +            }
 +            fb.close();
 +          }
 +        };
        }
  
 -      public short getThriftFieldId() {
 -        return _thriftId;
 +      protected boolean isOneway() {
 +        return false;
        }
  
 -      public String getFieldName() {
 -        return _fieldName;
 +      public void start(I iface, set_keyspace_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws TException {
 +        iface.set_keyspace(args.keyspace,resultHandler);
        }
      }
  
 -    // isset id assignments
 -
 -    public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
 -    static {
 -      Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
 -      tmpMap.put(_Fields.AUTH_REQUEST, new org.apache.thrift.meta_data.FieldMetaData("auth_request", org.apache.thrift.TFieldRequirementType.REQUIRED, 
 -          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, AuthenticationRequest.class)));
 -      metaDataMap = Collections.unmodifiableMap(tmpMap);
 -      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(login_args.class, metaDataMap);
 -    }
 +    public static class get<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, get_args, ColumnOrSuperColumn> {
 +      public get() {
 +        super("get");
 +      }
  
 -    public login_args() {
 -    }
 +      public get_args getEmptyArgsInstance() {
 +        return new get_args();
 +      }
  
 -    public login_args(
 -      AuthenticationRequest auth_request)
 -    {
 -      this();
 -      this.auth_request = auth_request;
 -    }
 +      public AsyncMethodCallback<ColumnOrSuperColumn> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
 +        final org.apache.thrift.AsyncProcessFunction fcall = this;
 +        return new AsyncMethodCallback<ColumnOrSuperColumn>() { 
 +          public void onComplete(ColumnOrSuperColumn o) {
 +            get_result result = new get_result();
 +            result.success = o;
 +            try {
 +              fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
 +              return;
 +            } catch (Exception e) {
 +              LOGGER.error("Exception writing to internal frame buffer", e);
 +            }
 +            fb.close();
 +          }
 +          public void onError(Exception e) {
 +            byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
 +            org.apache.thrift.TBase msg;
 +            get_result result = new get_result();
 +            if (e instanceof InvalidRequestException) {
 +                        result.ire = (InvalidRequestException) e;
 +                        result.setIreIsSet(true);
 +                        msg = result;
 +            }
 +            else             if (e instanceof NotFoundException) {
 +                        result.nfe = (NotFoundException) e;
 +                        result.setNfeIsSet(true);
 +                        msg = result;
 +            }
 +            else             if (e instanceof UnavailableException) {
 +                        result.ue = (UnavailableException) e;
 +                        result.setUeIsSet(true);
 +                        msg = result;
 +            }
 +            else             if (e instanceof TimedOutException) {
 +                        result.te = (TimedOutException) e;
 +                        result.setTeIsSet(true);
 +                        msg = result;
 +            }
 +             else 
 +            {
 +              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
 +              msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
 +            }
 +            try {
 +              fcall.sendResponse(fb,msg,msgType,seqid);
 +              return;
 +            } catch (Exception ex) {
 +              LOGGER.error("Exception writing to internal frame buffer", ex);
 +            }
 +            fb.close();
 +          }
 +        };
 +      }
  
 -    /**
 -     * Performs a deep copy on <i>other</i>.
 -     */
 -    public login_args(login_args other) {
 -      if (other.isSetAuth_request()) {
 -        this.auth_request = new AuthenticationRequest(other.auth_request);
 +      protected boolean isOneway() {
 +        return false;
        }
 -    }
  
 -    public login_args deepCopy() {
 -      return new login_args(this);
 +      public void start(I iface, get_args args, org.apache.thrift.async.AsyncMethodCallback<ColumnOrSuperColumn> resultHandler) throws TException {
 +        iface.get(args.key, args.column_path, args.consistency_level,resultHandler);
 +      }
      }
  
 -    @Override
 -    public void clear() {
 -      this.auth_request = null;
 -    }
 +    public static class get_slice<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, get_slice_args, List<ColumnOrSuperColumn>> {
 +      public get_slice() {
 +        super("get_slice");
 +      }
  
 -    public AuthenticationRequest getAuth_request() {
 -      return this.auth_request;
 -    }
 +      public get_slice_args getEmptyArgsInstance() {
 +        return new get_slice_args();
 +      }
  
 -    public login_args setAuth_request(AuthenticationRequest auth_request) {
 -      this.auth_request = auth_request;
 -      return this;
 -    }
 +      public AsyncMethodCallback<List<ColumnOrSuperColumn>> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
 +        final org.apache.thrift.AsyncProcessFunction fcall = this;
 +        return new AsyncMethodCallback<List<ColumnOrSuperColumn>>() { 
 +          public void onComplete(List<ColumnOrSuperColumn> o) {
 +            get_slice_result result = new get_slice_result();
 +            result.success = o;
 +            try {
 +              fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
 +              return;
 +            } catch (Exception e) {
 +              LOGGER.error("Exception writing to internal frame buffer", e);
 +            }
 +            fb.close();
 +          }
 +          public void onError(Exception e) {
 +            byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
 +            org.apache.thrift.TBase msg;
 +            get_slice_result result = new get_slice_result();
 +            if (e instanceof InvalidRequestException) {
 +                        result.ire = (InvalidRequestException) e;
 +                        result.setIreIsSet(true);
 +                        msg = result;
 +            }
 +            else             if (e instanceof UnavailableException) {
 +                        result.ue = (UnavailableException) e;
 +                        result.setUeIsSet(true);
 +                        msg = result;
 +            }
 +            else             if (e instanceof TimedOutException) {
 +                        result.te = (TimedOutException) e;
 +                        result.setTeIsSet(true);
 +                        msg = result;
 +            }
 +             else 
 +            {
 +              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
 +              msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
 +            }
 +            try {
 +              fcall.sendResponse(fb,msg,msgType,seqid);
 +              return;
 +            } catch (Exception ex) {
 +              LOGGER.error("Exception writing to internal frame buffer", ex);
 +            }
 +            fb.close();
 +          }
 +        };
 +      }
  
 -    public void unsetAuth_request() {
 -      this.auth_request = null;
 -    }
 +      protected boolean isOneway() {
 +        return false;
 +      }
  
 -    /** Returns true if field auth_request is set (has been assigned a value) and false otherwise */
 -    public boolean isSetAuth_request() {
 -      return this.auth_request != null;
 +      public void start(I iface, get_slice_args args, org.apache.thrift.async.AsyncMethodCallback<List<ColumnOrSuperColumn>> resultHandler) throws TException {
 +        iface.get_slice(args.key, args.column_parent, args.predicate, args.consistency_level,resultHandler);
 +      }
      }
  
 -    public void setAuth_requestIsSet(boolean value) {
 -      if (!value) {
 -        this.auth_request = null;
 +    public static class get_count<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, get_count_args, Integer> {
 +      public get_count() {
 +        super("get_count");
        }
 -    }
  
 -    public void setFieldValue(_Fields field, Object value) {
 -      switch (field) {
 -      case AUTH_REQUEST:
 -        if (value == null) {
 -          unsetAuth_request();
 -        } else {
 -          setAuth_request((AuthenticationRequest)value);
 -        }
 -        break;
 +      public get_count_args getEmptyArgsInstance() {
 +        return new get_count_args();
 +      }
  
 +      public AsyncMethodCallback<Integer> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
 +        final org.apache.thrift.AsyncProcessFunction fcall = this;
 +        return new AsyncMethodCallback<Integer>() { 
 +          public void onComplete(Integer o) {
 +            get_count_result result = new get_count_result();
 +            result.success = o;
 +            result.setSuccessIsSet(true);
 +            try {
 +              fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
 +              return;
 +            } catch (Exception e) {
 +              LOGGER.error("Exception writing to internal frame buffer", e);
 +            }
 +            fb.close();
 +          }
 +          public void onError(Exception e) {
 +            byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
 +            org.apache.thrift.TBase msg;
 +            get_count_result result = new get_count_result();
 +            if (e instanceof InvalidRequestException) {
 +                        result.ire = (InvalidRequestException) e;
 +                        result.setIreIsSet(true);
 +                        msg = result;
 +            }
 +            else             if (e instanceof UnavailableException) {
 +                        result.ue = (UnavailableException) e;
 +                        result.setUeIsSet(true);
 +                        msg = result;
 +            }
 +            else             if (e instanceof TimedOutException) {
 +                        result.te = (TimedOutException) e;
 +                        result.setTeIsSet(true);
 +                        msg = result;
 +            }
 +             else 
 +            {
 +              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
 +              msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
 +            }
 +            try {
 +              fcall.sendResponse(fb,msg,msgType,seqid);
 +              return;
 +            } catch (Exception ex) {
 +              LOGGER.error("Exception writing to internal frame buffer", ex);
 +            }
 +            fb.close();
 +          }
 +        };
        }
 -    }
  
 -    public Object getFieldValue(_Fields field) {
 -      switch (field) {
 -      case AUTH_REQUEST:
 -        return getAuth_request();
 +      protected boolean isOneway() {
 +        return false;
 +      }
  
 +      public void start(I iface, get_count_args args, org.apache.thrift.async.AsyncMethodCallback<Integer> resultHandler) throws TException {
 +        iface.get_count(args.key, args.column_parent, args.predicate, args.consistency_level,resultHandler);
        }
 -      throw new IllegalStateException();
      }
  
 -    /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
 -    public boolean isSet(_Fields field) {
 -      if (field == null) {
 -        throw new IllegalArgumentException();
 +    public static class multiget_slice<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, multiget_slice_args, Map<ByteBuffer,List<ColumnOrSuperColumn>>> {
 +      public multiget_slice() {
 +        super("multiget_slice");
        }
  
 -      switch (field) {
 -      case AUTH_REQUEST:
 -        return isSetAuth_request();
 +      public multiget_slice_args getEmptyArgsInstance() {
 +        return new multiget_slice_args();
        }
 -      throw new IllegalStateException();
 -    }
  
 -    @Override
 -    public boolean equals(Object that) {
 -      if (that == null)
 -        return false;
 -      if (that instanceof login_args)
 -        return this.equals((login_args)that);
 -      return false;
 -    }
 +      public AsyncMethodCallback<Map<ByteBuffer,List<ColumnOrSuperColumn>>> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
 +        final org.apache.thrift.AsyncProcessFunction fcall = this;
 +        return new AsyncMethodCallback<Map<ByteBuffer,List<ColumnOrSuperColumn>>>() { 
 +          public void onComplete(Map<ByteBuffer,List<ColumnOrSuperColumn>> o) {
 +            multiget_slice_result result = new multiget_slice_result();
 +            result.success = o;
 +            try {
 +              fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
 +              return;
 +            } catch (Exception e) {
 +              LOGGER.error("Exception writing to internal frame buffer", e);
 +            }
 +            fb.close();
 +          }
 +          public void onError(Exception e) {
 +            byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
 +            org.apache.thrift.TBase msg;
 +            multiget_slice_result result = new multiget_slice_result();
 +            if (e instanceof InvalidRequestException) {
 +                        result.ire = (InvalidRequestException) e;
 +                        result.setIreIsSet(true);
 +                        msg = result;
 +            }
 +            else             if (e instanceof UnavailableException) {
 +                        result.ue = (UnavailableException) e;
 +                        result.setUeIsSet(true);
 +                        msg = result;
 +            }
 +            else             if (e instanceof TimedOutException) {
 +                        result.te = (TimedOutException) e;
 +                        result.setTeIsSet(true);
 +                        msg = result;
 +            }
 +             else 
 +            {
 +              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
 +              msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
 +            }
 +            try {
 +              fcall.sendResponse(fb,msg,msgType,seqid);
 +              return;
 +            } catch (Exception ex) {
 +              LOGGER.error("Exception writing to internal frame buffer", ex);
 +            }
 +            fb.close();
 +          }
 +        };
 +      }
  
 -    public boolean equals(login_args that) {
 -      if (that == null)
 +      protected boolean isOneway() {
          return false;
 -
 -      boolean this_present_auth_request = true && this.isSetAuth_request();
 -      boolean that_present_auth_request = true && that.isSetAuth_request();
 -      if (this_present_auth_request || that_present_auth_request) {
 -        if (!(this_present_auth_request && that_present_auth_request))
 -          return false;
 -        if (!this.auth_request.equals(that.auth_request))
 -          return false;
        }
  
 -      return true;
 +      public void start(I iface, multiget_slice_args args, org.apache.thrift.async.AsyncMethodCallback<Map<ByteBuffer,List<ColumnOrSuperColumn>>> resultHandler) throws TException {
 +        iface.multiget_slice(args.keys, args.column_parent, args.predicate, args.consistency_level,resultHandler);
 +      }
      }
  
 -    @Override
 -    public int hashCode() {
 -      HashCodeBuilder builder = new HashCodeBuilder();
 +    public static class multiget_count<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, multiget_count_args, Map<ByteBuffer,Integer>> {
 +      public multiget_count() {
 +        super("multiget_count");
 +      }
  
 -      boolean present_auth_request = true && (isSetAuth_request());
 -      builder.append(present_auth_request);
 -      if (present_auth_request)
 +      public multiget_count_args getEmptyArgsInstance() {
 +        return new multiget_count_args();
 +      }
 +
 +      public AsyncMethodCallback<Map<ByteBuffer,Integer>> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
 +        final org.apache.thrift.AsyncProcessFunction fcall = this;
 +        return new AsyncMethodCallback<Map<ByteBuffer,Integer>>() { 
 +          public void onComplete(Map<ByteBuffer,Integer> o) {
 +            multiget_count_result result = new multiget_count_result();
 +            result.success = o;
 +            try {
 +              fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
 +              return;
 +            } catch (Exception e) {
 +              LOGGER.error("Exception writing to internal frame buffer", e);
 +            }
 +            fb.close();
 +          }
 +          public void onError(Exception e) {
 +            byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
 +            org.apache.thrift.TBase msg;
 +            multiget_count_result result = new multiget_count_result();
 +            if (e instanceof InvalidRequestException) {
 +                        result.ire = (InvalidRequestException) e;
 +                        result.setIreIsSet(true);
 +                        msg = result;
 +            }
 +            else             if (e instanceof UnavailableException) {
 +                        result.ue = (UnavailableException) e;
 +                        result.setUeIsSet(true);
 +                        msg = result;
 +            }
 +            else             if (e instanceof TimedOutException) {
 +                        result.te = (TimedOutException) e;
 +                        result.setTeIsSet(true);
 +                        msg = result;
 +            }
 +             else 
 +            {
 +              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
 +              msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
 +            }
 +            try {
 +              fcall.sendResponse(fb,msg,msgType,seqid);
 +              return;
 +            } catch (Exception ex) {
 +              LOGGER.error("Exception writing to internal frame buffer", ex);
 +            }
 +            fb.close();
 +          }
 +        };
 +      }
 +
 +      protected boolean isOneway() {
 +        return false;
 +      }
 +
 +      public void start(I iface, multiget_count_args args, org.apache.thrift.async.AsyncMethodCallback<Map<ByteBuffer,Integer>> resultHandler) throws TException {
 +        iface.multiget_count(args.keys, args.column_parent, args.predicate, args.consistency_level,resultHandler);
 +      }
 +    }
 +
 +    public static class get_range_slices<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, get_range_slices_args, List<KeySlice>> {
 +      public get_range_slices() {
 +        super("get_range_slices");
 +      }
 +
 +      public get_range_slices_args getEmptyArgsInstance() {
 +        return new get_range_slices_args();
 +      }
 +
 +      public AsyncMethodCallback<List<KeySlice>> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
 +        final org.apache.thrift.AsyncProcessFunction fcall = this;
 +        return new AsyncMethodCallback<List<KeySlice>>() { 
 +          public void onComplete(List<KeySlice> o) {
 +            get_range_slices_result result = new get_range_slices_result();
 +            result.success = o;
 +            try {
 +              fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
 +              return;
 +            } catch (Exception e) {
 +              LOGGER.error("Exception writing to internal frame buffer", e);
 +            }
 +            fb.close();
 +          }
 +          public void onError(Exception e) {
 +            byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
 +            org.apache.thrift.TBase msg;
 +            get_range_slices_result result = new get_range_slices_result();
 +            if (e instanceof InvalidRequestException) {
 +                        result.ire = (InvalidRequestException) e;
 +                        result.setIreIsSet(true);
 +                        msg = result;
 +            }
 +            else             if (e instanceof UnavailableException) {
 +                        result.ue = (UnavailableException) e;
 +                        result.setUeIsSet(true);
 +                        msg = result;
 +            }
 +            else             if (e instanceof TimedOutException) {
 +                        result.te = (TimedOutException) e;
 +                        result.setTeIsSet(true);
 +                        msg = result;
 +            }
 +             else 
 +            {
 +              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
 +              msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
 +            }
 +            try {
 +              fcall.sendResponse(fb,msg,msgType,seqid);
 +              return;
 +            } catch (Exception ex) {
 +              LOGGER.error("Exception writing to internal frame buffer", ex);
 +            }
 +            fb.close();
 +          }
 +        };
 +      }
 +
 +      protected boolean isOneway() {
 +        return false;
 +      }
 +
 +      public void start(I iface, get_range_slices_args args, org.apache.thrift.async.AsyncMethodCallback<List<KeySlice>> resultHandler) throws TException {
 +        iface.get_range_slices(args.column_parent, args.predicate, args.range, args.consistency_level,resultHandler);
 +      }
 +    }
 +
 +    public static class get_paged_slice<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, get_paged_slice_args, List<KeySlice>> {
 +      public get_paged_slice() {
 +        super("get_paged_slice");
 +      }
 +
 +      public get_paged_slice_args getEmptyArgsInstance() {
 +        return new get_paged_slice_args();
 +      }
 +
 +      public AsyncMethodCallback<List<KeySlice>> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
 +        final org.apache.thrift.AsyncProcessFunction fcall = this;
 +        return new AsyncMethodCallback<List<KeySlice>>() { 
 +          public void onComplete(List<KeySlice> o) {
 +            get_paged_slice_result result = new get_paged_slice_result();
 +            result.success = o;
 +            try {
 +              fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
 +              return;
 +            } catch (Exception e) {
 +              LOGGER.error("Exception writing to internal frame buffer", e);
 +            }
 +            fb.close();
 +          }
 +          public void onError(Exception e) {
 +            byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
 +            org.apache.thrift.TBase msg;
 +            get_paged_slice_result result = new get_paged_slice_result();
 +            if (e instanceof InvalidRequestException) {
 +                        result.ire = (InvalidRequestException) e;
 +                        result.setIreIsSet(true);
 +                        msg = result;
 +            }
 +            else             if (e instanceof UnavailableException) {
 +                        result.ue = (UnavailableException) e;
 +                        result.setUeIsSet(true);
 +                        msg = result;
 +            }
 +            else             if (e instanceof TimedOutException) {
 +                        result.te = (TimedOutException) e;
 +                        result.setTeIsSet(true);
 +                        msg = result;
 +            }
 +             else 
 +            {
 +              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
 +              msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
 +            }
 +            try {
 +              fcall.sendResponse(fb,msg,msgType,seqid);
 +              return;
 +            } catch (Exception ex) {
 +              LOGGER.error("Exception writing to internal frame buffer", ex);
 +            }
 +            fb.close();
 +          }
 +        };
 +      }
 +
 +      protected boolean isOneway() {
 +        return false;
 +      }
 +
 +      public void start(I iface, get_paged_slice_args args, org.apache.thrift.async.AsyncMethodCallback<List<KeySlice>> resultHandler) throws TException {
 +        iface.get_paged_slice(args.column_family, args.range, args.start_column, args.consistency_level,resultHandler);
 +      }
 +    }
 +
 +    public static class get_indexed_slices<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, get_indexed_slices_args, List<KeySlice>> {
 +      public get_indexed_slices() {
 +        super("get_indexed_slices");
 +      }
 +
 +      public get_indexed_slices_args getEmptyArgsInstance() {
 +        return new get_indexed_slices_args();
 +      }
 +
 +      public AsyncMethodCallback<List<KeySlice>> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
 +        final org.apache.thrift.AsyncProcessFunction fcall = this;
 +        return new AsyncMethodCallback<List<KeySlice>>() { 
 +          public void onComplete(List<KeySlice> o) {
 +            get_indexed_slices_result result = new get_indexed_slices_result();
 +            result.success = o;
 +            try {
 +              fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
 +              return;
 +            } catch (Exception e) {
 +              LOGGER.error("Exception writing to internal frame buffer", e);
 +            }
 +            fb.close();
 +          }
 +          public void onError(Exception e) {
 +            byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
 +            org.apache.thrift.TBase msg;
 +            get_indexed_slices_result result = new get_indexed_slices_result();
 +            if (e instanceof InvalidRequestException) {
 +                        result.ire = (InvalidRequestException) e;
 +                        result.setIreIsSet(true);
 +                        msg = result;
 +            }
 +            else             if (e instanceof UnavailableException) {
 +                        result.ue = (UnavailableException) e;
 +                        result.setUeIsSet(true);
 +                        msg = result;
 +            }
 +            else             if (e instanceof TimedOutException) {
 +                        result.te = (TimedOutException) e;
 +                        result.setTeIsSet(true);
 +                        msg = result;
 +            }
 +             else 
 +            {
 +              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
 +              msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
 +            }
 +            try {
 +              fcall.sendResponse(fb,msg,msgType,seqid);
 +              return;
 +            } catch (Exception ex) {
 +              LOGGER.error("Exception writing to internal frame buffer", ex);
 +            }
 +            fb.close();
 +          }
 +        };
 +      }
 +
 +      protected boolean isOneway() {
 +        return false;
 +      }
 +
 +      public void start(I iface, get_indexed_slices_args args, org.apache.thrift.async.AsyncMethodCallback<List<KeySlice>> resultHandler) throws TException {
 +        iface.get_indexed_slices(args.column_parent, args.index_clause, args.column_predicate, args.consistency_level,resultHandler);
 +      }
 +    }
 +
 +    public static class insert<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, insert_args, Void> {
 +      public insert() {
 +        super("insert");
 +      }
 +
 +      public insert_args getEmptyArgsInstance() {
 +        return new insert_args();
 +      }
 +
 +      public AsyncMethodCallback<Void> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
 +        final org.apache.thrift.AsyncProcessFunction fcall = this;
 +        return new AsyncMethodCallback<Void>() { 
 +          public void onComplete(Void o) {
 +            insert_result result = new insert_result();
 +            try {
 +              fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
 +              return;
 +            } catch (Exception e) {
 +              LOGGER.error("Exception writing to internal frame buffer", e);
 +            }
 +            fb.close();
 +          }
 +          public void onError(Exception e) {
 +            byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
 +            org.apache.thrift.TBase msg;
 +            insert_result result = new insert_result();
 +            if (e instanceof InvalidRequestException) {
 +                        result.ire = (InvalidRequestException) e;
 +                        result.setIreIsSet(true);
 +                        msg = result;
 +            }
 +            else             if (e instanceof UnavailableException) {
 +                        result.ue = (UnavailableException) e;
 +                        result.setUeIsSet(true);
 +                        msg = result;
 +            }
 +            else             if (e instanceof TimedOutException) {
 +                        result.te = (TimedOutException) e;
 +                        result.setTeIsSet(true);
 +                        msg = result;
 +            }
 +             else 
 +            {
 +              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
 +              msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
 +            }
 +            try {
 +              fcall.sendResponse(fb,msg,msgType,seqid);
 +              return;
 +            } catch (Exception ex) {
 +              LOGGER.error("Exception writing to internal frame buffer", ex);
 +            }
 +            fb.close();
 +          }
 +        };
 +      }
 +
 +      protected boolean isOneway() {
 +        return false;
 +      }
 +
 +      public void start(I iface, insert_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws TException {
 +        iface.insert(args.key, args.column_parent, args.column, args.consistency_level,resultHandler);
 +      }
 +    }
 +
 +    public static class add<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, add_args, Void> {
 +      public add() {
 +        super("add");
 +      }
 +
 +      public add_args getEmptyArgsInstance() {
 +        return new add_args();
 +      }
 +
 +      public AsyncMethodCallback<Void> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
 +        final org.apache.thrift.AsyncProcessFunction fcall = this;
 +        return new AsyncMethodCallback<Void>() { 
 +          public void onComplete(Void o) {
 +            add_result result = new add_result();
 +            try {
 +              fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
 +              return;
 +            } catch (Exception e) {
 +              LOGGER.error("Exception writing to internal frame buffer", e);
 +            }
 +            fb.close();
 +          }
 +          public void onError(Exception e) {
 +            byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
 +            org.apache.thrift.TBase msg;
 +            add_result result = new add_result();
 +            if (e instanceof InvalidRequestException) {
 +                        result.ire = (InvalidRequestException) e;
 +                        result.setIreIsSet(true);
 +                        msg = result;
 +            }
 +            else             if (e instanceof UnavailableException) {
 +                        result.ue = (UnavailableException) e;
 +                        result.setUeIsSet(true);
 +                        msg = result;
 +            }
 +            else             if (e instanceof TimedOutException) {
 +                        result.te = (TimedOutException) e;
 +                        result.setTeIsSet(true);
 +                        msg = result;
 +            }
 +             else 
 +            {
 +              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
 +              msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
 +            }
 +            try {
 +              fcall.sendResponse(fb,msg,msgType,seqid);
 +              return;
 +            } catch (Exception ex) {
 +              LOGGER.error("Exception writing to internal frame buffer", ex);
 +            }
 +            fb.close();
 +          }
 +        };
 +      }
 +
 +      protected boolean isOneway() {
 +        return false;
 +      }
 +
 +      public void start(I iface, add_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws TException {
 +        iface.add(args.key, args.column_parent, args.column, args.consistency_level,resultHandler);
 +      }
 +    }
 +
 +    public static class cas<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, cas_args, CASResult> {
 +      public cas() {
 +        super("cas");
 +      }
 +
 +      public cas_args getEmptyArgsInstance() {
 +        return new cas_args();
 +      }
 +
 +      public AsyncMethodCallback<CASResult> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
 +        final org.apache.thrift.AsyncProcessFunction fcall = this;
 +        return new AsyncMethodCallback<CASResult>() { 
 +          public void onComplete(CASResult o) {
 +            cas_result result = new cas_result();
 +            result.success = o;
 +            try {
 +              fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
 +              return;
 +            } catch (Exception e) {
 +              LOGGER.error("Exception writing to internal frame buffer", e);
 +            }
 +            fb.close();
 +          }
 +          public void onError(Exception e) {
 +            byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
 +            org.apache.thrift.TBase msg;
 +            cas_result result = new cas_result();
 +            if (e instanceof InvalidRequestException) {
 +                        result.ire = (InvalidRequestException) e;
 +                        result.setIreIsSet(true);
 +                        msg = result;
 +            }
 +            else             if (e instanceof UnavailableException) {
 +                        result.ue = (UnavailableException) e;
 +                        result.setUeIsSet(true);
 +                        msg = result;
 +            }
 +            else             if (e instanceof TimedOutException) {
 +                        result.te = (TimedOutException) e;
 +                        result.setTeIsSet(true);
 +                        msg = result;
 +            }
 +             else 
 +            {
 +              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
 +              msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
 +            }
 +            try {
 +              fcall.sendResponse(fb,msg,msgType,seqid);
 +              return;
 +            } catch (Exception ex) {
 +              LOGGER.error("Exception writing to internal frame buffer", ex);
 +            }
 +            fb.close();
 +          }
 +        };
 +      }
 +
 +      protected boolean isOneway() {
 +        return false;
 +      }
 +
 +      public void start(I iface, cas_args args, org.apache.thrift.async.AsyncMethodCallback<CASResult> resultHandler) throws TException {
 +        iface.cas(args.key, args.column_family, args.expected, args.updates, args.serial_consistency_level, args.commit_consistency_level,resultHandler);
 +      }
 +    }
 +
 +    public static class remove<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, remove_args, Void> {
 +      public remove() {
 +        super("remove");
 +      }
 +
 +      public remove_args getEmptyArgsInstance() {
 +        return new remove_args();
 +      }
 +
 +      public AsyncMethodCallback<Void> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
 +        final org.apache.thrift.AsyncProcessFunction fcall = this;
 +        return new AsyncMethodCallback<Void>() { 
 +          public void onComplete(Void o) {
 +            remove_result result = new remove_result();
 +            try {
 +              fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
 +              return;
 +            } catch (Exception e) {
 +              LOGGER.error("Exception writing to internal frame buffer", e);
 +            }
 +            fb.close();
 +          }
 +          public void onError(Exception e) {
 +            byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
 +            org.apache.thrift.TBase msg;
 +            remove_result result = new remove_result();
 +            if (e instanceof InvalidRequestException) {
 +                        result.ire = (InvalidRequestException) e;
 +                        result.setIreIsSet(true);
 +                        msg = result;
 +            }
 +            else             if (e instanceof UnavailableException) {
 +                        result.ue = (UnavailableException) e;
 +                        result.setUeIsSet(true);
 +                        msg = result;
 +            }
 +            else             if (e instanceof TimedOutException) {
 +                        result.te = (TimedOutException) e;
 +                        result.setTeIsSet(true);
 +                        msg = result;
 +            }
 +             else 
 +            {
 +              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
 +              msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
 +            }
 +            try {
 +              fcall.sendResponse(fb,msg,msgType,seqid);
 +              return;
 +            } catch (Exception ex) {
 +              LOGGER.error("Exception writing to internal frame buffer", ex);
 +            }
 +            fb.close();
 +          }
 +        };
 +      }
 +
 +      protected boolean isOneway() {
 +        return false;
 +      }
 +
 +      public void start(I iface, remove_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws TException {
 +        iface.remove(args.key, args.column_path, args.timestamp, args.consistency_level,resultHandler);
 +      }
 +    }
 +
 +    public static class remove_counter<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, remove_counter_args, Void> {
 +      public remove_counter() {
 +        super("remove_counter");
 +      }
 +
 +      public remove_counter_args getEmptyArgsInstance() {
 +        return new remove_counter_args();
 +      }
 +
 +      public AsyncMethodCallback<Void> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
 +        final org.apache.thrift.AsyncProcessFunction fcall = this;
 +        return new AsyncMethodCallback<Void>() { 
 +          public void onComplete(Void o) {
 +            remove_counter_result result = new remove_counter_result();
 +            try {
 +              fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
 +              return;
 +            } catch (Exception e) {
 +              LOGGER.error("Exception writing to internal frame buffer", e);
 +            }
 +            fb.close();
 +          }
 +          public void onError(Exception e) {
 +            byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
 +            org.apache.thrift.TBase msg;
 +            remove_counter_result result = new remove_counter_result();
 +            if (e instanceof InvalidRequestException) {
 +                        result.ire = (InvalidRequestException) e;
 +                        result.setIreIsSet(true);
 +                        msg = result;
 +            }
 +            else             if (e instanceof UnavailableException) {
 +                        result.ue = (UnavailableException) e;
 +                        result.setUeIsSet(true);
 +                        msg = result;
 +            }
 +            else             if (e instanceof TimedOutException) {
 +                        result.te = (TimedOutException) e;
 +                        result.setTeIsSet(true);
 +                        msg = result;
 +            }
 +             else 
 +            {
 +              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
 +              msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
 +            }
 +            try {
 +              fcall.sendResponse(fb,msg,msgType,seqid);
 +              return;
 +            } catch (Exception ex) {
 +              LOGGER.error("Exception writing to internal frame buffer", ex);
 +            }
 +            fb.close();
 +          }
 +        };
 +      }
 +
 +      protected boolean isOneway() {
 +        return false;
 +      }
 +
 +      public void start(I iface, remove_counter_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws TException {
 +        iface.remove_counter(args.key, args.path, args.consistency_level,resultHandler);
 +      }
 +    }
 +
 +    public static class batch_mutate<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, batch_mutate_args, Void> {
 +      public batch_mutate() {
 +        super("batch_mutate");
 +      }
 +
 +      public batch_mutate_args getEmptyArgsInstance() {
 +        return new batch_mutate_args();
 +      }
 +
 +      public AsyncMethodCallback<Void> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
 +        final org.apache.thrift.AsyncProcessFunction fcall = this;
 +        return new AsyncMethodCallback<Void>() { 
 +          public void onComplete(Void o) {
 +            batch_mutate_result result = new batch_mutate_result();
 +            try {
 +              fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
 +              return;
 +            } catch (Exception e) {
 +              LOGGER.error("Exception writing to internal frame buffer", e);
 +            }
 +            fb.close();
 +          }
 +          public void onError(Exception e) {
 +            byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
 +            org.apache.thrift.TBase msg;
 +            batch_mutate_result result = new batch_mutate_result();
 +            if (e instanceof InvalidRequestException) {
 +                        result.ire = (InvalidRequestException) e;
 +                        result.setIreIsSet(true);
 +                        msg = result;
 +            }
 +            else             if (e instanceof UnavailableException) {
 +                        result.ue = (UnavailableException) e;
 +                        result.setUeIsSet(true);
 +                        msg = result;
 +            }
 +            else             if (e instanceof TimedOutException) {
 +                        result.te = (TimedOutException) e;
 +                        result.setTeIsSet(true);
 +                        msg = result;
 +            }
 +             else 
 +            {
 +              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
 +              msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
 +            }
 +            try {
 +              fcall.sendResponse(fb,msg,msgType,seqid);
 +              return;
 +            } catch (Exception ex) {
 +              LOGGER.error("Exception writing to internal frame buffer", ex);
 +            }
 +            fb.close();
 +          }
 +        };
 +      }
 +
 +      protected boolean isOneway() {
 +        return false;
 +      }
 +
 +      public void start(I iface, batch_mutate_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws TException {
 +        iface.batch_mutate(args.mutation_map, args.consistency_level,resultHandler);
 +      }
 +    }
 +
 +    public static class atomic_batch_mutate<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, atomic_batch_mutate_args, Void> {
 +      public atomic_batch_mutate() {
 +        super("atomic_batch_mutate");
 +      }
 +
 +      public atomic_batch_mutate_args getEmptyArgsInstance() {
 +        return new atomic_batch_mutate_args();
 +      }
 +
 +      public AsyncMethodCallback<Void> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
 +        final org.apache.thrift.AsyncProcessFunction fcall = this;
 +        return new AsyncMethodCallback<Void>() { 
 +          public void onComplete(Void o) {
 +            atomic_batch_mutate_result result = new atomic_batch_mutate_result();
 +            try {
 +              fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
 +              return;
 +            } catch (Exception e) {
 +              LOGGER.error("Exception writing to internal frame buffer", e);
 +            }
 +            fb.close();
 +          }
 +          public void onError(Exception e) {
 +            byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
 +            org.apache.thrift.TBase msg;
 +            atomic_batch_mutate_result result = new atomic_batch_mutate_result();
 +            if (e instanceof InvalidRequestException) {
 +                        result.ire = (InvalidRequestException) e;
 +                        result.setIreIsSet(true);
 +                        msg = result;
 +            }
 +            else             if (e instanceof UnavailableException) {
 +                        result.ue = (UnavailableException) e;
 +                        result.setUeIsSet(true);
 +                        msg = result;
 +            }
 +            else             if (e instanceof TimedOutException) {
 +                        result.te = (TimedOutException) e;
 +                        result.setTeIsSet(true);
 +                        msg = result;
 +            }
 +             else 
 +            {
 +              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
 +              msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
 +            }
 +            try {
 +              fcall.sendResponse(fb,msg,msgType,seqid);
 +              return;
 +            } catch (Exception ex) {
 +              LOGGER.error("Exception writing to internal frame buffer", ex);
 +            }
 +            fb.close();
 +          }
 +        };
 +      }
 +
 +      protected boolean isOneway() {
 +        return false;
 +      }
 +
 +      public void start(I iface, atomic_batch_mutate_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws TException {
 +        iface.atomic_batch_mutate(args.mutation_map, args.consistency_level,resultHandler);
 +      }
 +    }
 +
 +    public static class truncate<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, truncate_args, Void> {
 +      public truncate() {
 +        super("truncate");
 +      }
 +
 +      public truncate_args getEmptyArgsInstance() {
 +        return new truncate_args();
 +      }
 +
 +      public AsyncMethodCallback<Void> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
 +        final org.apache.thrift.AsyncProcessFunction fcall = this;
 +        return new AsyncMethodCallback<Void>() { 
 +          public void onComplete(Void o) {
 +            truncate_result result = new truncate_result();
 +            try {
 +              fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
 +              return;
 +            } catch (Exception e) {
 +              LOGGER.error("Exception writing to internal frame buffer", e);
 +            }
 +            fb.close();
 +          }
 +          public void onError(Exception e) {
 +            byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
 +            org.apache.thrift.TBase msg;
 +            truncate_result result = new truncate_result();
 +            if (e instanceof InvalidRequestException) {
 +                        result.ire = (InvalidRequestException) e;
 +                        result.setIreIsSet(true);
 +                        msg = result;
 +            }
 +            else             if (e instanceof UnavailableException) {
 +                        result.ue = (UnavailableException) e;
 +                        result.setUeIsSet(true);
 +                        msg = result;
 +            }
 +            else             if (e instanceof TimedOutException) {
 +                        result.te = (TimedOutException) e;
 +                        result.setTeIsSet(true);
 +                        msg = result;
 +            }
 +             else 
 +            {
 +              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
 +              msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
 +            }
 +            try {
 +              fcall.sendResponse(fb,msg,msgType,seqid);
 +              return;
 +            } catch (Exception ex) {
 +              LOGGER.error("Exception writing to internal frame buffer", ex);
 +            }
 +            fb.close();
 +          }
 +        };
 +      }
 +
 +      protected boolean isOneway() {
 +        return false;
 +      }
 +
 +      public void start(I iface, truncate_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws TException {
 +        iface.truncate(args.cfname,resultHandler);
 +      }
 +    }
 +
 +    public static class describe_schema_versions<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, describe_schema_versions_args, Map<String,List<String>>> {
 +      public describe_schema_versions() {
 +        super("describe_schema_versions");
 +      }
 +
 +      public describe_schema_versions_args getEmptyArgsInstance() {
 +        return new describe_schema_versions_args();
 +      }
 +
 +      public AsyncMethodCallback<Map<String,List<String>>> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
 +        final org.apache.thrift.AsyncProcessFunction fcall = this;
 +        return new AsyncMethodCallback<Map<String,List<String>>>() { 
 +          public void onComplete(Map<String,List<String>> o) {
 +            describe_schema_versions_result result = new describe_schema_versions_result();
 +            result.success = o;
 +            try {
 +              fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
 +              return;
 +            } catch (Exception e) {
 +              LOGGER.error("Exception writing to internal frame buffer", e);
 +            }
 +            fb.close();
 +          }
 +          public void onError(Exception e) {
 +            byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
 +            org.apache.thrift.TBase msg;
 +            describe_schema_versions_result result = new describe_schema_versions_result();
 +            if (e instanceof InvalidRequestException) {
 +                        result.ire = (InvalidRequestException) e;
 +                        result.setIreIsSet(true);
 +                        msg = result;
 +            }
 +             else 
 +            {
 +              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
 +              msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
 +            }
 +            try {
 +              fcall.sendResponse(fb,msg,msgType,seqid);
 +              return;
 +            } catch (Exception ex) {
 +              LOGGER.error("Exception writing to internal frame buffer", ex);
 +            }
 +            fb.close();
 +          }
 +        };
 +      }
 +
 +      protected boolean isOneway() {
 +        return false;
 +      }
 +
 +      public void start(I iface, describe_schema_versions_args args, org.apache.thrift.async.AsyncMethodCallback<Map<String,List<String>>> resultHandler) throws TException {
 +        iface.describe_schema_versions(resultHandler);
 +      }
 +    }
 +
 +    public static class describe_keyspaces<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, describe_keyspaces_args, List<KsDef>> {
 +      public describe_keyspaces() {
 +        super("describe_keyspaces");
 +      }
 +
 +      public describe_keyspaces_args getEmptyArgsInstance() {
 +        return new describe_keyspaces_args();
 +      }
 +
 +      public AsyncMethodCallback<List<KsDef>> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
 +        final org.apache.thrift.AsyncProcessFunction fcall = this;
 +        return new AsyncMethodCallback<List<KsDef>>() { 
 +          public void onComplete(List<KsDef> o) {
 +            describe_keyspaces_result result = new describe_keyspaces_result();
 +            result.success = o;
 +            try {
 +              fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
 +              return;
 +            } catch (Exception e) {
 +              LOGGER.error("Exception writing to internal frame buffer", e);
 +            }
 +            fb.close();
 +          }
 +          public void onError(Exception e) {
 +            byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
 +            org.apache.thrift.TBase msg;
 +            describe_keyspaces_result result = new describe_keyspaces_result();
 +            if (e instanceof InvalidRequestException) {
 +                        result.ire = (InvalidRequestException) e;
 +                        result.setIreIsSet(true);
 +                        msg = result;
 +            }
 +             else 
 +            {
 +              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
 +              msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
 +            }
 +            try {
 +              fcall.sendResponse(fb,msg,msgType,seqid);
 +              return;
 +            } catch (Exception ex) {
 +              LOGGER.error("Exception writing to internal frame buffer", ex);
 +            }
 +            fb.close();
 +          }
 +        };
 +      }
 +
 +      protected boolean isOneway() {
 +        return false;
 +      }
 +
 +      publi

<TRUNCATED>

[02/16] add describe_local_ring for Hadoop's benefit patch by Piotr Kołaczkowski; reviewed by jbellis for CASSANDRA-6268

Posted by jb...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/86e949f3/interface/thrift/gen-java/org/apache/cassandra/thrift/Cassandra.java
----------------------------------------------------------------------
diff --git a/interface/thrift/gen-java/org/apache/cassandra/thrift/Cassandra.java b/interface/thrift/gen-java/org/apache/cassandra/thrift/Cassandra.java
index 7edbbd0..5c8d286 100644
--- a/interface/thrift/gen-java/org/apache/cassandra/thrift/Cassandra.java
+++ b/interface/thrift/gen-java/org/apache/cassandra/thrift/Cassandra.java
@@ -5,7 +5,7 @@
  */
 package org.apache.cassandra.thrift;
 /*
- *
+ * 
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -13,16 +13,16 @@ package org.apache.cassandra.thrift;
  * to you under the Apache License, Version 2.0 (the
  * "License"); you may not use this file except in compliance
  * with the License.  You may obtain a copy of the License at
- *
+ * 
  *   http://www.apache.org/licenses/LICENSE-2.0
- *
+ * 
  * Unless required by applicable law or agreed to in writing,
  * software distributed under the License is distributed on an
  * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
  * KIND, either express or implied.  See the License for the
  * specific language governing permissions and limitations
  * under the License.
- *
+ * 
  */
 
 
@@ -53,7 +53,7 @@ public class Cassandra {
     /**
      * Get the Column or SuperColumn at the given column_path. If no value is present, NotFoundException is thrown. (This is
      * the only method that can throw an exception under non-failure conditions.)
-     *
+     * 
      * @param key
      * @param column_path
      * @param consistency_level
@@ -63,7 +63,7 @@ public class Cassandra {
     /**
      * Get the group of columns contained by column_parent (either a ColumnFamily name or a ColumnFamily/SuperColumn name
      * pair) specified by the given SlicePredicate. If no matching values are found, an empty list is returned.
-     *
+     * 
      * @param key
      * @param column_parent
      * @param predicate
@@ -74,7 +74,7 @@ public class Cassandra {
     /**
      * returns the number of columns matching <code>predicate</code> for a particular <code>key</code>,
      * <code>ColumnFamily</code> and optionally <code>SuperColumn</code>.
-     *
+     * 
      * @param key
      * @param column_parent
      * @param predicate
@@ -84,7 +84,7 @@ public class Cassandra {
 
     /**
      * Performs a get_slice for column_parent and predicate for the given keys in parallel.
-     *
+     * 
      * @param keys
      * @param column_parent
      * @param predicate
@@ -94,7 +94,7 @@ public class Cassandra {
 
     /**
      * Perform a get_count in parallel on the given list<binary> keys. The return value maps keys to the count found.
-     *
+     * 
      * @param keys
      * @param column_parent
      * @param predicate
@@ -104,7 +104,7 @@ public class Cassandra {
 
     /**
      * returns a subset of columns for a contiguous range of keys.
-     *
+     * 
      * @param column_parent
      * @param predicate
      * @param range
@@ -114,7 +114,7 @@ public class Cassandra {
 
     /**
      * returns a range of columns, wrapping to the next rows if necessary to collect max_results.
-     *
+     * 
      * @param column_family
      * @param range
      * @param start_column
@@ -124,8 +124,8 @@ public class Cassandra {
 
     /**
      * Returns the subset of columns specified in SlicePredicate for the rows matching the IndexClause
-     * @deprecated; use get_range_slices instead with range.row_filter specified
-     *
+     * @deprecated use get_range_slices instead with range.row_filter specified
+     * 
      * @param column_parent
      * @param index_clause
      * @param column_predicate
@@ -135,7 +135,7 @@ public class Cassandra {
 
     /**
      * Insert a Column at the given column_parent.column_family and optional column_parent.super_column.
-     *
+     * 
      * @param key
      * @param column_parent
      * @param column
@@ -145,7 +145,7 @@ public class Cassandra {
 
     /**
      * Increment or decrement a counter.
-     *
+     * 
      * @param key
      * @param column_parent
      * @param column
@@ -157,7 +157,7 @@ public class Cassandra {
      * Remove data from the row specified by key at the granularity specified by column_path, and the given timestamp. Note
      * that all the values in column_path besides column_path.column_family are truly optional: you can remove the entire
      * row by just specifying the ColumnFamily, or you can remove a SuperColumn or a single Column by specifying those levels too.
-     *
+     * 
      * @param key
      * @param column_path
      * @param timestamp
@@ -169,7 +169,7 @@ public class Cassandra {
      * Remove a counter at the specified location.
      * Note that counters have limited support for deletes: if you remove a counter, you must wait to issue any following update
      * until the delete has reached all the nodes and all of them have been fully compacted.
-     *
+     * 
      * @param key
      * @param path
      * @param consistency_level
@@ -178,10 +178,10 @@ public class Cassandra {
 
     /**
      *   Mutate many columns or super columns for many row keys. See also: Mutation.
-     *
+     * 
      *   mutation_map maps key to column family to a list of Mutation objects to take place at that scope.
      * *
-     *
+     * 
      * @param mutation_map
      * @param consistency_level
      */
@@ -189,10 +189,10 @@ public class Cassandra {
 
     /**
      *   Atomically mutate many columns or super columns for many row keys. See also: Mutation.
-     *
+     * 
      *   mutation_map maps key to column family to a list of Mutation objects to take place at that scope.
      * *
-     *
+     * 
      * @param mutation_map
      * @param consistency_level
      */
@@ -205,7 +205,7 @@ public class Cassandra {
      * only marks the data as deleted.
      * The operation succeeds only if all hosts in the cluster at available and will throw an UnavailableException if
      * some hosts are down.
-     *
+     * 
      * @param cfname
      */
     public void truncate(String cfname) throws InvalidRequestException, UnavailableException, TimedOutException, org.apache.thrift.TException;
@@ -238,15 +238,22 @@ public class Cassandra {
      * to list of endpoints, because you can't use Thrift structs as
      * map keys:
      * https://issues.apache.org/jira/browse/THRIFT-162
-     *
+     * 
      * for the same reason, we can't return a set here, even though
      * order is neither important nor predictable.
-     *
+     * 
      * @param keyspace
      */
     public List<TokenRange> describe_ring(String keyspace) throws InvalidRequestException, org.apache.thrift.TException;
 
     /**
+     * same as describe_ring, but considers only nodes in the local DC
+     * 
+     * @param keyspace
+     */
+    public List<TokenRange> describe_local_ring(String keyspace) throws InvalidRequestException, org.apache.thrift.TException;
+
+    /**
      * get the mapping between token->node ip
      * without taking replication into consideration
      * https://issues.apache.org/jira/browse/CASSANDRA-4092
@@ -265,7 +272,7 @@ public class Cassandra {
 
     /**
      * describe specified keyspace
-     *
+     * 
      * @param keyspace
      */
     public KsDef describe_keyspace(String keyspace) throws NotFoundException, InvalidRequestException, org.apache.thrift.TException;
@@ -273,10 +280,10 @@ public class Cassandra {
     /**
      * experimental API for hadoop/parallel query support.
      * may change violently and without warning.
-     *
+     * 
      * returns list of token strings such that first subrange is (list[0], list[1]],
      * next is (list[1], list[2]], etc.
-     *
+     * 
      * @param cfName
      * @param start_token
      * @param end_token
@@ -294,42 +301,42 @@ public class Cassandra {
 
     /**
      * adds a column family. returns the new schema id.
-     *
+     * 
      * @param cf_def
      */
     public String system_add_column_family(CfDef cf_def) throws InvalidRequestException, SchemaDisagreementException, org.apache.thrift.TException;
 
     /**
      * drops a column family. returns the new schema id.
-     *
+     * 
      * @param column_family
      */
     public String system_drop_column_family(String column_family) throws InvalidRequestException, SchemaDisagreementException, org.apache.thrift.TException;
 
     /**
      * adds a keyspace and any column families that are part of it. returns the new schema id.
-     *
+     * 
      * @param ks_def
      */
     public String system_add_keyspace(KsDef ks_def) throws InvalidRequestException, SchemaDisagreementException, org.apache.thrift.TException;
 
     /**
      * drops a keyspace and any column families that are part of it. returns the new schema id.
-     *
+     * 
      * @param keyspace
      */
     public String system_drop_keyspace(String keyspace) throws InvalidRequestException, SchemaDisagreementException, org.apache.thrift.TException;
 
     /**
      * updates properties of a keyspace. returns the new schema id.
-     *
+     * 
      * @param ks_def
      */
     public String system_update_keyspace(KsDef ks_def) throws InvalidRequestException, SchemaDisagreementException, org.apache.thrift.TException;
 
     /**
      * updates properties of a column family. returns the new schema id.
-     *
+     * 
      * @param cf_def
      */
     public String system_update_column_family(CfDef cf_def) throws InvalidRequestException, SchemaDisagreementException, org.apache.thrift.TException;
@@ -337,7 +344,7 @@ public class Cassandra {
     /**
      * Executes a CQL (Cassandra Query Language) statement and returns a
      * CqlResult containing the results.
-     *
+     * 
      * @param query
      * @param compression
      */
@@ -350,7 +357,7 @@ public class Cassandra {
      * - the type of CQL statement
      * - an id token of the compiled CQL stored on the server side.
      * - a count of the discovered bound markers in the statement
-     *
+     * 
      * @param query
      * @param compression
      */
@@ -361,7 +368,7 @@ public class Cassandra {
     /**
      * Executes a prepared CQL (Cassandra Query Language) statement by passing an id token and  a list of variables
      * to bind and returns a CqlResult containing the results.
-     *
+     * 
      * @param itemId
      * @param values
      */
@@ -371,7 +378,7 @@ public class Cassandra {
 
     /**
      * @deprecated This is now a no-op. Please use the CQL3 specific methods instead.
-     *
+     * 
      * @param version
      */
     public void set_cql_version(String version) throws InvalidRequestException, org.apache.thrift.TException;
@@ -424,6 +431,8 @@ public class Cassandra {
 
     public void describe_ring(String keyspace, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.describe_ring_call> resultHandler) throws org.apache.thrift.TException;
 
+    public void describe_local_ring(String keyspace, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.describe_local_ring_call> resultHandler) throws org.apache.thrift.TException;
+
     public void describe_token_map(org.apache.thrift.async.AsyncMethodCallback<AsyncClient.describe_token_map_call> resultHandler) throws org.apache.thrift.TException;
 
     public void describe_partitioner(org.apache.thrift.async.AsyncMethodCallback<AsyncClient.describe_partitioner_call> resultHandler) throws org.apache.thrift.TException;
@@ -1153,6 +1162,32 @@ public class Cassandra {
       throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "describe_ring failed: unknown result");
     }
 
+    public List<TokenRange> describe_local_ring(String keyspace) throws InvalidRequestException, org.apache.thrift.TException
+    {
+      send_describe_local_ring(keyspace);
+      return recv_describe_local_ring();
+    }
+
+    public void send_describe_local_ring(String keyspace) throws org.apache.thrift.TException
+    {
+      describe_local_ring_args args = new describe_local_ring_args();
+      args.setKeyspace(keyspace);
+      sendBase("describe_local_ring", args);
+    }
+
+    public List<TokenRange> recv_describe_local_ring() throws InvalidRequestException, org.apache.thrift.TException
+    {
+      describe_local_ring_result result = new describe_local_ring_result();
+      receiveBase(result, "describe_local_ring");
+      if (result.isSetSuccess()) {
+        return result.success;
+      }
+      if (result.ire != null) {
+        throw result.ire;
+      }
+      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "describe_local_ring failed: unknown result");
+    }
+
     public Map<String,String> describe_token_map() throws InvalidRequestException, org.apache.thrift.TException
     {
       send_describe_token_map();
@@ -2546,6 +2581,38 @@ public class Cassandra {
       }
     }
 
+    public void describe_local_ring(String keyspace, org.apache.thrift.async.AsyncMethodCallback<describe_local_ring_call> resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      describe_local_ring_call method_call = new describe_local_ring_call(keyspace, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    public static class describe_local_ring_call extends org.apache.thrift.async.TAsyncMethodCall {
+      private String keyspace;
+      public describe_local_ring_call(String keyspace, org.apache.thrift.async.AsyncMethodCallback<describe_local_ring_call> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+        super(client, protocolFactory, transport, resultHandler, false);
+        this.keyspace = keyspace;
+      }
+
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("describe_local_ring", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        describe_local_ring_args args = new describe_local_ring_args();
+        args.setKeyspace(keyspace);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      public List<TokenRange> getResult() throws InvalidRequestException, org.apache.thrift.TException {
+        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+          throw new IllegalStateException("Method call not finished!");
+        }
+        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+        return (new Client(prot)).recv_describe_local_ring();
+      }
+    }
+
     public void describe_token_map(org.apache.thrift.async.AsyncMethodCallback<describe_token_map_call> resultHandler) throws org.apache.thrift.TException {
       checkReady();
       describe_token_map_call method_call = new describe_token_map_call(resultHandler, this, ___protocolFactory, ___transport);
@@ -3251,6 +3318,7 @@ public class Cassandra {
       processMap.put("describe_cluster_name", new describe_cluster_name());
       processMap.put("describe_version", new describe_version());
       processMap.put("describe_ring", new describe_ring());
+      processMap.put("describe_local_ring", new describe_local_ring());
       processMap.put("describe_token_map", new describe_token_map());
       processMap.put("describe_partitioner", new describe_partitioner());
       processMap.put("describe_snitch", new describe_snitch());
@@ -3771,6 +3839,26 @@ public class Cassandra {
       }
     }
 
+    private static class describe_local_ring<I extends Iface> extends org.apache.thrift.ProcessFunction<I, describe_local_ring_args> {
+      public describe_local_ring() {
+        super("describe_local_ring");
+      }
+
+      protected describe_local_ring_args getEmptyArgsInstance() {
+        return new describe_local_ring_args();
+      }
+
+      protected describe_local_ring_result getResult(I iface, describe_local_ring_args args) throws org.apache.thrift.TException {
+        describe_local_ring_result result = new describe_local_ring_result();
+        try {
+          result.success = iface.describe_local_ring(args.keyspace);
+        } catch (InvalidRequestException ire) {
+          result.ire = ire;
+        }
+        return result;
+      }
+    }
+
     private static class describe_token_map<I extends Iface> extends org.apache.thrift.ProcessFunction<I, describe_token_map_args> {
       public describe_token_map() {
         super("describe_token_map");
@@ -4269,7 +4357,7 @@ public class Cassandra {
     public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
     static {
       Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
-      tmpMap.put(_Fields.AUTH_REQUEST, new org.apache.thrift.meta_data.FieldMetaData("auth_request", org.apache.thrift.TFieldRequirementType.REQUIRED,
+      tmpMap.put(_Fields.AUTH_REQUEST, new org.apache.thrift.meta_data.FieldMetaData("auth_request", org.apache.thrift.TFieldRequirementType.REQUIRED, 
           new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, AuthenticationRequest.class)));
       metaDataMap = Collections.unmodifiableMap(tmpMap);
       org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(login_args.class, metaDataMap);
@@ -4430,7 +4518,7 @@ public class Cassandra {
       while (true)
       {
         field = iprot.readFieldBegin();
-        if (field.type == org.apache.thrift.protocol.TType.STOP) {
+        if (field.type == org.apache.thrift.protocol.TType.STOP) { 
           break;
         }
         switch (field.id) {
@@ -4438,7 +4526,7 @@ public class Cassandra {
             if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
               this.auth_request = new AuthenticationRequest();
               this.auth_request.read(iprot);
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -4582,9 +4670,9 @@ public class Cassandra {
     public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
     static {
       Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
-      tmpMap.put(_Fields.AUTHNX, new org.apache.thrift.meta_data.FieldMetaData("authnx", org.apache.thrift.TFieldRequirementType.DEFAULT,
+      tmpMap.put(_Fields.AUTHNX, new org.apache.thrift.meta_data.FieldMetaData("authnx", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT)));
-      tmpMap.put(_Fields.AUTHZX, new org.apache.thrift.meta_data.FieldMetaData("authzx", org.apache.thrift.TFieldRequirementType.DEFAULT,
+      tmpMap.put(_Fields.AUTHZX, new org.apache.thrift.meta_data.FieldMetaData("authzx", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT)));
       metaDataMap = Collections.unmodifiableMap(tmpMap);
       org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(login_result.class, metaDataMap);
@@ -4812,7 +4900,7 @@ public class Cassandra {
       while (true)
       {
         field = iprot.readFieldBegin();
-        if (field.type == org.apache.thrift.protocol.TType.STOP) {
+        if (field.type == org.apache.thrift.protocol.TType.STOP) { 
           break;
         }
         switch (field.id) {
@@ -4820,7 +4908,7 @@ public class Cassandra {
             if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
               this.authnx = new AuthenticationException();
               this.authnx.read(iprot);
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -4828,7 +4916,7 @@ public class Cassandra {
             if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
               this.authzx = new AuthorizationException();
               this.authzx.read(iprot);
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -4975,7 +5063,7 @@ public class Cassandra {
     public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
     static {
       Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
-      tmpMap.put(_Fields.KEYSPACE, new org.apache.thrift.meta_data.FieldMetaData("keyspace", org.apache.thrift.TFieldRequirementType.REQUIRED,
+      tmpMap.put(_Fields.KEYSPACE, new org.apache.thrift.meta_data.FieldMetaData("keyspace", org.apache.thrift.TFieldRequirementType.REQUIRED, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
       metaDataMap = Collections.unmodifiableMap(tmpMap);
       org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(set_keyspace_args.class, metaDataMap);
@@ -5136,14 +5224,14 @@ public class Cassandra {
       while (true)
       {
         field = iprot.readFieldBegin();
-        if (field.type == org.apache.thrift.protocol.TType.STOP) {
+        if (field.type == org.apache.thrift.protocol.TType.STOP) { 
           break;
         }
         switch (field.id) {
           case 1: // KEYSPACE
             if (field.type == org.apache.thrift.protocol.TType.STRING) {
               this.keyspace = iprot.readString();
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -5282,7 +5370,7 @@ public class Cassandra {
     public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
     static {
       Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
-      tmpMap.put(_Fields.IRE, new org.apache.thrift.meta_data.FieldMetaData("ire", org.apache.thrift.TFieldRequirementType.DEFAULT,
+      tmpMap.put(_Fields.IRE, new org.apache.thrift.meta_data.FieldMetaData("ire", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT)));
       metaDataMap = Collections.unmodifiableMap(tmpMap);
       org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(set_keyspace_result.class, metaDataMap);
@@ -5443,7 +5531,7 @@ public class Cassandra {
       while (true)
       {
         field = iprot.readFieldBegin();
-        if (field.type == org.apache.thrift.protocol.TType.STOP) {
+        if (field.type == org.apache.thrift.protocol.TType.STOP) { 
           break;
         }
         switch (field.id) {
@@ -5451,7 +5539,7 @@ public class Cassandra {
             if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
               this.ire = new InvalidRequestException();
               this.ire.read(iprot);
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -5526,7 +5614,7 @@ public class Cassandra {
     public ByteBuffer key; // required
     public ColumnPath column_path; // required
     /**
-     *
+     * 
      * @see ConsistencyLevel
      */
     public ConsistencyLevel consistency_level; // required
@@ -5536,7 +5624,7 @@ public class Cassandra {
       KEY((short)1, "key"),
       COLUMN_PATH((short)2, "column_path"),
       /**
-       *
+       * 
        * @see ConsistencyLevel
        */
       CONSISTENCY_LEVEL((short)3, "consistency_level");
@@ -5604,11 +5692,11 @@ public class Cassandra {
     public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
     static {
       Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
-      tmpMap.put(_Fields.KEY, new org.apache.thrift.meta_data.FieldMetaData("key", org.apache.thrift.TFieldRequirementType.REQUIRED,
+      tmpMap.put(_Fields.KEY, new org.apache.thrift.meta_data.FieldMetaData("key", org.apache.thrift.TFieldRequirementType.REQUIRED, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING          , true)));
-      tmpMap.put(_Fields.COLUMN_PATH, new org.apache.thrift.meta_data.FieldMetaData("column_path", org.apache.thrift.TFieldRequirementType.REQUIRED,
+      tmpMap.put(_Fields.COLUMN_PATH, new org.apache.thrift.meta_data.FieldMetaData("column_path", org.apache.thrift.TFieldRequirementType.REQUIRED, 
           new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ColumnPath.class)));
-      tmpMap.put(_Fields.CONSISTENCY_LEVEL, new org.apache.thrift.meta_data.FieldMetaData("consistency_level", org.apache.thrift.TFieldRequirementType.REQUIRED,
+      tmpMap.put(_Fields.CONSISTENCY_LEVEL, new org.apache.thrift.meta_data.FieldMetaData("consistency_level", org.apache.thrift.TFieldRequirementType.REQUIRED, 
           new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, ConsistencyLevel.class)));
       metaDataMap = Collections.unmodifiableMap(tmpMap);
       org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(get_args.class, metaDataMap);
@@ -5717,7 +5805,7 @@ public class Cassandra {
     }
 
     /**
-     *
+     * 
      * @see ConsistencyLevel
      */
     public ConsistencyLevel getConsistency_level() {
@@ -5725,7 +5813,7 @@ public class Cassandra {
     }
 
     /**
-     *
+     * 
      * @see ConsistencyLevel
      */
     public get_args setConsistency_level(ConsistencyLevel consistency_level) {
@@ -5925,14 +6013,14 @@ public class Cassandra {
       while (true)
       {
         field = iprot.readFieldBegin();
-        if (field.type == org.apache.thrift.protocol.TType.STOP) {
+        if (field.type == org.apache.thrift.protocol.TType.STOP) { 
           break;
         }
         switch (field.id) {
           case 1: // KEY
             if (field.type == org.apache.thrift.protocol.TType.STRING) {
               this.key = iprot.readBinary();
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -5940,14 +6028,14 @@ public class Cassandra {
             if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
               this.column_path = new ColumnPath();
               this.column_path.read(iprot);
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
           case 3: // CONSISTENCY_LEVEL
             if (field.type == org.apache.thrift.protocol.TType.I32) {
               this.consistency_level = ConsistencyLevel.findByValue(iprot.readI32());
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -6138,15 +6226,15 @@ public class Cassandra {
     public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
     static {
       Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
-      tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT,
+      tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ColumnOrSuperColumn.class)));
-      tmpMap.put(_Fields.IRE, new org.apache.thrift.meta_data.FieldMetaData("ire", org.apache.thrift.TFieldRequirementType.DEFAULT,
+      tmpMap.put(_Fields.IRE, new org.apache.thrift.meta_data.FieldMetaData("ire", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT)));
-      tmpMap.put(_Fields.NFE, new org.apache.thrift.meta_data.FieldMetaData("nfe", org.apache.thrift.TFieldRequirementType.DEFAULT,
+      tmpMap.put(_Fields.NFE, new org.apache.thrift.meta_data.FieldMetaData("nfe", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT)));
-      tmpMap.put(_Fields.UE, new org.apache.thrift.meta_data.FieldMetaData("ue", org.apache.thrift.TFieldRequirementType.DEFAULT,
+      tmpMap.put(_Fields.UE, new org.apache.thrift.meta_data.FieldMetaData("ue", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT)));
-      tmpMap.put(_Fields.TE, new org.apache.thrift.meta_data.FieldMetaData("te", org.apache.thrift.TFieldRequirementType.DEFAULT,
+      tmpMap.put(_Fields.TE, new org.apache.thrift.meta_data.FieldMetaData("te", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT)));
       metaDataMap = Collections.unmodifiableMap(tmpMap);
       org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(get_result.class, metaDataMap);
@@ -6575,7 +6663,7 @@ public class Cassandra {
       while (true)
       {
         field = iprot.readFieldBegin();
-        if (field.type == org.apache.thrift.protocol.TType.STOP) {
+        if (field.type == org.apache.thrift.protocol.TType.STOP) { 
           break;
         }
         switch (field.id) {
@@ -6583,7 +6671,7 @@ public class Cassandra {
             if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
               this.success = new ColumnOrSuperColumn();
               this.success.read(iprot);
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -6591,7 +6679,7 @@ public class Cassandra {
             if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
               this.ire = new InvalidRequestException();
               this.ire.read(iprot);
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -6599,7 +6687,7 @@ public class Cassandra {
             if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
               this.nfe = new NotFoundException();
               this.nfe.read(iprot);
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -6607,7 +6695,7 @@ public class Cassandra {
             if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
               this.ue = new UnavailableException();
               this.ue.read(iprot);
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -6615,7 +6703,7 @@ public class Cassandra {
             if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
               this.te = new TimedOutException();
               this.te.read(iprot);
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -6740,7 +6828,7 @@ public class Cassandra {
     public ColumnParent column_parent; // required
     public SlicePredicate predicate; // required
     /**
-     *
+     * 
      * @see ConsistencyLevel
      */
     public ConsistencyLevel consistency_level; // required
@@ -6751,7 +6839,7 @@ public class Cassandra {
       COLUMN_PARENT((short)2, "column_parent"),
       PREDICATE((short)3, "predicate"),
       /**
-       *
+       * 
        * @see ConsistencyLevel
        */
       CONSISTENCY_LEVEL((short)4, "consistency_level");
@@ -6821,13 +6909,13 @@ public class Cassandra {
     public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
     static {
       Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
-      tmpMap.put(_Fields.KEY, new org.apache.thrift.meta_data.FieldMetaData("key", org.apache.thrift.TFieldRequirementType.REQUIRED,
+      tmpMap.put(_Fields.KEY, new org.apache.thrift.meta_data.FieldMetaData("key", org.apache.thrift.TFieldRequirementType.REQUIRED, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING          , true)));
-      tmpMap.put(_Fields.COLUMN_PARENT, new org.apache.thrift.meta_data.FieldMetaData("column_parent", org.apache.thrift.TFieldRequirementType.REQUIRED,
+      tmpMap.put(_Fields.COLUMN_PARENT, new org.apache.thrift.meta_data.FieldMetaData("column_parent", org.apache.thrift.TFieldRequirementType.REQUIRED, 
           new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ColumnParent.class)));
-      tmpMap.put(_Fields.PREDICATE, new org.apache.thrift.meta_data.FieldMetaData("predicate", org.apache.thrift.TFieldRequirementType.REQUIRED,
+      tmpMap.put(_Fields.PREDICATE, new org.apache.thrift.meta_data.FieldMetaData("predicate", org.apache.thrift.TFieldRequirementType.REQUIRED, 
           new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, SlicePredicate.class)));
-      tmpMap.put(_Fields.CONSISTENCY_LEVEL, new org.apache.thrift.meta_data.FieldMetaData("consistency_level", org.apache.thrift.TFieldRequirementType.REQUIRED,
+      tmpMap.put(_Fields.CONSISTENCY_LEVEL, new org.apache.thrift.meta_data.FieldMetaData("consistency_level", org.apache.thrift.TFieldRequirementType.REQUIRED, 
           new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, ConsistencyLevel.class)));
       metaDataMap = Collections.unmodifiableMap(tmpMap);
       org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(get_slice_args.class, metaDataMap);
@@ -6966,7 +7054,7 @@ public class Cassandra {
     }
 
     /**
-     *
+     * 
      * @see ConsistencyLevel
      */
     public ConsistencyLevel getConsistency_level() {
@@ -6974,7 +7062,7 @@ public class Cassandra {
     }
 
     /**
-     *
+     * 
      * @see ConsistencyLevel
      */
     public get_slice_args setConsistency_level(ConsistencyLevel consistency_level) {
@@ -7211,14 +7299,14 @@ public class Cassandra {
       while (true)
       {
         field = iprot.readFieldBegin();
-        if (field.type == org.apache.thrift.protocol.TType.STOP) {
+        if (field.type == org.apache.thrift.protocol.TType.STOP) { 
           break;
         }
         switch (field.id) {
           case 1: // KEY
             if (field.type == org.apache.thrift.protocol.TType.STRING) {
               this.key = iprot.readBinary();
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -7226,7 +7314,7 @@ public class Cassandra {
             if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
               this.column_parent = new ColumnParent();
               this.column_parent.read(iprot);
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -7234,14 +7322,14 @@ public class Cassandra {
             if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
               this.predicate = new SlicePredicate();
               this.predicate.read(iprot);
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
           case 4: // CONSISTENCY_LEVEL
             if (field.type == org.apache.thrift.protocol.TType.I32) {
               this.consistency_level = ConsistencyLevel.findByValue(iprot.readI32());
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -7443,14 +7531,14 @@ public class Cassandra {
     public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
     static {
       Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
-      tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT,
-          new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST,
+      tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
               new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ColumnOrSuperColumn.class))));
-      tmpMap.put(_Fields.IRE, new org.apache.thrift.meta_data.FieldMetaData("ire", org.apache.thrift.TFieldRequirementType.DEFAULT,
+      tmpMap.put(_Fields.IRE, new org.apache.thrift.meta_data.FieldMetaData("ire", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT)));
-      tmpMap.put(_Fields.UE, new org.apache.thrift.meta_data.FieldMetaData("ue", org.apache.thrift.TFieldRequirementType.DEFAULT,
+      tmpMap.put(_Fields.UE, new org.apache.thrift.meta_data.FieldMetaData("ue", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT)));
-      tmpMap.put(_Fields.TE, new org.apache.thrift.meta_data.FieldMetaData("te", org.apache.thrift.TFieldRequirementType.DEFAULT,
+      tmpMap.put(_Fields.TE, new org.apache.thrift.meta_data.FieldMetaData("te", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT)));
       metaDataMap = Collections.unmodifiableMap(tmpMap);
       org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(get_slice_result.class, metaDataMap);
@@ -7831,7 +7919,7 @@ public class Cassandra {
       while (true)
       {
         field = iprot.readFieldBegin();
-        if (field.type == org.apache.thrift.protocol.TType.STOP) {
+        if (field.type == org.apache.thrift.protocol.TType.STOP) { 
           break;
         }
         switch (field.id) {
@@ -7849,7 +7937,7 @@ public class Cassandra {
                 }
                 iprot.readListEnd();
               }
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -7857,7 +7945,7 @@ public class Cassandra {
             if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
               this.ire = new InvalidRequestException();
               this.ire.read(iprot);
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -7865,7 +7953,7 @@ public class Cassandra {
             if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
               this.ue = new UnavailableException();
               this.ue.read(iprot);
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -7873,7 +7961,7 @@ public class Cassandra {
             if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
               this.te = new TimedOutException();
               this.te.read(iprot);
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -7993,7 +8081,7 @@ public class Cassandra {
     public ColumnParent column_parent; // required
     public SlicePredicate predicate; // required
     /**
-     *
+     * 
      * @see ConsistencyLevel
      */
     public ConsistencyLevel consistency_level; // required
@@ -8004,7 +8092,7 @@ public class Cassandra {
       COLUMN_PARENT((short)2, "column_parent"),
       PREDICATE((short)3, "predicate"),
       /**
-       *
+       * 
        * @see ConsistencyLevel
        */
       CONSISTENCY_LEVEL((short)4, "consistency_level");
@@ -8074,13 +8162,13 @@ public class Cassandra {
     public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
     static {
       Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
-      tmpMap.put(_Fields.KEY, new org.apache.thrift.meta_data.FieldMetaData("key", org.apache.thrift.TFieldRequirementType.REQUIRED,
+      tmpMap.put(_Fields.KEY, new org.apache.thrift.meta_data.FieldMetaData("key", org.apache.thrift.TFieldRequirementType.REQUIRED, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING          , true)));
-      tmpMap.put(_Fields.COLUMN_PARENT, new org.apache.thrift.meta_data.FieldMetaData("column_parent", org.apache.thrift.TFieldRequirementType.REQUIRED,
+      tmpMap.put(_Fields.COLUMN_PARENT, new org.apache.thrift.meta_data.FieldMetaData("column_parent", org.apache.thrift.TFieldRequirementType.REQUIRED, 
           new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ColumnParent.class)));
-      tmpMap.put(_Fields.PREDICATE, new org.apache.thrift.meta_data.FieldMetaData("predicate", org.apache.thrift.TFieldRequirementType.REQUIRED,
+      tmpMap.put(_Fields.PREDICATE, new org.apache.thrift.meta_data.FieldMetaData("predicate", org.apache.thrift.TFieldRequirementType.REQUIRED, 
           new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, SlicePredicate.class)));
-      tmpMap.put(_Fields.CONSISTENCY_LEVEL, new org.apache.thrift.meta_data.FieldMetaData("consistency_level", org.apache.thrift.TFieldRequirementType.REQUIRED,
+      tmpMap.put(_Fields.CONSISTENCY_LEVEL, new org.apache.thrift.meta_data.FieldMetaData("consistency_level", org.apache.thrift.TFieldRequirementType.REQUIRED, 
           new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, ConsistencyLevel.class)));
       metaDataMap = Collections.unmodifiableMap(tmpMap);
       org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(get_count_args.class, metaDataMap);
@@ -8219,7 +8307,7 @@ public class Cassandra {
     }
 
     /**
-     *
+     * 
      * @see ConsistencyLevel
      */
     public ConsistencyLevel getConsistency_level() {
@@ -8227,7 +8315,7 @@ public class Cassandra {
     }
 
     /**
-     *
+     * 
      * @see ConsistencyLevel
      */
     public get_count_args setConsistency_level(ConsistencyLevel consistency_level) {
@@ -8464,14 +8552,14 @@ public class Cassandra {
       while (true)
       {
         field = iprot.readFieldBegin();
-        if (field.type == org.apache.thrift.protocol.TType.STOP) {
+        if (field.type == org.apache.thrift.protocol.TType.STOP) { 
           break;
         }
         switch (field.id) {
           case 1: // KEY
             if (field.type == org.apache.thrift.protocol.TType.STRING) {
               this.key = iprot.readBinary();
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -8479,7 +8567,7 @@ public class Cassandra {
             if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
               this.column_parent = new ColumnParent();
               this.column_parent.read(iprot);
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -8487,14 +8575,14 @@ public class Cassandra {
             if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
               this.predicate = new SlicePredicate();
               this.predicate.read(iprot);
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
           case 4: // CONSISTENCY_LEVEL
             if (field.type == org.apache.thrift.protocol.TType.I32) {
               this.consistency_level = ConsistencyLevel.findByValue(iprot.readI32());
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -8698,13 +8786,13 @@ public class Cassandra {
     public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
     static {
       Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
-      tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT,
+      tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
-      tmpMap.put(_Fields.IRE, new org.apache.thrift.meta_data.FieldMetaData("ire", org.apache.thrift.TFieldRequirementType.DEFAULT,
+      tmpMap.put(_Fields.IRE, new org.apache.thrift.meta_data.FieldMetaData("ire", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT)));
-      tmpMap.put(_Fields.UE, new org.apache.thrift.meta_data.FieldMetaData("ue", org.apache.thrift.TFieldRequirementType.DEFAULT,
+      tmpMap.put(_Fields.UE, new org.apache.thrift.meta_data.FieldMetaData("ue", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT)));
-      tmpMap.put(_Fields.TE, new org.apache.thrift.meta_data.FieldMetaData("te", org.apache.thrift.TFieldRequirementType.DEFAULT,
+      tmpMap.put(_Fields.TE, new org.apache.thrift.meta_data.FieldMetaData("te", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT)));
       metaDataMap = Collections.unmodifiableMap(tmpMap);
       org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(get_count_result.class, metaDataMap);
@@ -9067,7 +9155,7 @@ public class Cassandra {
       while (true)
       {
         field = iprot.readFieldBegin();
-        if (field.type == org.apache.thrift.protocol.TType.STOP) {
+        if (field.type == org.apache.thrift.protocol.TType.STOP) { 
           break;
         }
         switch (field.id) {
@@ -9075,7 +9163,7 @@ public class Cassandra {
             if (field.type == org.apache.thrift.protocol.TType.I32) {
               this.success = iprot.readI32();
               setSuccessIsSet(true);
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -9083,7 +9171,7 @@ public class Cassandra {
             if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
               this.ire = new InvalidRequestException();
               this.ire.read(iprot);
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -9091,7 +9179,7 @@ public class Cassandra {
             if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
               this.ue = new UnavailableException();
               this.ue.read(iprot);
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -9099,7 +9187,7 @@ public class Cassandra {
             if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
               this.te = new TimedOutException();
               this.te.read(iprot);
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -9208,7 +9296,7 @@ public class Cassandra {
     public ColumnParent column_parent; // required
     public SlicePredicate predicate; // required
     /**
-     *
+     * 
      * @see ConsistencyLevel
      */
     public ConsistencyLevel consistency_level; // required
@@ -9219,7 +9307,7 @@ public class Cassandra {
       COLUMN_PARENT((short)2, "column_parent"),
       PREDICATE((short)3, "predicate"),
       /**
-       *
+       * 
        * @see ConsistencyLevel
        */
       CONSISTENCY_LEVEL((short)4, "consistency_level");
@@ -9289,14 +9377,14 @@ public class Cassandra {
     public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
     static {
       Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
-      tmpMap.put(_Fields.KEYS, new org.apache.thrift.meta_data.FieldMetaData("keys", org.apache.thrift.TFieldRequirementType.REQUIRED,
-          new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST,
+      tmpMap.put(_Fields.KEYS, new org.apache.thrift.meta_data.FieldMetaData("keys", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+          new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
               new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING              , true))));
-      tmpMap.put(_Fields.COLUMN_PARENT, new org.apache.thrift.meta_data.FieldMetaData("column_parent", org.apache.thrift.TFieldRequirementType.REQUIRED,
+      tmpMap.put(_Fields.COLUMN_PARENT, new org.apache.thrift.meta_data.FieldMetaData("column_parent", org.apache.thrift.TFieldRequirementType.REQUIRED, 
           new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ColumnParent.class)));
-      tmpMap.put(_Fields.PREDICATE, new org.apache.thrift.meta_data.FieldMetaData("predicate", org.apache.thrift.TFieldRequirementType.REQUIRED,
+      tmpMap.put(_Fields.PREDICATE, new org.apache.thrift.meta_data.FieldMetaData("predicate", org.apache.thrift.TFieldRequirementType.REQUIRED, 
           new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, SlicePredicate.class)));
-      tmpMap.put(_Fields.CONSISTENCY_LEVEL, new org.apache.thrift.meta_data.FieldMetaData("consistency_level", org.apache.thrift.TFieldRequirementType.REQUIRED,
+      tmpMap.put(_Fields.CONSISTENCY_LEVEL, new org.apache.thrift.meta_data.FieldMetaData("consistency_level", org.apache.thrift.TFieldRequirementType.REQUIRED, 
           new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, ConsistencyLevel.class)));
       metaDataMap = Collections.unmodifiableMap(tmpMap);
       org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(multiget_slice_args.class, metaDataMap);
@@ -9445,7 +9533,7 @@ public class Cassandra {
     }
 
     /**
-     *
+     * 
      * @see ConsistencyLevel
      */
     public ConsistencyLevel getConsistency_level() {
@@ -9453,7 +9541,7 @@ public class Cassandra {
     }
 
     /**
-     *
+     * 
      * @see ConsistencyLevel
      */
     public multiget_slice_args setConsistency_level(ConsistencyLevel consistency_level) {
@@ -9690,7 +9778,7 @@ public class Cassandra {
       while (true)
       {
         field = iprot.readFieldBegin();
-        if (field.type == org.apache.thrift.protocol.TType.STOP) {
+        if (field.type == org.apache.thrift.protocol.TType.STOP) { 
           break;
         }
         switch (field.id) {
@@ -9707,7 +9795,7 @@ public class Cassandra {
                 }
                 iprot.readListEnd();
               }
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -9715,7 +9803,7 @@ public class Cassandra {
             if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
               this.column_parent = new ColumnParent();
               this.column_parent.read(iprot);
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -9723,14 +9811,14 @@ public class Cassandra {
             if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
               this.predicate = new SlicePredicate();
               this.predicate.read(iprot);
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
           case 4: // CONSISTENCY_LEVEL
             if (field.type == org.apache.thrift.protocol.TType.I32) {
               this.consistency_level = ConsistencyLevel.findByValue(iprot.readI32());
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -9939,16 +10027,16 @@ public class Cassandra {
     public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
     static {
       Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
-      tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT,
-          new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP,
-              new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING              , true),
-              new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST,
+      tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, 
+              new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING              , true), 
+              new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
                   new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ColumnOrSuperColumn.class)))));
-      tmpMap.put(_Fields.IRE, new org.apache.thrift.meta_data.FieldMetaData("ire", org.apache.thrift.TFieldRequirementType.DEFAULT,
+      tmpMap.put(_Fields.IRE, new org.apache.thrift.meta_data.FieldMetaData("ire", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT)));
-      tmpMap.put(_Fields.UE, new org.apache.thrift.meta_data.FieldMetaData("ue", org.apache.thrift.TFieldRequirementType.DEFAULT,
+      tmpMap.put(_Fields.UE, new org.apache.thrift.meta_data.FieldMetaData("ue", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT)));
-      tmpMap.put(_Fields.TE, new org.apache.thrift.meta_data.FieldMetaData("te", org.apache.thrift.TFieldRequirementType.DEFAULT,
+      tmpMap.put(_Fields.TE, new org.apache.thrift.meta_data.FieldMetaData("te", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT)));
       metaDataMap = Collections.unmodifiableMap(tmpMap);
       org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(multiget_slice_result.class, metaDataMap);
@@ -10337,7 +10425,7 @@ public class Cassandra {
       while (true)
       {
         field = iprot.readFieldBegin();
-        if (field.type == org.apache.thrift.protocol.TType.STOP) {
+        if (field.type == org.apache.thrift.protocol.TType.STOP) { 
           break;
         }
         switch (field.id) {
@@ -10367,7 +10455,7 @@ public class Cassandra {
                 }
                 iprot.readMapEnd();
               }
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -10375,7 +10463,7 @@ public class Cassandra {
             if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
               this.ire = new InvalidRequestException();
               this.ire.read(iprot);
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -10383,7 +10471,7 @@ public class Cassandra {
             if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
               this.ue = new UnavailableException();
               this.ue.read(iprot);
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -10391,7 +10479,7 @@ public class Cassandra {
             if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
               this.te = new TimedOutException();
               this.te.read(iprot);
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -10519,7 +10607,7 @@ public class Cassandra {
     public ColumnParent column_parent; // required
     public SlicePredicate predicate; // required
     /**
-     *
+     * 
      * @see ConsistencyLevel
      */
     public ConsistencyLevel consistency_level; // required
@@ -10530,7 +10618,7 @@ public class Cassandra {
       COLUMN_PARENT((short)2, "column_parent"),
       PREDICATE((short)3, "predicate"),
       /**
-       *
+       * 
        * @see ConsistencyLevel
        */
       CONSISTENCY_LEVEL((short)4, "consistency_level");
@@ -10600,14 +10688,14 @@ public class Cassandra {
     public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
     static {
       Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
-      tmpMap.put(_Fields.KEYS, new org.apache.thrift.meta_data.FieldMetaData("keys", org.apache.thrift.TFieldRequirementType.REQUIRED,
-          new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST,
+      tmpMap.put(_Fields.KEYS, new org.apache.thrift.meta_data.FieldMetaData("keys", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+          new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
               new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING              , true))));
-      tmpMap.put(_Fields.COLUMN_PARENT, new org.apache.thrift.meta_data.FieldMetaData("column_parent", org.apache.thrift.TFieldRequirementType.REQUIRED,
+      tmpMap.put(_Fields.COLUMN_PARENT, new org.apache.thrift.meta_data.FieldMetaData("column_parent", org.apache.thrift.TFieldRequirementType.REQUIRED, 
           new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ColumnParent.class)));
-      tmpMap.put(_Fields.PREDICATE, new org.apache.thrift.meta_data.FieldMetaData("predicate", org.apache.thrift.TFieldRequirementType.REQUIRED,
+      tmpMap.put(_Fields.PREDICATE, new org.apache.thrift.meta_data.FieldMetaData("predicate", org.apache.thrift.TFieldRequirementType.REQUIRED, 
           new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, SlicePredicate.class)));
-      tmpMap.put(_Fields.CONSISTENCY_LEVEL, new org.apache.thrift.meta_data.FieldMetaData("consistency_level", org.apache.thrift.TFieldRequirementType.REQUIRED,
+      tmpMap.put(_Fields.CONSISTENCY_LEVEL, new org.apache.thrift.meta_data.FieldMetaData("consistency_level", org.apache.thrift.TFieldRequirementType.REQUIRED, 
           new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, ConsistencyLevel.class)));
       metaDataMap = Collections.unmodifiableMap(tmpMap);
       org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(multiget_count_args.class, metaDataMap);
@@ -10756,7 +10844,7 @@ public class Cassandra {
     }
 
     /**
-     *
+     * 
      * @see ConsistencyLevel
      */
     public ConsistencyLevel getConsistency_level() {
@@ -10764,7 +10852,7 @@ public class Cassandra {
     }
 
     /**
-     *
+     * 
      * @see ConsistencyLevel
      */
     public multiget_count_args setConsistency_level(ConsistencyLevel consistency_level) {
@@ -11001,7 +11089,7 @@ public class Cassandra {
       while (true)
       {
         field = iprot.readFieldBegin();
-        if (field.type == org.apache.thrift.protocol.TType.STOP) {
+        if (field.type == org.apache.thrift.protocol.TType.STOP) { 
           break;
         }
         switch (field.id) {
@@ -11018,7 +11106,7 @@ public class Cassandra {
                 }
                 iprot.readListEnd();
               }
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -11026,7 +11114,7 @@ public class Cassandra {
             if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
               this.column_parent = new ColumnParent();
               this.column_parent.read(iprot);
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -11034,14 +11122,14 @@ public class Cassandra {
             if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
               this.predicate = new SlicePredicate();
               this.predicate.read(iprot);
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
           case 4: // CONSISTENCY_LEVEL
             if (field.type == org.apache.thrift.protocol.TType.I32) {
               this.consistency_level = ConsistencyLevel.findByValue(iprot.readI32());
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -11250,15 +11338,15 @@ public class Cassandra {
     public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
     static {
       Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
-      tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT,
-          new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP,
-              new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING              , true),
+      tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, 
+              new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING              , true), 
               new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32))));
-      tmpMap.put(_Fields.IRE, new org.apache.thrift.meta_data.FieldMetaData("ire", org.apache.thrift.TFieldRequirementType.DEFAULT,
+      tmpMap.put(_Fields.IRE, new org.apache.thrift.meta_data.FieldMetaData("ire", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT)));
-      tmpMap.put(_Fields.UE, new org.apache.thrift.meta_data.FieldMetaData("ue", org.apache.thrift.TFieldRequirementType.DEFAULT,
+      tmpMap.put(_Fields.UE, new org.apache.thrift.meta_data.FieldMetaData("ue", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT)));
-      tmpMap.put(_Fields.TE, new org.apache.thrift.meta_data.FieldMetaData("te", org.apache.thrift.TFieldRequirementType.DEFAULT,
+      tmpMap.put(_Fields.TE, new org.apache.thrift.meta_data.FieldMetaData("te", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT)));
       metaDataMap = Collections.unmodifiableMap(tmpMap);
       org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(multiget_count_result.class, metaDataMap);
@@ -11644,7 +11732,7 @@ public class Cassandra {
       while (true)
       {
         field = iprot.readFieldBegin();
-        if (field.type == org.apache.thrift.protocol.TType.STOP) {
+        if (field.type == org.apache.thrift.protocol.TType.STOP) { 
           break;
         }
         switch (field.id) {
@@ -11663,7 +11751,7 @@ public class Cassandra {
                 }
                 iprot.readMapEnd();
               }
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -11671,7 +11759,7 @@ public class Cassandra {
             if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
               this.ire = new InvalidRequestException();
               this.ire.read(iprot);
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -11679,7 +11767,7 @@ public class Cassandra {
             if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
               this.ue = new UnavailableException();
               this.ue.read(iprot);
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -11687,7 +11775,7 @@ public class Cassandra {
             if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
               this.te = new TimedOutException();
               this.te.read(iprot);
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -11808,7 +11896,7 @@ public class Cassandra {
     public SlicePredicate predicate; // required
     public KeyRange range; // required
     /**
-     *
+     * 
      * @see ConsistencyLevel
      */
     public ConsistencyLevel consistency_level; // required
@@ -11819,7 +11907,7 @@ public class Cassandra {
       PREDICATE((short)2, "predicate"),
       RANGE((short)3, "range"),
       /**
-       *
+       * 
        * @see ConsistencyLevel
        */
       CONSISTENCY_LEVEL((short)4, "consistency_level");
@@ -11889,13 +11977,13 @@ public class Cassandra {
     public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
     static {
       Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
-      tmpMap.put(_Fields.COLUMN_PARENT, new org.apache.thrift.meta_data.FieldMetaData("column_parent", org.apache.thrift.TFieldRequirementType.REQUIRED,
+      tmpMap.put(_Fields.COLUMN_PARENT, new org.apache.thrift.meta_data.FieldMetaData("column_parent", org.apache.thrift.TFieldRequirementType.REQUIRED, 
           new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ColumnParent.class)));
-      tmpMap.put(_Fields.PREDICATE, new org.apache.thrift.meta_data.FieldMetaData("predicate", org.apache.thrift.TFieldRequirementType.REQUIRED,
+      tmpMap.put(_Fields.PREDICATE, new org.apache.thrift.meta_data.FieldMetaData("predicate", org.apache.thrift.TFieldRequirementType.REQUIRED, 
           new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, SlicePredicate.class)));
-      tmpMap.put(_Fields.RANGE, new org.apache.thrift.meta_data.FieldMetaData("range", org.apache.thrift.TFieldRequirementType.REQUIRED,
+      tmpMap.put(_Fields.RANGE, new org.apache.thrift.meta_data.FieldMetaData("range", org.apache.thrift.TFieldRequirementType.REQUIRED, 
           new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, KeyRange.class)));
-      tmpMap.put(_Fields.CONSISTENCY_LEVEL, new org.apache.thrift.meta_data.FieldMetaData("consistency_level", org.apache.thrift.TFieldRequirementType.REQUIRED,
+      tmpMap.put(_Fields.CONSISTENCY_LEVEL, new org.apache.thrift.meta_data.FieldMetaData("consistency_level", org.apache.thrift.TFieldRequirementType.REQUIRED, 
           new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, ConsistencyLevel.class)));
       metaDataMap = Collections.unmodifiableMap(tmpMap);
       org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(get_range_slices_args.class, metaDataMap);
@@ -12023,7 +12111,7 @@ public class Cassandra {
     }
 
     /**
-     *
+     * 
      * @see ConsistencyLevel
      */
     public ConsistencyLevel getConsistency_level() {
@@ -12031,7 +12119,7 @@ public class Cassandra {
     }
 
     /**
-     *
+     * 
      * @see ConsistencyLevel
      */
     public get_range_slices_args setConsistency_level(ConsistencyLevel consistency_level) {
@@ -12268,7 +12356,7 @@ public class Cassandra {
       while (true)
       {
         field = iprot.readFieldBegin();
-        if (field.type == org.apache.thrift.protocol.TType.STOP) {
+        if (field.type == org.apache.thrift.protocol.TType.STOP) { 
           break;
         }
         switch (field.id) {
@@ -12276,7 +12364,7 @@ public class Cassandra {
             if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
               this.column_parent = new ColumnParent();
               this.column_parent.read(iprot);
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -12284,7 +12372,7 @@ public class Cassandra {
             if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
               this.predicate = new SlicePredicate();
               this.predicate.read(iprot);
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -12292,14 +12380,14 @@ public class Cassandra {
             if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
               this.range = new KeyRange();
               this.range.read(iprot);
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
           case 4: // CONSISTENCY_LEVEL
             if (field.type == org.apache.thrift.protocol.TType.I32) {
               this.consistency_level = ConsistencyLevel.findByValue(iprot.readI32());
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -12501,14 +12589,14 @@ public class Cassandra {
     public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
     static {
       Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
-      tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT,
-          new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST,
+      tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
               new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, KeySlice.class))));
-      tmpMap.put(_Fields.IRE, new org.apache.thrift.meta_data.FieldMetaData("ire", org.apache.thrift.TFieldRequirementType.DEFAULT,
+      tmpMap.put(_Fields.IRE, new org.apache.thrift.meta_data.FieldMetaData("ire", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT)));
-      tmpMap.put(_Fields.UE, new org.apache.thrift.meta_data.FieldMetaData("ue", org.apache.thrift.TFieldRequirementType.DEFAULT,
+      tmpMap.put(_Fields.UE, new org.apache.thrift.meta_data.FieldMetaData("ue", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT)));
-      tmpMap.put(_Fields.TE, new org.apache.thrift.meta_data.FieldMetaData("te", org.apache.thrift.TFieldRequirementType.DEFAULT,
+      tmpMap.put(_Fields.TE, new org.apache.thrift.meta_data.FieldMetaData("te", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT)));
       metaDataMap = Collections.unmodifiableMap(tmpMap);
       org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(get_range_slices_result.class, metaDataMap);
@@ -12889,7 +12977,7 @@ public class Cassandra {
       while (true)
       {
         field = iprot.readFieldBegin();
-        if (field.type == org.apache.thrift.protocol.TType.STOP) {
+        if (field.type == org.apache.thrift.protocol.TType.STOP) { 
           break;
         }
         switch (field.id) {
@@ -12907,7 +12995,7 @@ public class Cassandra {
                 }
                 iprot.readListEnd();
               }
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -12915,7 +13003,7 @@ public class Cassandra {
             if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
               this.ire = new InvalidRequestException();
               this.ire.read(iprot);
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -12923,7 +13011,7 @@ public class Cassandra {
             if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
               this.ue = new UnavailableException();
               this.ue.read(iprot);
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -12931,7 +13019,7 @@ public class Cassandra {
             if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
               this.te = new TimedOutException();
               this.te.read(iprot);
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -13051,7 +13139,7 @@ public class Cassandra {
     public KeyRange range; // required
     public ByteBuffer start_column; // required
     /**
-     *
+     * 
      * @see ConsistencyLevel
      */
     public ConsistencyLevel consistency_level; // required
@@ -13062,7 +13150,7 @@ public class Cassandra {
       RANGE((short)2, "range"),
       START_COLUMN((short)3, "start_column"),
       /**
-       *
+       * 
        * @see ConsistencyLevel
        */
       CONSISTENCY_LEVEL((short)4, "consistency_level");
@@ -13132,13 +13220,13 @@ public class Cassandra {
     public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
     static {
       Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
-      tmpMap.put(_Fields.COLUMN_FAMILY, new org.apache.thrift.meta_data.FieldMetaData("column_family", org.apache.thrift.TFieldRequirementType.REQUIRED,
+      tmpMap.put(_Fields.COLUMN_FAMILY, new org.apache.thrift.meta_data.FieldMetaData("column_family", org.apache.thrift.TFieldRequirementType.REQUIRED, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
-      tmpMap.put(_Fields.RANGE, new org.apache.thrift.meta_data.FieldMetaData("range", org.apache.thrift.TFieldRequirementType.REQUIRED,
+      tmpMap.put(_Fields.RANGE, new org.apache.thrift.meta_data.FieldMetaData("range", org.apache.thrift.TFieldRequirementType.REQUIRED, 
           new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, KeyRange.class)));
-      tmpMap.put(_Fields.START_COLUMN, new org.apache.thrift.meta_data.FieldMetaData("start_column", org.apache.thrift.TFieldRequirementType.REQUIRED,
+      tmpMap.put(_Fields.START_COLUMN, new org.apache.thrift.meta_data.FieldMetaData("start_column", org.apache.thrift.TFieldRequirementType.REQUIRED, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING          , true)));
-      tmpMap.put(_Fields.CONSISTENCY_LEVEL, new org.apache.thrift.meta_data.FieldMetaData("consistency_level", org.apache.thrift.TFieldRequirementType.REQUIRED,
+      tmpMap.put(_Fields.CONSISTENCY_LEVEL, new org.apache.thrift.meta_data.FieldMetaData("consistency_level", org.apache.thrift.TFieldRequirementType.REQUIRED, 
           new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, ConsistencyLevel.class)));
       metaDataMap = Collections.unmodifiableMap(tmpMap);
       org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(get_paged_slice_args.class, metaDataMap);
@@ -13277,7 +13365,7 @@ public class Cassandra {
     }
 
     /**
-     *
+     * 
      * @see ConsistencyLevel
      */
     public ConsistencyLevel getConsistency_level() {
@@ -13285,7 +13373,7 @@ public class Cassandra {
     }
 
     /**
-     *
+     * 
      * @see ConsistencyLevel
      */
     public get_paged_slice_args setConsistency_level(ConsistencyLevel consistency_level) {
@@ -13522,14 +13610,14 @@ public class Cassandra {
       while (true)
       {
         field = iprot.readFieldBegin();
-        if (field.type == org.apache.thrift.protocol.TType.STOP) {
+        if (field.type == org.apache.thrift.protocol.TType.STOP) { 
           break;
         }
         switch (field.id) {
           case 1: // COLUMN_FAMILY
             if (field.type == org.apache.thrift.protocol.TType.STRING) {
               this.column_family = iprot.readString();
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -13537,21 +13625,21 @@ public class Cassandra {
             if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
               this.range = new KeyRange();
               this.range.read(iprot);
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
           case 3: // START_COLUMN
             if (field.type == org.apache.thrift.protocol.TType.STRING) {
               this.start_column = iprot.readBinary();
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
           case 4: // CONSISTENCY_LEVEL
             if (field.type == org.apache.thrift.protocol.TType.I32) {
               this.consistency_level = ConsistencyLevel.findByValue(iprot.readI32());
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -13753,14 +13841,14 @@ public class Cassandra {
     public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
     static {
       Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
-      tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT,
-          new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST,
+      tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
               new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, KeySlice.class))));
-      tmpMap.put(_Fields.IRE, new org.apache.thrift.meta_data.FieldMetaData("ire", org.apache.thrift.TFieldRequirementType.DEFAULT,
+      tmpMap.put(_Fields.IRE, new org.apache.thrift.meta_data.FieldMetaData("ire", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT)));
-      tmpMap.put(_Fields.UE, new org.apache.thrift.meta_data.FieldMetaData("ue", org.apache.thrift.TFieldRequirementType.DEFAULT,
+      tmpMap.put(_Fields.UE, new org.apache.thrift.meta_data.FieldMetaData("ue", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT)));
-      tmpMap.put(_Fields.TE, new org.apache.thrift.meta_data.FieldMetaData("te", org.apache.thrift.TFieldRequirementType.DEFAULT,
+      tmpMap.put(_Fields.TE, new org.apache.thrift.meta_data.FieldMetaData("te", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT)));
       metaDataMap = Collections.unmodifiableMap(tmpMap);
       org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(get_paged_slice_result.class, metaDataMap);
@@ -14141,7 +14229,7 @@ public class Cassandra {
       while (true)
       {
         field = iprot.readFieldBegin();
-        if (field.type == org.apache.thrift.protocol.TType.STOP) {
+        if (field.type == org.apache.thrift.protocol.TType.STOP) { 
           break;
         }
         switch (field.id) {
@@ -14159,7 +14247,7 @@ public class Cassandra {
                 }
                 iprot.readListEnd();
               }
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -14167,7 +14255,7 @@ public class Cassandra {
             if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
               this.ire = new InvalidRequestException();
               this.ire.read(iprot);
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -14175,7 +14263,7 @@ public class Cassandra {
             if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
               this.ue = new UnavailableException();
               this.ue.read(iprot);
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -14183,7 +14271,7 @@ public class Cassandra {
             if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
               this.te = new TimedOutException();
               this.te.read(iprot);
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -14303,7 +14391,7 @@ public class Cassandra {
     public IndexClause index_clause; // required
     public SlicePredicate column_predicate; // required
     /**
-     *
+     * 
      * @see ConsistencyLevel
      */
     public ConsistencyLevel consistency_level; // required
@@ -14314,7 +14402,7 @@ public class Cassandra {
       INDEX_CLAUSE((short)2, "index_clause"),
       COLUMN_PREDICATE((short)3, "column_predicate"),
       /**
-       *
+       * 
        * @see ConsistencyLevel
        */
       CONSISTENCY_LEVEL((short)4, "consistency_level");
@@ -14384,13 +14472,13 @@ public class Cassandra {
     public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
     static {
       Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
-      tmpMap.put(_Fields.COLUMN_PARENT, new org.apache.thrift.meta_data.FieldMetaData("column_parent", org.apache.thrift.TFieldRequirementType.REQUIRED,
+      tmpMap.put(_Fields.COLUMN_PARENT, new org.apache.thrift.meta_data.FieldMetaData("column_parent", org.apache.thrift.TFieldRequirementType.REQUIRED, 
           new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ColumnParent.class)));
-      tmpMap.put(_Fields.INDEX_CLAUSE, new org.apache.thrift.meta_data.FieldMetaData("index_clause", org.apache.thrift.TFieldRequirementType.REQUIRED,
+      tmpMap.put(_Fields.INDEX_CLAUSE, new org.apache.thrift.meta_data.FieldMetaData("index_clause", org.apache.thrift.TFieldRequirementType.REQUIRED, 
           new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, IndexClause.class)));
-      tmpMap.put(_Fields.COLUMN_PREDICATE, new org.apache.thrift.meta_data.FieldMetaData("column_predicate", org.apache.thrift.TFieldRequirementType.REQUIRED,
+      tmpMap.put(_Fields.COLUMN_PREDICATE, new org.apache.thrift.meta_data.FieldMetaData("column_predicate", org.apache.thrift.TFieldRequirementType.REQUIRED, 
           new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, SlicePredicate.class)));
-      tmpMap.put(_Fields.CONSISTENCY_LEVEL, new org.apache.thrift.meta_data.FieldMetaData("consistency_level", org.apache.thrift.TFieldRequirementType.REQUIRED,
+      tmpMap.put(_Fields.CONSISTENCY_LEVEL, new org.apache.thrift.meta_data.FieldMetaData("consistency_level", org.apache.thrift.TFieldRequirementType.REQUIRED, 
           new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, ConsistencyLevel.class)));
       metaDataMap = Collections.unmodifiableMap(tmpMap);
       org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(get_indexed_slices_args.class, metaDataMap);
@@ -14518,7 +14606,7 @@ public class Cassandra {
     }
 
     /**
-     *
+     * 
      * @see ConsistencyLevel
      */
     public ConsistencyLevel getConsistency_level() {
@@ -14526,7 +14614,7 @@ public class Cassandra {
     }
 
     /**
-     *
+     * 
      * @see ConsistencyLevel
      */
     public get_indexed_slices_args setConsistency_level(ConsistencyLevel consistency_level) {
@@ -14763,7 +14851,7 @@ public class Cassandra {
       while (true)
       {
         field = iprot.readFieldBegin();
-        if (field.type == org.apache.thrift.protocol.TType.STOP) {
+        if (field.type == org.apache.thrift.protocol.TType.STOP) { 

<TRUNCATED>

[09/16] git commit: add describe_local_ring for Hadoop's benefit patch by Piotr Kołaczkowski; reviewed by jbellis for CASSANDRA-6268

Posted by jb...@apache.org.
add describe_local_ring for Hadoop's benefit
patch by Piotr Kołaczkowski; reviewed by jbellis for CASSANDRA-6268


Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/86e949f3
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/86e949f3
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/86e949f3

Branch: refs/heads/trunk
Commit: 86e949f321cb1ce3b0bc0c9ed387218cc8a2b7b9
Parents: 57613dc
Author: Jonathan Ellis <jb...@apache.org>
Authored: Thu Dec 5 20:49:19 2013 -0600
Committer: Jonathan Ellis <jb...@apache.org>
Committed: Thu Dec 5 20:49:19 2013 -0600

----------------------------------------------------------------------
 CHANGES.txt                                     |    1 +
 interface/cassandra.thrift                      |    7 +-
 .../org/apache/cassandra/thrift/Cassandra.java  | 2336 ++++++++++++------
 .../org/apache/cassandra/thrift/CfDef.java      |   88 +-
 .../org/apache/cassandra/thrift/Constants.java  |    2 +-
 .../hadoop/AbstractColumnFamilyInputFormat.java |   10 +-
 .../cassandra/service/StorageService.java       |   69 +-
 .../cassandra/thrift/CassandraServer.java       |   13 +
 8 files changed, 1720 insertions(+), 806 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/86e949f3/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 166c566..3b1f2f6 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -14,6 +14,7 @@
  * Warn when collection read has > 65K elements (CASSANDRA-5428)
  * Fix cache persistence when both row and key cache are enabled 
    (CASSANDRA-6413)
+ * (Hadoop) add describe_local_ring (CASSANDRA-6268)
 
 
 1.2.12

http://git-wip-us.apache.org/repos/asf/cassandra/blob/86e949f3/interface/cassandra.thrift
----------------------------------------------------------------------
diff --git a/interface/cassandra.thrift b/interface/cassandra.thrift
index c64b2d7..1fc7201 100644
--- a/interface/cassandra.thrift
+++ b/interface/cassandra.thrift
@@ -55,7 +55,7 @@ namespace rb CassandraThrift
 # An effort should be made not to break forward-client-compatibility either
 # (e.g. one should avoid removing obsolete fields from the IDL), but no
 # guarantees in this respect are made by the Cassandra project.
-const string VERSION = "19.36.1"
+const string VERSION = "19.36.2"
 
 
 #
@@ -718,6 +718,11 @@ service Cassandra {
   list<TokenRange> describe_ring(1:required string keyspace)
                    throws (1:InvalidRequestException ire),
 
+
+  /** same as describe_ring, but considers only nodes in the local DC */
+  list<TokenRange> describe_local_ring(1:required string keyspace)
+                   throws (1:InvalidRequestException ire),
+
   /** get the mapping between token->node ip
       without taking replication into consideration
       https://issues.apache.org/jira/browse/CASSANDRA-4092 */


[05/16] add describe_local_ring for Hadoop's benefit patch by Piotr Kołaczkowski; reviewed by jbellis for CASSANDRA-6268

Posted by jb...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/86e949f3/interface/thrift/gen-java/org/apache/cassandra/thrift/Cassandra.java
----------------------------------------------------------------------
diff --git a/interface/thrift/gen-java/org/apache/cassandra/thrift/Cassandra.java b/interface/thrift/gen-java/org/apache/cassandra/thrift/Cassandra.java
index 7edbbd0..5c8d286 100644
--- a/interface/thrift/gen-java/org/apache/cassandra/thrift/Cassandra.java
+++ b/interface/thrift/gen-java/org/apache/cassandra/thrift/Cassandra.java
@@ -5,7 +5,7 @@
  */
 package org.apache.cassandra.thrift;
 /*
- *
+ * 
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -13,16 +13,16 @@ package org.apache.cassandra.thrift;
  * to you under the Apache License, Version 2.0 (the
  * "License"); you may not use this file except in compliance
  * with the License.  You may obtain a copy of the License at
- *
+ * 
  *   http://www.apache.org/licenses/LICENSE-2.0
- *
+ * 
  * Unless required by applicable law or agreed to in writing,
  * software distributed under the License is distributed on an
  * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
  * KIND, either express or implied.  See the License for the
  * specific language governing permissions and limitations
  * under the License.
- *
+ * 
  */
 
 
@@ -53,7 +53,7 @@ public class Cassandra {
     /**
      * Get the Column or SuperColumn at the given column_path. If no value is present, NotFoundException is thrown. (This is
      * the only method that can throw an exception under non-failure conditions.)
-     *
+     * 
      * @param key
      * @param column_path
      * @param consistency_level
@@ -63,7 +63,7 @@ public class Cassandra {
     /**
      * Get the group of columns contained by column_parent (either a ColumnFamily name or a ColumnFamily/SuperColumn name
      * pair) specified by the given SlicePredicate. If no matching values are found, an empty list is returned.
-     *
+     * 
      * @param key
      * @param column_parent
      * @param predicate
@@ -74,7 +74,7 @@ public class Cassandra {
     /**
      * returns the number of columns matching <code>predicate</code> for a particular <code>key</code>,
      * <code>ColumnFamily</code> and optionally <code>SuperColumn</code>.
-     *
+     * 
      * @param key
      * @param column_parent
      * @param predicate
@@ -84,7 +84,7 @@ public class Cassandra {
 
     /**
      * Performs a get_slice for column_parent and predicate for the given keys in parallel.
-     *
+     * 
      * @param keys
      * @param column_parent
      * @param predicate
@@ -94,7 +94,7 @@ public class Cassandra {
 
     /**
      * Perform a get_count in parallel on the given list<binary> keys. The return value maps keys to the count found.
-     *
+     * 
      * @param keys
      * @param column_parent
      * @param predicate
@@ -104,7 +104,7 @@ public class Cassandra {
 
     /**
      * returns a subset of columns for a contiguous range of keys.
-     *
+     * 
      * @param column_parent
      * @param predicate
      * @param range
@@ -114,7 +114,7 @@ public class Cassandra {
 
     /**
      * returns a range of columns, wrapping to the next rows if necessary to collect max_results.
-     *
+     * 
      * @param column_family
      * @param range
      * @param start_column
@@ -124,8 +124,8 @@ public class Cassandra {
 
     /**
      * Returns the subset of columns specified in SlicePredicate for the rows matching the IndexClause
-     * @deprecated; use get_range_slices instead with range.row_filter specified
-     *
+     * @deprecated use get_range_slices instead with range.row_filter specified
+     * 
      * @param column_parent
      * @param index_clause
      * @param column_predicate
@@ -135,7 +135,7 @@ public class Cassandra {
 
     /**
      * Insert a Column at the given column_parent.column_family and optional column_parent.super_column.
-     *
+     * 
      * @param key
      * @param column_parent
      * @param column
@@ -145,7 +145,7 @@ public class Cassandra {
 
     /**
      * Increment or decrement a counter.
-     *
+     * 
      * @param key
      * @param column_parent
      * @param column
@@ -157,7 +157,7 @@ public class Cassandra {
      * Remove data from the row specified by key at the granularity specified by column_path, and the given timestamp. Note
      * that all the values in column_path besides column_path.column_family are truly optional: you can remove the entire
      * row by just specifying the ColumnFamily, or you can remove a SuperColumn or a single Column by specifying those levels too.
-     *
+     * 
      * @param key
      * @param column_path
      * @param timestamp
@@ -169,7 +169,7 @@ public class Cassandra {
      * Remove a counter at the specified location.
      * Note that counters have limited support for deletes: if you remove a counter, you must wait to issue any following update
      * until the delete has reached all the nodes and all of them have been fully compacted.
-     *
+     * 
      * @param key
      * @param path
      * @param consistency_level
@@ -178,10 +178,10 @@ public class Cassandra {
 
     /**
      *   Mutate many columns or super columns for many row keys. See also: Mutation.
-     *
+     * 
      *   mutation_map maps key to column family to a list of Mutation objects to take place at that scope.
      * *
-     *
+     * 
      * @param mutation_map
      * @param consistency_level
      */
@@ -189,10 +189,10 @@ public class Cassandra {
 
     /**
      *   Atomically mutate many columns or super columns for many row keys. See also: Mutation.
-     *
+     * 
      *   mutation_map maps key to column family to a list of Mutation objects to take place at that scope.
      * *
-     *
+     * 
      * @param mutation_map
      * @param consistency_level
      */
@@ -205,7 +205,7 @@ public class Cassandra {
      * only marks the data as deleted.
      * The operation succeeds only if all hosts in the cluster at available and will throw an UnavailableException if
      * some hosts are down.
-     *
+     * 
      * @param cfname
      */
     public void truncate(String cfname) throws InvalidRequestException, UnavailableException, TimedOutException, org.apache.thrift.TException;
@@ -238,15 +238,22 @@ public class Cassandra {
      * to list of endpoints, because you can't use Thrift structs as
      * map keys:
      * https://issues.apache.org/jira/browse/THRIFT-162
-     *
+     * 
      * for the same reason, we can't return a set here, even though
      * order is neither important nor predictable.
-     *
+     * 
      * @param keyspace
      */
     public List<TokenRange> describe_ring(String keyspace) throws InvalidRequestException, org.apache.thrift.TException;
 
     /**
+     * same as describe_ring, but considers only nodes in the local DC
+     * 
+     * @param keyspace
+     */
+    public List<TokenRange> describe_local_ring(String keyspace) throws InvalidRequestException, org.apache.thrift.TException;
+
+    /**
      * get the mapping between token->node ip
      * without taking replication into consideration
      * https://issues.apache.org/jira/browse/CASSANDRA-4092
@@ -265,7 +272,7 @@ public class Cassandra {
 
     /**
      * describe specified keyspace
-     *
+     * 
      * @param keyspace
      */
     public KsDef describe_keyspace(String keyspace) throws NotFoundException, InvalidRequestException, org.apache.thrift.TException;
@@ -273,10 +280,10 @@ public class Cassandra {
     /**
      * experimental API for hadoop/parallel query support.
      * may change violently and without warning.
-     *
+     * 
      * returns list of token strings such that first subrange is (list[0], list[1]],
      * next is (list[1], list[2]], etc.
-     *
+     * 
      * @param cfName
      * @param start_token
      * @param end_token
@@ -294,42 +301,42 @@ public class Cassandra {
 
     /**
      * adds a column family. returns the new schema id.
-     *
+     * 
      * @param cf_def
      */
     public String system_add_column_family(CfDef cf_def) throws InvalidRequestException, SchemaDisagreementException, org.apache.thrift.TException;
 
     /**
      * drops a column family. returns the new schema id.
-     *
+     * 
      * @param column_family
      */
     public String system_drop_column_family(String column_family) throws InvalidRequestException, SchemaDisagreementException, org.apache.thrift.TException;
 
     /**
      * adds a keyspace and any column families that are part of it. returns the new schema id.
-     *
+     * 
      * @param ks_def
      */
     public String system_add_keyspace(KsDef ks_def) throws InvalidRequestException, SchemaDisagreementException, org.apache.thrift.TException;
 
     /**
      * drops a keyspace and any column families that are part of it. returns the new schema id.
-     *
+     * 
      * @param keyspace
      */
     public String system_drop_keyspace(String keyspace) throws InvalidRequestException, SchemaDisagreementException, org.apache.thrift.TException;
 
     /**
      * updates properties of a keyspace. returns the new schema id.
-     *
+     * 
      * @param ks_def
      */
     public String system_update_keyspace(KsDef ks_def) throws InvalidRequestException, SchemaDisagreementException, org.apache.thrift.TException;
 
     /**
      * updates properties of a column family. returns the new schema id.
-     *
+     * 
      * @param cf_def
      */
     public String system_update_column_family(CfDef cf_def) throws InvalidRequestException, SchemaDisagreementException, org.apache.thrift.TException;
@@ -337,7 +344,7 @@ public class Cassandra {
     /**
      * Executes a CQL (Cassandra Query Language) statement and returns a
      * CqlResult containing the results.
-     *
+     * 
      * @param query
      * @param compression
      */
@@ -350,7 +357,7 @@ public class Cassandra {
      * - the type of CQL statement
      * - an id token of the compiled CQL stored on the server side.
      * - a count of the discovered bound markers in the statement
-     *
+     * 
      * @param query
      * @param compression
      */
@@ -361,7 +368,7 @@ public class Cassandra {
     /**
      * Executes a prepared CQL (Cassandra Query Language) statement by passing an id token and  a list of variables
      * to bind and returns a CqlResult containing the results.
-     *
+     * 
      * @param itemId
      * @param values
      */
@@ -371,7 +378,7 @@ public class Cassandra {
 
     /**
      * @deprecated This is now a no-op. Please use the CQL3 specific methods instead.
-     *
+     * 
      * @param version
      */
     public void set_cql_version(String version) throws InvalidRequestException, org.apache.thrift.TException;
@@ -424,6 +431,8 @@ public class Cassandra {
 
     public void describe_ring(String keyspace, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.describe_ring_call> resultHandler) throws org.apache.thrift.TException;
 
+    public void describe_local_ring(String keyspace, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.describe_local_ring_call> resultHandler) throws org.apache.thrift.TException;
+
     public void describe_token_map(org.apache.thrift.async.AsyncMethodCallback<AsyncClient.describe_token_map_call> resultHandler) throws org.apache.thrift.TException;
 
     public void describe_partitioner(org.apache.thrift.async.AsyncMethodCallback<AsyncClient.describe_partitioner_call> resultHandler) throws org.apache.thrift.TException;
@@ -1153,6 +1162,32 @@ public class Cassandra {
       throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "describe_ring failed: unknown result");
     }
 
+    public List<TokenRange> describe_local_ring(String keyspace) throws InvalidRequestException, org.apache.thrift.TException
+    {
+      send_describe_local_ring(keyspace);
+      return recv_describe_local_ring();
+    }
+
+    public void send_describe_local_ring(String keyspace) throws org.apache.thrift.TException
+    {
+      describe_local_ring_args args = new describe_local_ring_args();
+      args.setKeyspace(keyspace);
+      sendBase("describe_local_ring", args);
+    }
+
+    public List<TokenRange> recv_describe_local_ring() throws InvalidRequestException, org.apache.thrift.TException
+    {
+      describe_local_ring_result result = new describe_local_ring_result();
+      receiveBase(result, "describe_local_ring");
+      if (result.isSetSuccess()) {
+        return result.success;
+      }
+      if (result.ire != null) {
+        throw result.ire;
+      }
+      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "describe_local_ring failed: unknown result");
+    }
+
     public Map<String,String> describe_token_map() throws InvalidRequestException, org.apache.thrift.TException
     {
       send_describe_token_map();
@@ -2546,6 +2581,38 @@ public class Cassandra {
       }
     }
 
+    public void describe_local_ring(String keyspace, org.apache.thrift.async.AsyncMethodCallback<describe_local_ring_call> resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      describe_local_ring_call method_call = new describe_local_ring_call(keyspace, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    public static class describe_local_ring_call extends org.apache.thrift.async.TAsyncMethodCall {
+      private String keyspace;
+      public describe_local_ring_call(String keyspace, org.apache.thrift.async.AsyncMethodCallback<describe_local_ring_call> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+        super(client, protocolFactory, transport, resultHandler, false);
+        this.keyspace = keyspace;
+      }
+
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("describe_local_ring", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        describe_local_ring_args args = new describe_local_ring_args();
+        args.setKeyspace(keyspace);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      public List<TokenRange> getResult() throws InvalidRequestException, org.apache.thrift.TException {
+        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+          throw new IllegalStateException("Method call not finished!");
+        }
+        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+        return (new Client(prot)).recv_describe_local_ring();
+      }
+    }
+
     public void describe_token_map(org.apache.thrift.async.AsyncMethodCallback<describe_token_map_call> resultHandler) throws org.apache.thrift.TException {
       checkReady();
       describe_token_map_call method_call = new describe_token_map_call(resultHandler, this, ___protocolFactory, ___transport);
@@ -3251,6 +3318,7 @@ public class Cassandra {
       processMap.put("describe_cluster_name", new describe_cluster_name());
       processMap.put("describe_version", new describe_version());
       processMap.put("describe_ring", new describe_ring());
+      processMap.put("describe_local_ring", new describe_local_ring());
       processMap.put("describe_token_map", new describe_token_map());
       processMap.put("describe_partitioner", new describe_partitioner());
       processMap.put("describe_snitch", new describe_snitch());
@@ -3771,6 +3839,26 @@ public class Cassandra {
       }
     }
 
+    private static class describe_local_ring<I extends Iface> extends org.apache.thrift.ProcessFunction<I, describe_local_ring_args> {
+      public describe_local_ring() {
+        super("describe_local_ring");
+      }
+
+      protected describe_local_ring_args getEmptyArgsInstance() {
+        return new describe_local_ring_args();
+      }
+
+      protected describe_local_ring_result getResult(I iface, describe_local_ring_args args) throws org.apache.thrift.TException {
+        describe_local_ring_result result = new describe_local_ring_result();
+        try {
+          result.success = iface.describe_local_ring(args.keyspace);
+        } catch (InvalidRequestException ire) {
+          result.ire = ire;
+        }
+        return result;
+      }
+    }
+
     private static class describe_token_map<I extends Iface> extends org.apache.thrift.ProcessFunction<I, describe_token_map_args> {
       public describe_token_map() {
         super("describe_token_map");
@@ -4269,7 +4357,7 @@ public class Cassandra {
     public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
     static {
       Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
-      tmpMap.put(_Fields.AUTH_REQUEST, new org.apache.thrift.meta_data.FieldMetaData("auth_request", org.apache.thrift.TFieldRequirementType.REQUIRED,
+      tmpMap.put(_Fields.AUTH_REQUEST, new org.apache.thrift.meta_data.FieldMetaData("auth_request", org.apache.thrift.TFieldRequirementType.REQUIRED, 
           new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, AuthenticationRequest.class)));
       metaDataMap = Collections.unmodifiableMap(tmpMap);
       org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(login_args.class, metaDataMap);
@@ -4430,7 +4518,7 @@ public class Cassandra {
       while (true)
       {
         field = iprot.readFieldBegin();
-        if (field.type == org.apache.thrift.protocol.TType.STOP) {
+        if (field.type == org.apache.thrift.protocol.TType.STOP) { 
           break;
         }
         switch (field.id) {
@@ -4438,7 +4526,7 @@ public class Cassandra {
             if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
               this.auth_request = new AuthenticationRequest();
               this.auth_request.read(iprot);
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -4582,9 +4670,9 @@ public class Cassandra {
     public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
     static {
       Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
-      tmpMap.put(_Fields.AUTHNX, new org.apache.thrift.meta_data.FieldMetaData("authnx", org.apache.thrift.TFieldRequirementType.DEFAULT,
+      tmpMap.put(_Fields.AUTHNX, new org.apache.thrift.meta_data.FieldMetaData("authnx", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT)));
-      tmpMap.put(_Fields.AUTHZX, new org.apache.thrift.meta_data.FieldMetaData("authzx", org.apache.thrift.TFieldRequirementType.DEFAULT,
+      tmpMap.put(_Fields.AUTHZX, new org.apache.thrift.meta_data.FieldMetaData("authzx", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT)));
       metaDataMap = Collections.unmodifiableMap(tmpMap);
       org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(login_result.class, metaDataMap);
@@ -4812,7 +4900,7 @@ public class Cassandra {
       while (true)
       {
         field = iprot.readFieldBegin();
-        if (field.type == org.apache.thrift.protocol.TType.STOP) {
+        if (field.type == org.apache.thrift.protocol.TType.STOP) { 
           break;
         }
         switch (field.id) {
@@ -4820,7 +4908,7 @@ public class Cassandra {
             if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
               this.authnx = new AuthenticationException();
               this.authnx.read(iprot);
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -4828,7 +4916,7 @@ public class Cassandra {
             if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
               this.authzx = new AuthorizationException();
               this.authzx.read(iprot);
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -4975,7 +5063,7 @@ public class Cassandra {
     public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
     static {
       Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
-      tmpMap.put(_Fields.KEYSPACE, new org.apache.thrift.meta_data.FieldMetaData("keyspace", org.apache.thrift.TFieldRequirementType.REQUIRED,
+      tmpMap.put(_Fields.KEYSPACE, new org.apache.thrift.meta_data.FieldMetaData("keyspace", org.apache.thrift.TFieldRequirementType.REQUIRED, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
       metaDataMap = Collections.unmodifiableMap(tmpMap);
       org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(set_keyspace_args.class, metaDataMap);
@@ -5136,14 +5224,14 @@ public class Cassandra {
       while (true)
       {
         field = iprot.readFieldBegin();
-        if (field.type == org.apache.thrift.protocol.TType.STOP) {
+        if (field.type == org.apache.thrift.protocol.TType.STOP) { 
           break;
         }
         switch (field.id) {
           case 1: // KEYSPACE
             if (field.type == org.apache.thrift.protocol.TType.STRING) {
               this.keyspace = iprot.readString();
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -5282,7 +5370,7 @@ public class Cassandra {
     public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
     static {
       Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
-      tmpMap.put(_Fields.IRE, new org.apache.thrift.meta_data.FieldMetaData("ire", org.apache.thrift.TFieldRequirementType.DEFAULT,
+      tmpMap.put(_Fields.IRE, new org.apache.thrift.meta_data.FieldMetaData("ire", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT)));
       metaDataMap = Collections.unmodifiableMap(tmpMap);
       org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(set_keyspace_result.class, metaDataMap);
@@ -5443,7 +5531,7 @@ public class Cassandra {
       while (true)
       {
         field = iprot.readFieldBegin();
-        if (field.type == org.apache.thrift.protocol.TType.STOP) {
+        if (field.type == org.apache.thrift.protocol.TType.STOP) { 
           break;
         }
         switch (field.id) {
@@ -5451,7 +5539,7 @@ public class Cassandra {
             if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
               this.ire = new InvalidRequestException();
               this.ire.read(iprot);
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -5526,7 +5614,7 @@ public class Cassandra {
     public ByteBuffer key; // required
     public ColumnPath column_path; // required
     /**
-     *
+     * 
      * @see ConsistencyLevel
      */
     public ConsistencyLevel consistency_level; // required
@@ -5536,7 +5624,7 @@ public class Cassandra {
       KEY((short)1, "key"),
       COLUMN_PATH((short)2, "column_path"),
       /**
-       *
+       * 
        * @see ConsistencyLevel
        */
       CONSISTENCY_LEVEL((short)3, "consistency_level");
@@ -5604,11 +5692,11 @@ public class Cassandra {
     public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
     static {
       Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
-      tmpMap.put(_Fields.KEY, new org.apache.thrift.meta_data.FieldMetaData("key", org.apache.thrift.TFieldRequirementType.REQUIRED,
+      tmpMap.put(_Fields.KEY, new org.apache.thrift.meta_data.FieldMetaData("key", org.apache.thrift.TFieldRequirementType.REQUIRED, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING          , true)));
-      tmpMap.put(_Fields.COLUMN_PATH, new org.apache.thrift.meta_data.FieldMetaData("column_path", org.apache.thrift.TFieldRequirementType.REQUIRED,
+      tmpMap.put(_Fields.COLUMN_PATH, new org.apache.thrift.meta_data.FieldMetaData("column_path", org.apache.thrift.TFieldRequirementType.REQUIRED, 
           new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ColumnPath.class)));
-      tmpMap.put(_Fields.CONSISTENCY_LEVEL, new org.apache.thrift.meta_data.FieldMetaData("consistency_level", org.apache.thrift.TFieldRequirementType.REQUIRED,
+      tmpMap.put(_Fields.CONSISTENCY_LEVEL, new org.apache.thrift.meta_data.FieldMetaData("consistency_level", org.apache.thrift.TFieldRequirementType.REQUIRED, 
           new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, ConsistencyLevel.class)));
       metaDataMap = Collections.unmodifiableMap(tmpMap);
       org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(get_args.class, metaDataMap);
@@ -5717,7 +5805,7 @@ public class Cassandra {
     }
 
     /**
-     *
+     * 
      * @see ConsistencyLevel
      */
     public ConsistencyLevel getConsistency_level() {
@@ -5725,7 +5813,7 @@ public class Cassandra {
     }
 
     /**
-     *
+     * 
      * @see ConsistencyLevel
      */
     public get_args setConsistency_level(ConsistencyLevel consistency_level) {
@@ -5925,14 +6013,14 @@ public class Cassandra {
       while (true)
       {
         field = iprot.readFieldBegin();
-        if (field.type == org.apache.thrift.protocol.TType.STOP) {
+        if (field.type == org.apache.thrift.protocol.TType.STOP) { 
           break;
         }
         switch (field.id) {
           case 1: // KEY
             if (field.type == org.apache.thrift.protocol.TType.STRING) {
               this.key = iprot.readBinary();
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -5940,14 +6028,14 @@ public class Cassandra {
             if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
               this.column_path = new ColumnPath();
               this.column_path.read(iprot);
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
           case 3: // CONSISTENCY_LEVEL
             if (field.type == org.apache.thrift.protocol.TType.I32) {
               this.consistency_level = ConsistencyLevel.findByValue(iprot.readI32());
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -6138,15 +6226,15 @@ public class Cassandra {
     public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
     static {
       Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
-      tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT,
+      tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ColumnOrSuperColumn.class)));
-      tmpMap.put(_Fields.IRE, new org.apache.thrift.meta_data.FieldMetaData("ire", org.apache.thrift.TFieldRequirementType.DEFAULT,
+      tmpMap.put(_Fields.IRE, new org.apache.thrift.meta_data.FieldMetaData("ire", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT)));
-      tmpMap.put(_Fields.NFE, new org.apache.thrift.meta_data.FieldMetaData("nfe", org.apache.thrift.TFieldRequirementType.DEFAULT,
+      tmpMap.put(_Fields.NFE, new org.apache.thrift.meta_data.FieldMetaData("nfe", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT)));
-      tmpMap.put(_Fields.UE, new org.apache.thrift.meta_data.FieldMetaData("ue", org.apache.thrift.TFieldRequirementType.DEFAULT,
+      tmpMap.put(_Fields.UE, new org.apache.thrift.meta_data.FieldMetaData("ue", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT)));
-      tmpMap.put(_Fields.TE, new org.apache.thrift.meta_data.FieldMetaData("te", org.apache.thrift.TFieldRequirementType.DEFAULT,
+      tmpMap.put(_Fields.TE, new org.apache.thrift.meta_data.FieldMetaData("te", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT)));
       metaDataMap = Collections.unmodifiableMap(tmpMap);
       org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(get_result.class, metaDataMap);
@@ -6575,7 +6663,7 @@ public class Cassandra {
       while (true)
       {
         field = iprot.readFieldBegin();
-        if (field.type == org.apache.thrift.protocol.TType.STOP) {
+        if (field.type == org.apache.thrift.protocol.TType.STOP) { 
           break;
         }
         switch (field.id) {
@@ -6583,7 +6671,7 @@ public class Cassandra {
             if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
               this.success = new ColumnOrSuperColumn();
               this.success.read(iprot);
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -6591,7 +6679,7 @@ public class Cassandra {
             if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
               this.ire = new InvalidRequestException();
               this.ire.read(iprot);
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -6599,7 +6687,7 @@ public class Cassandra {
             if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
               this.nfe = new NotFoundException();
               this.nfe.read(iprot);
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -6607,7 +6695,7 @@ public class Cassandra {
             if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
               this.ue = new UnavailableException();
               this.ue.read(iprot);
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -6615,7 +6703,7 @@ public class Cassandra {
             if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
               this.te = new TimedOutException();
               this.te.read(iprot);
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -6740,7 +6828,7 @@ public class Cassandra {
     public ColumnParent column_parent; // required
     public SlicePredicate predicate; // required
     /**
-     *
+     * 
      * @see ConsistencyLevel
      */
     public ConsistencyLevel consistency_level; // required
@@ -6751,7 +6839,7 @@ public class Cassandra {
       COLUMN_PARENT((short)2, "column_parent"),
       PREDICATE((short)3, "predicate"),
       /**
-       *
+       * 
        * @see ConsistencyLevel
        */
       CONSISTENCY_LEVEL((short)4, "consistency_level");
@@ -6821,13 +6909,13 @@ public class Cassandra {
     public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
     static {
       Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
-      tmpMap.put(_Fields.KEY, new org.apache.thrift.meta_data.FieldMetaData("key", org.apache.thrift.TFieldRequirementType.REQUIRED,
+      tmpMap.put(_Fields.KEY, new org.apache.thrift.meta_data.FieldMetaData("key", org.apache.thrift.TFieldRequirementType.REQUIRED, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING          , true)));
-      tmpMap.put(_Fields.COLUMN_PARENT, new org.apache.thrift.meta_data.FieldMetaData("column_parent", org.apache.thrift.TFieldRequirementType.REQUIRED,
+      tmpMap.put(_Fields.COLUMN_PARENT, new org.apache.thrift.meta_data.FieldMetaData("column_parent", org.apache.thrift.TFieldRequirementType.REQUIRED, 
           new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ColumnParent.class)));
-      tmpMap.put(_Fields.PREDICATE, new org.apache.thrift.meta_data.FieldMetaData("predicate", org.apache.thrift.TFieldRequirementType.REQUIRED,
+      tmpMap.put(_Fields.PREDICATE, new org.apache.thrift.meta_data.FieldMetaData("predicate", org.apache.thrift.TFieldRequirementType.REQUIRED, 
           new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, SlicePredicate.class)));
-      tmpMap.put(_Fields.CONSISTENCY_LEVEL, new org.apache.thrift.meta_data.FieldMetaData("consistency_level", org.apache.thrift.TFieldRequirementType.REQUIRED,
+      tmpMap.put(_Fields.CONSISTENCY_LEVEL, new org.apache.thrift.meta_data.FieldMetaData("consistency_level", org.apache.thrift.TFieldRequirementType.REQUIRED, 
           new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, ConsistencyLevel.class)));
       metaDataMap = Collections.unmodifiableMap(tmpMap);
       org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(get_slice_args.class, metaDataMap);
@@ -6966,7 +7054,7 @@ public class Cassandra {
     }
 
     /**
-     *
+     * 
      * @see ConsistencyLevel
      */
     public ConsistencyLevel getConsistency_level() {
@@ -6974,7 +7062,7 @@ public class Cassandra {
     }
 
     /**
-     *
+     * 
      * @see ConsistencyLevel
      */
     public get_slice_args setConsistency_level(ConsistencyLevel consistency_level) {
@@ -7211,14 +7299,14 @@ public class Cassandra {
       while (true)
       {
         field = iprot.readFieldBegin();
-        if (field.type == org.apache.thrift.protocol.TType.STOP) {
+        if (field.type == org.apache.thrift.protocol.TType.STOP) { 
           break;
         }
         switch (field.id) {
           case 1: // KEY
             if (field.type == org.apache.thrift.protocol.TType.STRING) {
               this.key = iprot.readBinary();
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -7226,7 +7314,7 @@ public class Cassandra {
             if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
               this.column_parent = new ColumnParent();
               this.column_parent.read(iprot);
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -7234,14 +7322,14 @@ public class Cassandra {
             if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
               this.predicate = new SlicePredicate();
               this.predicate.read(iprot);
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
           case 4: // CONSISTENCY_LEVEL
             if (field.type == org.apache.thrift.protocol.TType.I32) {
               this.consistency_level = ConsistencyLevel.findByValue(iprot.readI32());
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -7443,14 +7531,14 @@ public class Cassandra {
     public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
     static {
       Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
-      tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT,
-          new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST,
+      tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
               new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ColumnOrSuperColumn.class))));
-      tmpMap.put(_Fields.IRE, new org.apache.thrift.meta_data.FieldMetaData("ire", org.apache.thrift.TFieldRequirementType.DEFAULT,
+      tmpMap.put(_Fields.IRE, new org.apache.thrift.meta_data.FieldMetaData("ire", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT)));
-      tmpMap.put(_Fields.UE, new org.apache.thrift.meta_data.FieldMetaData("ue", org.apache.thrift.TFieldRequirementType.DEFAULT,
+      tmpMap.put(_Fields.UE, new org.apache.thrift.meta_data.FieldMetaData("ue", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT)));
-      tmpMap.put(_Fields.TE, new org.apache.thrift.meta_data.FieldMetaData("te", org.apache.thrift.TFieldRequirementType.DEFAULT,
+      tmpMap.put(_Fields.TE, new org.apache.thrift.meta_data.FieldMetaData("te", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT)));
       metaDataMap = Collections.unmodifiableMap(tmpMap);
       org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(get_slice_result.class, metaDataMap);
@@ -7831,7 +7919,7 @@ public class Cassandra {
       while (true)
       {
         field = iprot.readFieldBegin();
-        if (field.type == org.apache.thrift.protocol.TType.STOP) {
+        if (field.type == org.apache.thrift.protocol.TType.STOP) { 
           break;
         }
         switch (field.id) {
@@ -7849,7 +7937,7 @@ public class Cassandra {
                 }
                 iprot.readListEnd();
               }
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -7857,7 +7945,7 @@ public class Cassandra {
             if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
               this.ire = new InvalidRequestException();
               this.ire.read(iprot);
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -7865,7 +7953,7 @@ public class Cassandra {
             if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
               this.ue = new UnavailableException();
               this.ue.read(iprot);
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -7873,7 +7961,7 @@ public class Cassandra {
             if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
               this.te = new TimedOutException();
               this.te.read(iprot);
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -7993,7 +8081,7 @@ public class Cassandra {
     public ColumnParent column_parent; // required
     public SlicePredicate predicate; // required
     /**
-     *
+     * 
      * @see ConsistencyLevel
      */
     public ConsistencyLevel consistency_level; // required
@@ -8004,7 +8092,7 @@ public class Cassandra {
       COLUMN_PARENT((short)2, "column_parent"),
       PREDICATE((short)3, "predicate"),
       /**
-       *
+       * 
        * @see ConsistencyLevel
        */
       CONSISTENCY_LEVEL((short)4, "consistency_level");
@@ -8074,13 +8162,13 @@ public class Cassandra {
     public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
     static {
       Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
-      tmpMap.put(_Fields.KEY, new org.apache.thrift.meta_data.FieldMetaData("key", org.apache.thrift.TFieldRequirementType.REQUIRED,
+      tmpMap.put(_Fields.KEY, new org.apache.thrift.meta_data.FieldMetaData("key", org.apache.thrift.TFieldRequirementType.REQUIRED, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING          , true)));
-      tmpMap.put(_Fields.COLUMN_PARENT, new org.apache.thrift.meta_data.FieldMetaData("column_parent", org.apache.thrift.TFieldRequirementType.REQUIRED,
+      tmpMap.put(_Fields.COLUMN_PARENT, new org.apache.thrift.meta_data.FieldMetaData("column_parent", org.apache.thrift.TFieldRequirementType.REQUIRED, 
           new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ColumnParent.class)));
-      tmpMap.put(_Fields.PREDICATE, new org.apache.thrift.meta_data.FieldMetaData("predicate", org.apache.thrift.TFieldRequirementType.REQUIRED,
+      tmpMap.put(_Fields.PREDICATE, new org.apache.thrift.meta_data.FieldMetaData("predicate", org.apache.thrift.TFieldRequirementType.REQUIRED, 
           new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, SlicePredicate.class)));
-      tmpMap.put(_Fields.CONSISTENCY_LEVEL, new org.apache.thrift.meta_data.FieldMetaData("consistency_level", org.apache.thrift.TFieldRequirementType.REQUIRED,
+      tmpMap.put(_Fields.CONSISTENCY_LEVEL, new org.apache.thrift.meta_data.FieldMetaData("consistency_level", org.apache.thrift.TFieldRequirementType.REQUIRED, 
           new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, ConsistencyLevel.class)));
       metaDataMap = Collections.unmodifiableMap(tmpMap);
       org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(get_count_args.class, metaDataMap);
@@ -8219,7 +8307,7 @@ public class Cassandra {
     }
 
     /**
-     *
+     * 
      * @see ConsistencyLevel
      */
     public ConsistencyLevel getConsistency_level() {
@@ -8227,7 +8315,7 @@ public class Cassandra {
     }
 
     /**
-     *
+     * 
      * @see ConsistencyLevel
      */
     public get_count_args setConsistency_level(ConsistencyLevel consistency_level) {
@@ -8464,14 +8552,14 @@ public class Cassandra {
       while (true)
       {
         field = iprot.readFieldBegin();
-        if (field.type == org.apache.thrift.protocol.TType.STOP) {
+        if (field.type == org.apache.thrift.protocol.TType.STOP) { 
           break;
         }
         switch (field.id) {
           case 1: // KEY
             if (field.type == org.apache.thrift.protocol.TType.STRING) {
               this.key = iprot.readBinary();
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -8479,7 +8567,7 @@ public class Cassandra {
             if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
               this.column_parent = new ColumnParent();
               this.column_parent.read(iprot);
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -8487,14 +8575,14 @@ public class Cassandra {
             if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
               this.predicate = new SlicePredicate();
               this.predicate.read(iprot);
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
           case 4: // CONSISTENCY_LEVEL
             if (field.type == org.apache.thrift.protocol.TType.I32) {
               this.consistency_level = ConsistencyLevel.findByValue(iprot.readI32());
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -8698,13 +8786,13 @@ public class Cassandra {
     public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
     static {
       Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
-      tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT,
+      tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
-      tmpMap.put(_Fields.IRE, new org.apache.thrift.meta_data.FieldMetaData("ire", org.apache.thrift.TFieldRequirementType.DEFAULT,
+      tmpMap.put(_Fields.IRE, new org.apache.thrift.meta_data.FieldMetaData("ire", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT)));
-      tmpMap.put(_Fields.UE, new org.apache.thrift.meta_data.FieldMetaData("ue", org.apache.thrift.TFieldRequirementType.DEFAULT,
+      tmpMap.put(_Fields.UE, new org.apache.thrift.meta_data.FieldMetaData("ue", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT)));
-      tmpMap.put(_Fields.TE, new org.apache.thrift.meta_data.FieldMetaData("te", org.apache.thrift.TFieldRequirementType.DEFAULT,
+      tmpMap.put(_Fields.TE, new org.apache.thrift.meta_data.FieldMetaData("te", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT)));
       metaDataMap = Collections.unmodifiableMap(tmpMap);
       org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(get_count_result.class, metaDataMap);
@@ -9067,7 +9155,7 @@ public class Cassandra {
       while (true)
       {
         field = iprot.readFieldBegin();
-        if (field.type == org.apache.thrift.protocol.TType.STOP) {
+        if (field.type == org.apache.thrift.protocol.TType.STOP) { 
           break;
         }
         switch (field.id) {
@@ -9075,7 +9163,7 @@ public class Cassandra {
             if (field.type == org.apache.thrift.protocol.TType.I32) {
               this.success = iprot.readI32();
               setSuccessIsSet(true);
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -9083,7 +9171,7 @@ public class Cassandra {
             if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
               this.ire = new InvalidRequestException();
               this.ire.read(iprot);
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -9091,7 +9179,7 @@ public class Cassandra {
             if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
               this.ue = new UnavailableException();
               this.ue.read(iprot);
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -9099,7 +9187,7 @@ public class Cassandra {
             if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
               this.te = new TimedOutException();
               this.te.read(iprot);
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -9208,7 +9296,7 @@ public class Cassandra {
     public ColumnParent column_parent; // required
     public SlicePredicate predicate; // required
     /**
-     *
+     * 
      * @see ConsistencyLevel
      */
     public ConsistencyLevel consistency_level; // required
@@ -9219,7 +9307,7 @@ public class Cassandra {
       COLUMN_PARENT((short)2, "column_parent"),
       PREDICATE((short)3, "predicate"),
       /**
-       *
+       * 
        * @see ConsistencyLevel
        */
       CONSISTENCY_LEVEL((short)4, "consistency_level");
@@ -9289,14 +9377,14 @@ public class Cassandra {
     public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
     static {
       Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
-      tmpMap.put(_Fields.KEYS, new org.apache.thrift.meta_data.FieldMetaData("keys", org.apache.thrift.TFieldRequirementType.REQUIRED,
-          new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST,
+      tmpMap.put(_Fields.KEYS, new org.apache.thrift.meta_data.FieldMetaData("keys", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+          new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
               new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING              , true))));
-      tmpMap.put(_Fields.COLUMN_PARENT, new org.apache.thrift.meta_data.FieldMetaData("column_parent", org.apache.thrift.TFieldRequirementType.REQUIRED,
+      tmpMap.put(_Fields.COLUMN_PARENT, new org.apache.thrift.meta_data.FieldMetaData("column_parent", org.apache.thrift.TFieldRequirementType.REQUIRED, 
           new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ColumnParent.class)));
-      tmpMap.put(_Fields.PREDICATE, new org.apache.thrift.meta_data.FieldMetaData("predicate", org.apache.thrift.TFieldRequirementType.REQUIRED,
+      tmpMap.put(_Fields.PREDICATE, new org.apache.thrift.meta_data.FieldMetaData("predicate", org.apache.thrift.TFieldRequirementType.REQUIRED, 
           new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, SlicePredicate.class)));
-      tmpMap.put(_Fields.CONSISTENCY_LEVEL, new org.apache.thrift.meta_data.FieldMetaData("consistency_level", org.apache.thrift.TFieldRequirementType.REQUIRED,
+      tmpMap.put(_Fields.CONSISTENCY_LEVEL, new org.apache.thrift.meta_data.FieldMetaData("consistency_level", org.apache.thrift.TFieldRequirementType.REQUIRED, 
           new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, ConsistencyLevel.class)));
       metaDataMap = Collections.unmodifiableMap(tmpMap);
       org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(multiget_slice_args.class, metaDataMap);
@@ -9445,7 +9533,7 @@ public class Cassandra {
     }
 
     /**
-     *
+     * 
      * @see ConsistencyLevel
      */
     public ConsistencyLevel getConsistency_level() {
@@ -9453,7 +9541,7 @@ public class Cassandra {
     }
 
     /**
-     *
+     * 
      * @see ConsistencyLevel
      */
     public multiget_slice_args setConsistency_level(ConsistencyLevel consistency_level) {
@@ -9690,7 +9778,7 @@ public class Cassandra {
       while (true)
       {
         field = iprot.readFieldBegin();
-        if (field.type == org.apache.thrift.protocol.TType.STOP) {
+        if (field.type == org.apache.thrift.protocol.TType.STOP) { 
           break;
         }
         switch (field.id) {
@@ -9707,7 +9795,7 @@ public class Cassandra {
                 }
                 iprot.readListEnd();
               }
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -9715,7 +9803,7 @@ public class Cassandra {
             if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
               this.column_parent = new ColumnParent();
               this.column_parent.read(iprot);
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -9723,14 +9811,14 @@ public class Cassandra {
             if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
               this.predicate = new SlicePredicate();
               this.predicate.read(iprot);
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
           case 4: // CONSISTENCY_LEVEL
             if (field.type == org.apache.thrift.protocol.TType.I32) {
               this.consistency_level = ConsistencyLevel.findByValue(iprot.readI32());
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -9939,16 +10027,16 @@ public class Cassandra {
     public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
     static {
       Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
-      tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT,
-          new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP,
-              new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING              , true),
-              new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST,
+      tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, 
+              new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING              , true), 
+              new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
                   new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ColumnOrSuperColumn.class)))));
-      tmpMap.put(_Fields.IRE, new org.apache.thrift.meta_data.FieldMetaData("ire", org.apache.thrift.TFieldRequirementType.DEFAULT,
+      tmpMap.put(_Fields.IRE, new org.apache.thrift.meta_data.FieldMetaData("ire", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT)));
-      tmpMap.put(_Fields.UE, new org.apache.thrift.meta_data.FieldMetaData("ue", org.apache.thrift.TFieldRequirementType.DEFAULT,
+      tmpMap.put(_Fields.UE, new org.apache.thrift.meta_data.FieldMetaData("ue", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT)));
-      tmpMap.put(_Fields.TE, new org.apache.thrift.meta_data.FieldMetaData("te", org.apache.thrift.TFieldRequirementType.DEFAULT,
+      tmpMap.put(_Fields.TE, new org.apache.thrift.meta_data.FieldMetaData("te", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT)));
       metaDataMap = Collections.unmodifiableMap(tmpMap);
       org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(multiget_slice_result.class, metaDataMap);
@@ -10337,7 +10425,7 @@ public class Cassandra {
       while (true)
       {
         field = iprot.readFieldBegin();
-        if (field.type == org.apache.thrift.protocol.TType.STOP) {
+        if (field.type == org.apache.thrift.protocol.TType.STOP) { 
           break;
         }
         switch (field.id) {
@@ -10367,7 +10455,7 @@ public class Cassandra {
                 }
                 iprot.readMapEnd();
               }
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -10375,7 +10463,7 @@ public class Cassandra {
             if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
               this.ire = new InvalidRequestException();
               this.ire.read(iprot);
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -10383,7 +10471,7 @@ public class Cassandra {
             if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
               this.ue = new UnavailableException();
               this.ue.read(iprot);
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -10391,7 +10479,7 @@ public class Cassandra {
             if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
               this.te = new TimedOutException();
               this.te.read(iprot);
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -10519,7 +10607,7 @@ public class Cassandra {
     public ColumnParent column_parent; // required
     public SlicePredicate predicate; // required
     /**
-     *
+     * 
      * @see ConsistencyLevel
      */
     public ConsistencyLevel consistency_level; // required
@@ -10530,7 +10618,7 @@ public class Cassandra {
       COLUMN_PARENT((short)2, "column_parent"),
       PREDICATE((short)3, "predicate"),
       /**
-       *
+       * 
        * @see ConsistencyLevel
        */
       CONSISTENCY_LEVEL((short)4, "consistency_level");
@@ -10600,14 +10688,14 @@ public class Cassandra {
     public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
     static {
       Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
-      tmpMap.put(_Fields.KEYS, new org.apache.thrift.meta_data.FieldMetaData("keys", org.apache.thrift.TFieldRequirementType.REQUIRED,
-          new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST,
+      tmpMap.put(_Fields.KEYS, new org.apache.thrift.meta_data.FieldMetaData("keys", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+          new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
               new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING              , true))));
-      tmpMap.put(_Fields.COLUMN_PARENT, new org.apache.thrift.meta_data.FieldMetaData("column_parent", org.apache.thrift.TFieldRequirementType.REQUIRED,
+      tmpMap.put(_Fields.COLUMN_PARENT, new org.apache.thrift.meta_data.FieldMetaData("column_parent", org.apache.thrift.TFieldRequirementType.REQUIRED, 
           new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ColumnParent.class)));
-      tmpMap.put(_Fields.PREDICATE, new org.apache.thrift.meta_data.FieldMetaData("predicate", org.apache.thrift.TFieldRequirementType.REQUIRED,
+      tmpMap.put(_Fields.PREDICATE, new org.apache.thrift.meta_data.FieldMetaData("predicate", org.apache.thrift.TFieldRequirementType.REQUIRED, 
           new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, SlicePredicate.class)));
-      tmpMap.put(_Fields.CONSISTENCY_LEVEL, new org.apache.thrift.meta_data.FieldMetaData("consistency_level", org.apache.thrift.TFieldRequirementType.REQUIRED,
+      tmpMap.put(_Fields.CONSISTENCY_LEVEL, new org.apache.thrift.meta_data.FieldMetaData("consistency_level", org.apache.thrift.TFieldRequirementType.REQUIRED, 
           new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, ConsistencyLevel.class)));
       metaDataMap = Collections.unmodifiableMap(tmpMap);
       org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(multiget_count_args.class, metaDataMap);
@@ -10756,7 +10844,7 @@ public class Cassandra {
     }
 
     /**
-     *
+     * 
      * @see ConsistencyLevel
      */
     public ConsistencyLevel getConsistency_level() {
@@ -10764,7 +10852,7 @@ public class Cassandra {
     }
 
     /**
-     *
+     * 
      * @see ConsistencyLevel
      */
     public multiget_count_args setConsistency_level(ConsistencyLevel consistency_level) {
@@ -11001,7 +11089,7 @@ public class Cassandra {
       while (true)
       {
         field = iprot.readFieldBegin();
-        if (field.type == org.apache.thrift.protocol.TType.STOP) {
+        if (field.type == org.apache.thrift.protocol.TType.STOP) { 
           break;
         }
         switch (field.id) {
@@ -11018,7 +11106,7 @@ public class Cassandra {
                 }
                 iprot.readListEnd();
               }
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -11026,7 +11114,7 @@ public class Cassandra {
             if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
               this.column_parent = new ColumnParent();
               this.column_parent.read(iprot);
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -11034,14 +11122,14 @@ public class Cassandra {
             if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
               this.predicate = new SlicePredicate();
               this.predicate.read(iprot);
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
           case 4: // CONSISTENCY_LEVEL
             if (field.type == org.apache.thrift.protocol.TType.I32) {
               this.consistency_level = ConsistencyLevel.findByValue(iprot.readI32());
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -11250,15 +11338,15 @@ public class Cassandra {
     public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
     static {
       Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
-      tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT,
-          new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP,
-              new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING              , true),
+      tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, 
+              new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING              , true), 
               new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32))));
-      tmpMap.put(_Fields.IRE, new org.apache.thrift.meta_data.FieldMetaData("ire", org.apache.thrift.TFieldRequirementType.DEFAULT,
+      tmpMap.put(_Fields.IRE, new org.apache.thrift.meta_data.FieldMetaData("ire", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT)));
-      tmpMap.put(_Fields.UE, new org.apache.thrift.meta_data.FieldMetaData("ue", org.apache.thrift.TFieldRequirementType.DEFAULT,
+      tmpMap.put(_Fields.UE, new org.apache.thrift.meta_data.FieldMetaData("ue", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT)));
-      tmpMap.put(_Fields.TE, new org.apache.thrift.meta_data.FieldMetaData("te", org.apache.thrift.TFieldRequirementType.DEFAULT,
+      tmpMap.put(_Fields.TE, new org.apache.thrift.meta_data.FieldMetaData("te", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT)));
       metaDataMap = Collections.unmodifiableMap(tmpMap);
       org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(multiget_count_result.class, metaDataMap);
@@ -11644,7 +11732,7 @@ public class Cassandra {
       while (true)
       {
         field = iprot.readFieldBegin();
-        if (field.type == org.apache.thrift.protocol.TType.STOP) {
+        if (field.type == org.apache.thrift.protocol.TType.STOP) { 
           break;
         }
         switch (field.id) {
@@ -11663,7 +11751,7 @@ public class Cassandra {
                 }
                 iprot.readMapEnd();
               }
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -11671,7 +11759,7 @@ public class Cassandra {
             if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
               this.ire = new InvalidRequestException();
               this.ire.read(iprot);
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -11679,7 +11767,7 @@ public class Cassandra {
             if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
               this.ue = new UnavailableException();
               this.ue.read(iprot);
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -11687,7 +11775,7 @@ public class Cassandra {
             if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
               this.te = new TimedOutException();
               this.te.read(iprot);
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -11808,7 +11896,7 @@ public class Cassandra {
     public SlicePredicate predicate; // required
     public KeyRange range; // required
     /**
-     *
+     * 
      * @see ConsistencyLevel
      */
     public ConsistencyLevel consistency_level; // required
@@ -11819,7 +11907,7 @@ public class Cassandra {
       PREDICATE((short)2, "predicate"),
       RANGE((short)3, "range"),
       /**
-       *
+       * 
        * @see ConsistencyLevel
        */
       CONSISTENCY_LEVEL((short)4, "consistency_level");
@@ -11889,13 +11977,13 @@ public class Cassandra {
     public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
     static {
       Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
-      tmpMap.put(_Fields.COLUMN_PARENT, new org.apache.thrift.meta_data.FieldMetaData("column_parent", org.apache.thrift.TFieldRequirementType.REQUIRED,
+      tmpMap.put(_Fields.COLUMN_PARENT, new org.apache.thrift.meta_data.FieldMetaData("column_parent", org.apache.thrift.TFieldRequirementType.REQUIRED, 
           new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ColumnParent.class)));
-      tmpMap.put(_Fields.PREDICATE, new org.apache.thrift.meta_data.FieldMetaData("predicate", org.apache.thrift.TFieldRequirementType.REQUIRED,
+      tmpMap.put(_Fields.PREDICATE, new org.apache.thrift.meta_data.FieldMetaData("predicate", org.apache.thrift.TFieldRequirementType.REQUIRED, 
           new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, SlicePredicate.class)));
-      tmpMap.put(_Fields.RANGE, new org.apache.thrift.meta_data.FieldMetaData("range", org.apache.thrift.TFieldRequirementType.REQUIRED,
+      tmpMap.put(_Fields.RANGE, new org.apache.thrift.meta_data.FieldMetaData("range", org.apache.thrift.TFieldRequirementType.REQUIRED, 
           new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, KeyRange.class)));
-      tmpMap.put(_Fields.CONSISTENCY_LEVEL, new org.apache.thrift.meta_data.FieldMetaData("consistency_level", org.apache.thrift.TFieldRequirementType.REQUIRED,
+      tmpMap.put(_Fields.CONSISTENCY_LEVEL, new org.apache.thrift.meta_data.FieldMetaData("consistency_level", org.apache.thrift.TFieldRequirementType.REQUIRED, 
           new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, ConsistencyLevel.class)));
       metaDataMap = Collections.unmodifiableMap(tmpMap);
       org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(get_range_slices_args.class, metaDataMap);
@@ -12023,7 +12111,7 @@ public class Cassandra {
     }
 
     /**
-     *
+     * 
      * @see ConsistencyLevel
      */
     public ConsistencyLevel getConsistency_level() {
@@ -12031,7 +12119,7 @@ public class Cassandra {
     }
 
     /**
-     *
+     * 
      * @see ConsistencyLevel
      */
     public get_range_slices_args setConsistency_level(ConsistencyLevel consistency_level) {
@@ -12268,7 +12356,7 @@ public class Cassandra {
       while (true)
       {
         field = iprot.readFieldBegin();
-        if (field.type == org.apache.thrift.protocol.TType.STOP) {
+        if (field.type == org.apache.thrift.protocol.TType.STOP) { 
           break;
         }
         switch (field.id) {
@@ -12276,7 +12364,7 @@ public class Cassandra {
             if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
               this.column_parent = new ColumnParent();
               this.column_parent.read(iprot);
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -12284,7 +12372,7 @@ public class Cassandra {
             if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
               this.predicate = new SlicePredicate();
               this.predicate.read(iprot);
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -12292,14 +12380,14 @@ public class Cassandra {
             if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
               this.range = new KeyRange();
               this.range.read(iprot);
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
           case 4: // CONSISTENCY_LEVEL
             if (field.type == org.apache.thrift.protocol.TType.I32) {
               this.consistency_level = ConsistencyLevel.findByValue(iprot.readI32());
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -12501,14 +12589,14 @@ public class Cassandra {
     public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
     static {
       Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
-      tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT,
-          new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST,
+      tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
               new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, KeySlice.class))));
-      tmpMap.put(_Fields.IRE, new org.apache.thrift.meta_data.FieldMetaData("ire", org.apache.thrift.TFieldRequirementType.DEFAULT,
+      tmpMap.put(_Fields.IRE, new org.apache.thrift.meta_data.FieldMetaData("ire", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT)));
-      tmpMap.put(_Fields.UE, new org.apache.thrift.meta_data.FieldMetaData("ue", org.apache.thrift.TFieldRequirementType.DEFAULT,
+      tmpMap.put(_Fields.UE, new org.apache.thrift.meta_data.FieldMetaData("ue", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT)));
-      tmpMap.put(_Fields.TE, new org.apache.thrift.meta_data.FieldMetaData("te", org.apache.thrift.TFieldRequirementType.DEFAULT,
+      tmpMap.put(_Fields.TE, new org.apache.thrift.meta_data.FieldMetaData("te", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT)));
       metaDataMap = Collections.unmodifiableMap(tmpMap);
       org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(get_range_slices_result.class, metaDataMap);
@@ -12889,7 +12977,7 @@ public class Cassandra {
       while (true)
       {
         field = iprot.readFieldBegin();
-        if (field.type == org.apache.thrift.protocol.TType.STOP) {
+        if (field.type == org.apache.thrift.protocol.TType.STOP) { 
           break;
         }
         switch (field.id) {
@@ -12907,7 +12995,7 @@ public class Cassandra {
                 }
                 iprot.readListEnd();
               }
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -12915,7 +13003,7 @@ public class Cassandra {
             if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
               this.ire = new InvalidRequestException();
               this.ire.read(iprot);
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -12923,7 +13011,7 @@ public class Cassandra {
             if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
               this.ue = new UnavailableException();
               this.ue.read(iprot);
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -12931,7 +13019,7 @@ public class Cassandra {
             if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
               this.te = new TimedOutException();
               this.te.read(iprot);
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -13051,7 +13139,7 @@ public class Cassandra {
     public KeyRange range; // required
     public ByteBuffer start_column; // required
     /**
-     *
+     * 
      * @see ConsistencyLevel
      */
     public ConsistencyLevel consistency_level; // required
@@ -13062,7 +13150,7 @@ public class Cassandra {
       RANGE((short)2, "range"),
       START_COLUMN((short)3, "start_column"),
       /**
-       *
+       * 
        * @see ConsistencyLevel
        */
       CONSISTENCY_LEVEL((short)4, "consistency_level");
@@ -13132,13 +13220,13 @@ public class Cassandra {
     public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
     static {
       Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
-      tmpMap.put(_Fields.COLUMN_FAMILY, new org.apache.thrift.meta_data.FieldMetaData("column_family", org.apache.thrift.TFieldRequirementType.REQUIRED,
+      tmpMap.put(_Fields.COLUMN_FAMILY, new org.apache.thrift.meta_data.FieldMetaData("column_family", org.apache.thrift.TFieldRequirementType.REQUIRED, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
-      tmpMap.put(_Fields.RANGE, new org.apache.thrift.meta_data.FieldMetaData("range", org.apache.thrift.TFieldRequirementType.REQUIRED,
+      tmpMap.put(_Fields.RANGE, new org.apache.thrift.meta_data.FieldMetaData("range", org.apache.thrift.TFieldRequirementType.REQUIRED, 
           new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, KeyRange.class)));
-      tmpMap.put(_Fields.START_COLUMN, new org.apache.thrift.meta_data.FieldMetaData("start_column", org.apache.thrift.TFieldRequirementType.REQUIRED,
+      tmpMap.put(_Fields.START_COLUMN, new org.apache.thrift.meta_data.FieldMetaData("start_column", org.apache.thrift.TFieldRequirementType.REQUIRED, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING          , true)));
-      tmpMap.put(_Fields.CONSISTENCY_LEVEL, new org.apache.thrift.meta_data.FieldMetaData("consistency_level", org.apache.thrift.TFieldRequirementType.REQUIRED,
+      tmpMap.put(_Fields.CONSISTENCY_LEVEL, new org.apache.thrift.meta_data.FieldMetaData("consistency_level", org.apache.thrift.TFieldRequirementType.REQUIRED, 
           new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, ConsistencyLevel.class)));
       metaDataMap = Collections.unmodifiableMap(tmpMap);
       org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(get_paged_slice_args.class, metaDataMap);
@@ -13277,7 +13365,7 @@ public class Cassandra {
     }
 
     /**
-     *
+     * 
      * @see ConsistencyLevel
      */
     public ConsistencyLevel getConsistency_level() {
@@ -13285,7 +13373,7 @@ public class Cassandra {
     }
 
     /**
-     *
+     * 
      * @see ConsistencyLevel
      */
     public get_paged_slice_args setConsistency_level(ConsistencyLevel consistency_level) {
@@ -13522,14 +13610,14 @@ public class Cassandra {
       while (true)
       {
         field = iprot.readFieldBegin();
-        if (field.type == org.apache.thrift.protocol.TType.STOP) {
+        if (field.type == org.apache.thrift.protocol.TType.STOP) { 
           break;
         }
         switch (field.id) {
           case 1: // COLUMN_FAMILY
             if (field.type == org.apache.thrift.protocol.TType.STRING) {
               this.column_family = iprot.readString();
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -13537,21 +13625,21 @@ public class Cassandra {
             if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
               this.range = new KeyRange();
               this.range.read(iprot);
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
           case 3: // START_COLUMN
             if (field.type == org.apache.thrift.protocol.TType.STRING) {
               this.start_column = iprot.readBinary();
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
           case 4: // CONSISTENCY_LEVEL
             if (field.type == org.apache.thrift.protocol.TType.I32) {
               this.consistency_level = ConsistencyLevel.findByValue(iprot.readI32());
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -13753,14 +13841,14 @@ public class Cassandra {
     public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
     static {
       Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
-      tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT,
-          new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST,
+      tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
               new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, KeySlice.class))));
-      tmpMap.put(_Fields.IRE, new org.apache.thrift.meta_data.FieldMetaData("ire", org.apache.thrift.TFieldRequirementType.DEFAULT,
+      tmpMap.put(_Fields.IRE, new org.apache.thrift.meta_data.FieldMetaData("ire", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT)));
-      tmpMap.put(_Fields.UE, new org.apache.thrift.meta_data.FieldMetaData("ue", org.apache.thrift.TFieldRequirementType.DEFAULT,
+      tmpMap.put(_Fields.UE, new org.apache.thrift.meta_data.FieldMetaData("ue", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT)));
-      tmpMap.put(_Fields.TE, new org.apache.thrift.meta_data.FieldMetaData("te", org.apache.thrift.TFieldRequirementType.DEFAULT,
+      tmpMap.put(_Fields.TE, new org.apache.thrift.meta_data.FieldMetaData("te", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT)));
       metaDataMap = Collections.unmodifiableMap(tmpMap);
       org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(get_paged_slice_result.class, metaDataMap);
@@ -14141,7 +14229,7 @@ public class Cassandra {
       while (true)
       {
         field = iprot.readFieldBegin();
-        if (field.type == org.apache.thrift.protocol.TType.STOP) {
+        if (field.type == org.apache.thrift.protocol.TType.STOP) { 
           break;
         }
         switch (field.id) {
@@ -14159,7 +14247,7 @@ public class Cassandra {
                 }
                 iprot.readListEnd();
               }
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -14167,7 +14255,7 @@ public class Cassandra {
             if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
               this.ire = new InvalidRequestException();
               this.ire.read(iprot);
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -14175,7 +14263,7 @@ public class Cassandra {
             if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
               this.ue = new UnavailableException();
               this.ue.read(iprot);
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -14183,7 +14271,7 @@ public class Cassandra {
             if (field.type == org.apache.thrift.protocol.TType.STRUCT) {
               this.te = new TimedOutException();
               this.te.read(iprot);
-            } else {
+            } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
@@ -14303,7 +14391,7 @@ public class Cassandra {
     public IndexClause index_clause; // required
     public SlicePredicate column_predicate; // required
     /**
-     *
+     * 
      * @see ConsistencyLevel
      */
     public ConsistencyLevel consistency_level; // required
@@ -14314,7 +14402,7 @@ public class Cassandra {
       INDEX_CLAUSE((short)2, "index_clause"),
       COLUMN_PREDICATE((short)3, "column_predicate"),
       /**
-       *
+       * 
        * @see ConsistencyLevel
        */
       CONSISTENCY_LEVEL((short)4, "consistency_level");
@@ -14384,13 +14472,13 @@ public class Cassandra {
     public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
     static {
       Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
-      tmpMap.put(_Fields.COLUMN_PARENT, new org.apache.thrift.meta_data.FieldMetaData("column_parent", org.apache.thrift.TFieldRequirementType.REQUIRED,
+      tmpMap.put(_Fields.COLUMN_PARENT, new org.apache.thrift.meta_data.FieldMetaData("column_parent", org.apache.thrift.TFieldRequirementType.REQUIRED, 
           new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ColumnParent.class)));
-      tmpMap.put(_Fields.INDEX_CLAUSE, new org.apache.thrift.meta_data.FieldMetaData("index_clause", org.apache.thrift.TFieldRequirementType.REQUIRED,
+      tmpMap.put(_Fields.INDEX_CLAUSE, new org.apache.thrift.meta_data.FieldMetaData("index_clause", org.apache.thrift.TFieldRequirementType.REQUIRED, 
           new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, IndexClause.class)));
-      tmpMap.put(_Fields.COLUMN_PREDICATE, new org.apache.thrift.meta_data.FieldMetaData("column_predicate", org.apache.thrift.TFieldRequirementType.REQUIRED,
+      tmpMap.put(_Fields.COLUMN_PREDICATE, new org.apache.thrift.meta_data.FieldMetaData("column_predicate", org.apache.thrift.TFieldRequirementType.REQUIRED, 
           new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, SlicePredicate.class)));
-      tmpMap.put(_Fields.CONSISTENCY_LEVEL, new org.apache.thrift.meta_data.FieldMetaData("consistency_level", org.apache.thrift.TFieldRequirementType.REQUIRED,
+      tmpMap.put(_Fields.CONSISTENCY_LEVEL, new org.apache.thrift.meta_data.FieldMetaData("consistency_level", org.apache.thrift.TFieldRequirementType.REQUIRED, 
           new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, ConsistencyLevel.class)));
       metaDataMap = Collections.unmodifiableMap(tmpMap);
       org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(get_indexed_slices_args.class, metaDataMap);
@@ -14518,7 +14606,7 @@ public class Cassandra {
     }
 
     /**
-     *
+     * 
      * @see ConsistencyLevel
      */
     public ConsistencyLevel getConsistency_level() {
@@ -14526,7 +14614,7 @@ public class Cassandra {
     }
 
     /**
-     *
+     * 
      * @see ConsistencyLevel
      */
     public get_indexed_slices_args setConsistency_level(ConsistencyLevel consistency_level) {
@@ -14763,7 +14851,7 @@ public class Cassandra {
       while (true)
       {
         field = iprot.readFieldBegin();
-        if (field.type == org.apache.thrift.protocol.TType.STOP) {
+        if (field.type == org.apache.thrift.protocol.TType.STOP) { 

<TRUNCATED>

[10/16] Merge branch 'cassandra-1.2' into cassandra-2.0

Posted by jb...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/f2a82ee7/interface/thrift/gen-java/org/apache/cassandra/thrift/cassandraConstants.java
----------------------------------------------------------------------
diff --cc interface/thrift/gen-java/org/apache/cassandra/thrift/cassandraConstants.java
index a2c482e,0000000..81b0ffb
mode 100644,000000..100644
--- a/interface/thrift/gen-java/org/apache/cassandra/thrift/cassandraConstants.java
+++ b/interface/thrift/gen-java/org/apache/cassandra/thrift/cassandraConstants.java
@@@ -1,61 -1,0 +1,61 @@@
 +/**
 + * Autogenerated by Thrift Compiler (0.9.1)
 + *
 + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
 + *  @generated
 + */
 +package org.apache.cassandra.thrift;
 +/*
 + * 
 + * Licensed to the Apache Software Foundation (ASF) under one
 + * or more contributor license agreements.  See the NOTICE file
 + * distributed with this work for additional information
 + * regarding copyright ownership.  The ASF licenses this file
 + * to you under the Apache License, Version 2.0 (the
 + * "License"); you may not use this file except in compliance
 + * with the License.  You may obtain a copy of the License at
 + * 
 + *   http://www.apache.org/licenses/LICENSE-2.0
 + * 
 + * Unless required by applicable law or agreed to in writing,
 + * software distributed under the License is distributed on an
 + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
 + * KIND, either express or implied.  See the License for the
 + * specific language governing permissions and limitations
 + * under the License.
 + * 
 + */
 +
 +
 +import org.apache.commons.lang3.builder.HashCodeBuilder;
 +import org.apache.thrift.scheme.IScheme;
 +import org.apache.thrift.scheme.SchemeFactory;
 +import org.apache.thrift.scheme.StandardScheme;
 +
 +import org.apache.thrift.scheme.TupleScheme;
 +import org.apache.thrift.protocol.TTupleProtocol;
 +import org.apache.thrift.protocol.TProtocolException;
 +import org.apache.thrift.EncodingUtils;
 +import org.apache.thrift.TException;
 +import org.apache.thrift.async.AsyncMethodCallback;
 +import org.apache.thrift.server.AbstractNonblockingServer.*;
 +import java.util.List;
 +import java.util.ArrayList;
 +import java.util.Map;
 +import java.util.HashMap;
 +import java.util.EnumMap;
 +import java.util.Set;
 +import java.util.HashSet;
 +import java.util.EnumSet;
 +import java.util.Collections;
 +import java.util.BitSet;
 +import java.nio.ByteBuffer;
 +import java.util.Arrays;
 +import org.slf4j.Logger;
 +import org.slf4j.LoggerFactory;
 +
 +public class cassandraConstants {
 +
-   public static final String VERSION = "19.38.0";
++  public static final String VERSION = "19.39.0";
 +
 +}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f2a82ee7/src/java/org/apache/cassandra/hadoop/AbstractColumnFamilyInputFormat.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f2a82ee7/src/java/org/apache/cassandra/service/StorageService.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/service/StorageService.java
index 18ce97b,b85f478..c341684
--- a/src/java/org/apache/cassandra/service/StorageService.java
+++ b/src/java/org/apache/cassandra/service/StorageService.java
@@@ -34,17 -34,13 +34,18 @@@ import javax.management.Notification
  import javax.management.NotificationBroadcasterSupport;
  import javax.management.ObjectName;
  
 +import static java.nio.charset.StandardCharsets.ISO_8859_1;
 +
 +import com.google.common.annotations.VisibleForTesting;
+ import com.google.common.base.Predicate;
  import com.google.common.collect.*;
 -
  import com.google.common.util.concurrent.AtomicDouble;
 -import org.apache.cassandra.db.index.SecondaryIndex;
 +import com.google.common.util.concurrent.FutureCallback;
 +import com.google.common.util.concurrent.Futures;
 +import com.google.common.util.concurrent.Uninterruptibles;
 +
 +import org.apache.commons.lang3.StringUtils;
  import org.apache.log4j.Level;
 -import org.apache.commons.lang.StringUtils;
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
@@@ -1063,12 -1081,57 +1064,57 @@@ public class StorageService extends Not
  
      public Map<Range<Token>, List<InetAddress>> getRangeToAddressMap(String keyspace)
      {
+         return getRangeToAddressMap(keyspace, tokenMetadata.sortedTokens());
+     }
+ 
+     public Map<Range<Token>, List<InetAddress>> getRangeToAddressMapInLocalDC(String keyspace)
+     {
+         Predicate<InetAddress> isLocalDC = new Predicate<InetAddress>()
+         {
+             public boolean apply(InetAddress address)
+             {
+                 return isLocalDC(address);
+             }
+         };
+ 
+         Map<Range<Token>, List<InetAddress>> origMap = getRangeToAddressMap(keyspace, getTokensInLocalDC());
+         Map<Range<Token>, List<InetAddress>> filteredMap = Maps.newHashMap();
+         for (Map.Entry<Range<Token>, List<InetAddress>> entry : origMap.entrySet())
+         {
+             List<InetAddress> endpointsInLocalDC = Lists.newArrayList(Collections2.filter(entry.getValue(), isLocalDC));
+             filteredMap.put(entry.getKey(), endpointsInLocalDC);
+         }
+ 
+         return filteredMap;
+     }
+ 
+     private List<Token> getTokensInLocalDC()
+     {
+         List<Token> filteredTokens = Lists.newArrayList();
+         for (Token token : tokenMetadata.sortedTokens())
+         {
+             InetAddress endpoint = tokenMetadata.getEndpoint(token);
+             if (isLocalDC(endpoint))
+                 filteredTokens.add(token);
+         }
+         return filteredTokens;
+     }
+ 
+     private boolean isLocalDC(InetAddress targetHost)
+     {
+         String remoteDC = DatabaseDescriptor.getEndpointSnitch().getDatacenter(targetHost);
+         String localDC = DatabaseDescriptor.getEndpointSnitch().getDatacenter(FBUtilities.getBroadcastAddress());
+         return remoteDC.equals(localDC);
+     }
+ 
+     private Map<Range<Token>, List<InetAddress>> getRangeToAddressMap(String keyspace, List<Token> sortedTokens)
+     {
          // some people just want to get a visual representation of things. Allow null and set it to the first
 -        // non-system table.
 +        // non-system keyspace.
          if (keyspace == null)
 -            keyspace = Schema.instance.getNonSystemTables().get(0);
 +            keyspace = Schema.instance.getNonSystemKeyspaces().get(0);
  
-         List<Range<Token>> ranges = getAllRanges(tokenMetadata.sortedTokens());
+         List<Range<Token>> ranges = getAllRanges(sortedTokens);
          return constructRangeToEndpointMap(keyspace, ranges);
      }
  
@@@ -1109,7 -1173,20 +1156,20 @@@
       */
      public List<TokenRange> describeRing(String keyspace) throws InvalidRequestException
      {
+         return describeRing(keyspace, false);
+     }
+ 
+     /**
+      * The same as {@code describeRing(String)} but considers only the part of the ring formed by nodes in the local DC.
+      */
+     public List<TokenRange> describeLocalRing(String keyspace) throws InvalidRequestException
+     {
+         return describeRing(keyspace, true);
+     }
+ 
+     private List<TokenRange> describeRing(String keyspace, boolean includeOnlyLocalDC) throws InvalidRequestException
+     {
 -        if (keyspace == null || Table.open(keyspace).getReplicationStrategy() instanceof LocalStrategy)
 +        if (keyspace == null || Keyspace.open(keyspace).getReplicationStrategy() instanceof LocalStrategy)
              throw new InvalidRequestException("There is no ring for the keyspace: " + keyspace);
  
          List<TokenRange> ranges = new ArrayList<TokenRange>();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f2a82ee7/src/java/org/apache/cassandra/thrift/CassandraServer.java
----------------------------------------------------------------------


[13/16] Merge branch 'cassandra-1.2' into cassandra-2.0

Posted by jb...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/f2a82ee7/interface/thrift/gen-java/org/apache/cassandra/thrift/cassandraConstants.java
----------------------------------------------------------------------
diff --cc interface/thrift/gen-java/org/apache/cassandra/thrift/cassandraConstants.java
index a2c482e,0000000..81b0ffb
mode 100644,000000..100644
--- a/interface/thrift/gen-java/org/apache/cassandra/thrift/cassandraConstants.java
+++ b/interface/thrift/gen-java/org/apache/cassandra/thrift/cassandraConstants.java
@@@ -1,61 -1,0 +1,61 @@@
 +/**
 + * Autogenerated by Thrift Compiler (0.9.1)
 + *
 + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
 + *  @generated
 + */
 +package org.apache.cassandra.thrift;
 +/*
 + * 
 + * Licensed to the Apache Software Foundation (ASF) under one
 + * or more contributor license agreements.  See the NOTICE file
 + * distributed with this work for additional information
 + * regarding copyright ownership.  The ASF licenses this file
 + * to you under the Apache License, Version 2.0 (the
 + * "License"); you may not use this file except in compliance
 + * with the License.  You may obtain a copy of the License at
 + * 
 + *   http://www.apache.org/licenses/LICENSE-2.0
 + * 
 + * Unless required by applicable law or agreed to in writing,
 + * software distributed under the License is distributed on an
 + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
 + * KIND, either express or implied.  See the License for the
 + * specific language governing permissions and limitations
 + * under the License.
 + * 
 + */
 +
 +
 +import org.apache.commons.lang3.builder.HashCodeBuilder;
 +import org.apache.thrift.scheme.IScheme;
 +import org.apache.thrift.scheme.SchemeFactory;
 +import org.apache.thrift.scheme.StandardScheme;
 +
 +import org.apache.thrift.scheme.TupleScheme;
 +import org.apache.thrift.protocol.TTupleProtocol;
 +import org.apache.thrift.protocol.TProtocolException;
 +import org.apache.thrift.EncodingUtils;
 +import org.apache.thrift.TException;
 +import org.apache.thrift.async.AsyncMethodCallback;
 +import org.apache.thrift.server.AbstractNonblockingServer.*;
 +import java.util.List;
 +import java.util.ArrayList;
 +import java.util.Map;
 +import java.util.HashMap;
 +import java.util.EnumMap;
 +import java.util.Set;
 +import java.util.HashSet;
 +import java.util.EnumSet;
 +import java.util.Collections;
 +import java.util.BitSet;
 +import java.nio.ByteBuffer;
 +import java.util.Arrays;
 +import org.slf4j.Logger;
 +import org.slf4j.LoggerFactory;
 +
 +public class cassandraConstants {
 +
-   public static final String VERSION = "19.38.0";
++  public static final String VERSION = "19.39.0";
 +
 +}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f2a82ee7/src/java/org/apache/cassandra/hadoop/AbstractColumnFamilyInputFormat.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f2a82ee7/src/java/org/apache/cassandra/service/StorageService.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/service/StorageService.java
index 18ce97b,b85f478..c341684
--- a/src/java/org/apache/cassandra/service/StorageService.java
+++ b/src/java/org/apache/cassandra/service/StorageService.java
@@@ -34,17 -34,13 +34,18 @@@ import javax.management.Notification
  import javax.management.NotificationBroadcasterSupport;
  import javax.management.ObjectName;
  
 +import static java.nio.charset.StandardCharsets.ISO_8859_1;
 +
 +import com.google.common.annotations.VisibleForTesting;
+ import com.google.common.base.Predicate;
  import com.google.common.collect.*;
 -
  import com.google.common.util.concurrent.AtomicDouble;
 -import org.apache.cassandra.db.index.SecondaryIndex;
 +import com.google.common.util.concurrent.FutureCallback;
 +import com.google.common.util.concurrent.Futures;
 +import com.google.common.util.concurrent.Uninterruptibles;
 +
 +import org.apache.commons.lang3.StringUtils;
  import org.apache.log4j.Level;
 -import org.apache.commons.lang.StringUtils;
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
@@@ -1063,12 -1081,57 +1064,57 @@@ public class StorageService extends Not
  
      public Map<Range<Token>, List<InetAddress>> getRangeToAddressMap(String keyspace)
      {
+         return getRangeToAddressMap(keyspace, tokenMetadata.sortedTokens());
+     }
+ 
+     public Map<Range<Token>, List<InetAddress>> getRangeToAddressMapInLocalDC(String keyspace)
+     {
+         Predicate<InetAddress> isLocalDC = new Predicate<InetAddress>()
+         {
+             public boolean apply(InetAddress address)
+             {
+                 return isLocalDC(address);
+             }
+         };
+ 
+         Map<Range<Token>, List<InetAddress>> origMap = getRangeToAddressMap(keyspace, getTokensInLocalDC());
+         Map<Range<Token>, List<InetAddress>> filteredMap = Maps.newHashMap();
+         for (Map.Entry<Range<Token>, List<InetAddress>> entry : origMap.entrySet())
+         {
+             List<InetAddress> endpointsInLocalDC = Lists.newArrayList(Collections2.filter(entry.getValue(), isLocalDC));
+             filteredMap.put(entry.getKey(), endpointsInLocalDC);
+         }
+ 
+         return filteredMap;
+     }
+ 
+     private List<Token> getTokensInLocalDC()
+     {
+         List<Token> filteredTokens = Lists.newArrayList();
+         for (Token token : tokenMetadata.sortedTokens())
+         {
+             InetAddress endpoint = tokenMetadata.getEndpoint(token);
+             if (isLocalDC(endpoint))
+                 filteredTokens.add(token);
+         }
+         return filteredTokens;
+     }
+ 
+     private boolean isLocalDC(InetAddress targetHost)
+     {
+         String remoteDC = DatabaseDescriptor.getEndpointSnitch().getDatacenter(targetHost);
+         String localDC = DatabaseDescriptor.getEndpointSnitch().getDatacenter(FBUtilities.getBroadcastAddress());
+         return remoteDC.equals(localDC);
+     }
+ 
+     private Map<Range<Token>, List<InetAddress>> getRangeToAddressMap(String keyspace, List<Token> sortedTokens)
+     {
          // some people just want to get a visual representation of things. Allow null and set it to the first
 -        // non-system table.
 +        // non-system keyspace.
          if (keyspace == null)
 -            keyspace = Schema.instance.getNonSystemTables().get(0);
 +            keyspace = Schema.instance.getNonSystemKeyspaces().get(0);
  
-         List<Range<Token>> ranges = getAllRanges(tokenMetadata.sortedTokens());
+         List<Range<Token>> ranges = getAllRanges(sortedTokens);
          return constructRangeToEndpointMap(keyspace, ranges);
      }
  
@@@ -1109,7 -1173,20 +1156,20 @@@
       */
      public List<TokenRange> describeRing(String keyspace) throws InvalidRequestException
      {
+         return describeRing(keyspace, false);
+     }
+ 
+     /**
+      * The same as {@code describeRing(String)} but considers only the part of the ring formed by nodes in the local DC.
+      */
+     public List<TokenRange> describeLocalRing(String keyspace) throws InvalidRequestException
+     {
+         return describeRing(keyspace, true);
+     }
+ 
+     private List<TokenRange> describeRing(String keyspace, boolean includeOnlyLocalDC) throws InvalidRequestException
+     {
 -        if (keyspace == null || Table.open(keyspace).getReplicationStrategy() instanceof LocalStrategy)
 +        if (keyspace == null || Keyspace.open(keyspace).getReplicationStrategy() instanceof LocalStrategy)
              throw new InvalidRequestException("There is no ring for the keyspace: " + keyspace);
  
          List<TokenRange> ranges = new ArrayList<TokenRange>();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f2a82ee7/src/java/org/apache/cassandra/thrift/CassandraServer.java
----------------------------------------------------------------------


[06/16] git commit: add describe_local_ring for Hadoop's benefit patch by Piotr Kołaczkowski; reviewed by jbellis for CASSANDRA-6268

Posted by jb...@apache.org.
add describe_local_ring for Hadoop's benefit
patch by Piotr Kołaczkowski; reviewed by jbellis for CASSANDRA-6268


Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/86e949f3
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/86e949f3
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/86e949f3

Branch: refs/heads/cassandra-2.0
Commit: 86e949f321cb1ce3b0bc0c9ed387218cc8a2b7b9
Parents: 57613dc
Author: Jonathan Ellis <jb...@apache.org>
Authored: Thu Dec 5 20:49:19 2013 -0600
Committer: Jonathan Ellis <jb...@apache.org>
Committed: Thu Dec 5 20:49:19 2013 -0600

----------------------------------------------------------------------
 CHANGES.txt                                     |    1 +
 interface/cassandra.thrift                      |    7 +-
 .../org/apache/cassandra/thrift/Cassandra.java  | 2336 ++++++++++++------
 .../org/apache/cassandra/thrift/CfDef.java      |   88 +-
 .../org/apache/cassandra/thrift/Constants.java  |    2 +-
 .../hadoop/AbstractColumnFamilyInputFormat.java |   10 +-
 .../cassandra/service/StorageService.java       |   69 +-
 .../cassandra/thrift/CassandraServer.java       |   13 +
 8 files changed, 1720 insertions(+), 806 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/86e949f3/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 166c566..3b1f2f6 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -14,6 +14,7 @@
  * Warn when collection read has > 65K elements (CASSANDRA-5428)
  * Fix cache persistence when both row and key cache are enabled 
    (CASSANDRA-6413)
+ * (Hadoop) add describe_local_ring (CASSANDRA-6268)
 
 
 1.2.12

http://git-wip-us.apache.org/repos/asf/cassandra/blob/86e949f3/interface/cassandra.thrift
----------------------------------------------------------------------
diff --git a/interface/cassandra.thrift b/interface/cassandra.thrift
index c64b2d7..1fc7201 100644
--- a/interface/cassandra.thrift
+++ b/interface/cassandra.thrift
@@ -55,7 +55,7 @@ namespace rb CassandraThrift
 # An effort should be made not to break forward-client-compatibility either
 # (e.g. one should avoid removing obsolete fields from the IDL), but no
 # guarantees in this respect are made by the Cassandra project.
-const string VERSION = "19.36.1"
+const string VERSION = "19.36.2"
 
 
 #
@@ -718,6 +718,11 @@ service Cassandra {
   list<TokenRange> describe_ring(1:required string keyspace)
                    throws (1:InvalidRequestException ire),
 
+
+  /** same as describe_ring, but considers only nodes in the local DC */
+  list<TokenRange> describe_local_ring(1:required string keyspace)
+                   throws (1:InvalidRequestException ire),
+
   /** get the mapping between token->node ip
       without taking replication into consideration
       https://issues.apache.org/jira/browse/CASSANDRA-4092 */


[14/16] Merge branch 'cassandra-1.2' into cassandra-2.0

Posted by jb...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/f2a82ee7/interface/thrift/gen-java/org/apache/cassandra/thrift/Cassandra.java
----------------------------------------------------------------------
diff --cc interface/thrift/gen-java/org/apache/cassandra/thrift/Cassandra.java
index 837acfc,5c8d286..15b99fa
--- a/interface/thrift/gen-java/org/apache/cassandra/thrift/Cassandra.java
+++ b/interface/thrift/gen-java/org/apache/cassandra/thrift/Cassandra.java
@@@ -437,91 -387,91 +444,93 @@@ public class Cassandra 
  
    public interface AsyncIface {
  
 -    public void login(AuthenticationRequest auth_request, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.login_call> resultHandler) throws org.apache.thrift.TException;
 +    public void login(AuthenticationRequest auth_request, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
 -    public void set_keyspace(String keyspace, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.set_keyspace_call> resultHandler) throws org.apache.thrift.TException;
 +    public void set_keyspace(String keyspace, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
 -    public void get(ByteBuffer key, ColumnPath column_path, ConsistencyLevel consistency_level, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.get_call> resultHandler) throws org.apache.thrift.TException;
 +    public void get(ByteBuffer key, ColumnPath column_path, ConsistencyLevel consistency_level, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
 -    public void get_slice(ByteBuffer key, ColumnParent column_parent, SlicePredicate predicate, ConsistencyLevel consistency_level, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.get_slice_call> resultHandler) throws org.apache.thrift.TException;
 +    public void get_slice(ByteBuffer key, ColumnParent column_parent, SlicePredicate predicate, ConsistencyLevel consistency_level, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
 -    public void get_count(ByteBuffer key, ColumnParent column_parent, SlicePredicate predicate, ConsistencyLevel consistency_level, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.get_count_call> resultHandler) throws org.apache.thrift.TException;
 +    public void get_count(ByteBuffer key, ColumnParent column_parent, SlicePredicate predicate, ConsistencyLevel consistency_level, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
 -    public void multiget_slice(List<ByteBuffer> keys, ColumnParent column_parent, SlicePredicate predicate, ConsistencyLevel consistency_level, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.multiget_slice_call> resultHandler) throws org.apache.thrift.TException;
 +    public void multiget_slice(List<ByteBuffer> keys, ColumnParent column_parent, SlicePredicate predicate, ConsistencyLevel consistency_level, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
 -    public void multiget_count(List<ByteBuffer> keys, ColumnParent column_parent, SlicePredicate predicate, ConsistencyLevel consistency_level, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.multiget_count_call> resultHandler) throws org.apache.thrift.TException;
 +    public void multiget_count(List<ByteBuffer> keys, ColumnParent column_parent, SlicePredicate predicate, ConsistencyLevel consistency_level, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
 -    public void get_range_slices(ColumnParent column_parent, SlicePredicate predicate, KeyRange range, ConsistencyLevel consistency_level, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.get_range_slices_call> resultHandler) throws org.apache.thrift.TException;
 +    public void get_range_slices(ColumnParent column_parent, SlicePredicate predicate, KeyRange range, ConsistencyLevel consistency_level, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
 -    public void get_paged_slice(String column_family, KeyRange range, ByteBuffer start_column, ConsistencyLevel consistency_level, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.get_paged_slice_call> resultHandler) throws org.apache.thrift.TException;
 +    public void get_paged_slice(String column_family, KeyRange range, ByteBuffer start_column, ConsistencyLevel consistency_level, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
 -    public void get_indexed_slices(ColumnParent column_parent, IndexClause index_clause, SlicePredicate column_predicate, ConsistencyLevel consistency_level, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.get_indexed_slices_call> resultHandler) throws org.apache.thrift.TException;
 +    public void get_indexed_slices(ColumnParent column_parent, IndexClause index_clause, SlicePredicate column_predicate, ConsistencyLevel consistency_level, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
 -    public void insert(ByteBuffer key, ColumnParent column_parent, Column column, ConsistencyLevel consistency_level, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.insert_call> resultHandler) throws org.apache.thrift.TException;
 +    public void insert(ByteBuffer key, ColumnParent column_parent, Column column, ConsistencyLevel consistency_level, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
 -    public void add(ByteBuffer key, ColumnParent column_parent, CounterColumn column, ConsistencyLevel consistency_level, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.add_call> resultHandler) throws org.apache.thrift.TException;
 +    public void add(ByteBuffer key, ColumnParent column_parent, CounterColumn column, ConsistencyLevel consistency_level, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
 -    public void remove(ByteBuffer key, ColumnPath column_path, long timestamp, ConsistencyLevel consistency_level, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.remove_call> resultHandler) throws org.apache.thrift.TException;
 +    public void cas(ByteBuffer key, String column_family, List<Column> expected, List<Column> updates, ConsistencyLevel serial_consistency_level, ConsistencyLevel commit_consistency_level, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
 -    public void remove_counter(ByteBuffer key, ColumnPath path, ConsistencyLevel consistency_level, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.remove_counter_call> resultHandler) throws org.apache.thrift.TException;
 +    public void remove(ByteBuffer key, ColumnPath column_path, long timestamp, ConsistencyLevel consistency_level, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
 -    public void batch_mutate(Map<ByteBuffer,Map<String,List<Mutation>>> mutation_map, ConsistencyLevel consistency_level, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.batch_mutate_call> resultHandler) throws org.apache.thrift.TException;
 +    public void remove_counter(ByteBuffer key, ColumnPath path, ConsistencyLevel consistency_level, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
 -    public void atomic_batch_mutate(Map<ByteBuffer,Map<String,List<Mutation>>> mutation_map, ConsistencyLevel consistency_level, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.atomic_batch_mutate_call> resultHandler) throws org.apache.thrift.TException;
 +    public void batch_mutate(Map<ByteBuffer,Map<String,List<Mutation>>> mutation_map, ConsistencyLevel consistency_level, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
 -    public void truncate(String cfname, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.truncate_call> resultHandler) throws org.apache.thrift.TException;
 +    public void atomic_batch_mutate(Map<ByteBuffer,Map<String,List<Mutation>>> mutation_map, ConsistencyLevel consistency_level, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
 -    public void describe_schema_versions(org.apache.thrift.async.AsyncMethodCallback<AsyncClient.describe_schema_versions_call> resultHandler) throws org.apache.thrift.TException;
 +    public void truncate(String cfname, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
 -    public void describe_keyspaces(org.apache.thrift.async.AsyncMethodCallback<AsyncClient.describe_keyspaces_call> resultHandler) throws org.apache.thrift.TException;
 +    public void describe_schema_versions(org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
 -    public void describe_cluster_name(org.apache.thrift.async.AsyncMethodCallback<AsyncClient.describe_cluster_name_call> resultHandler) throws org.apache.thrift.TException;
 +    public void describe_keyspaces(org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
 -    public void describe_version(org.apache.thrift.async.AsyncMethodCallback<AsyncClient.describe_version_call> resultHandler) throws org.apache.thrift.TException;
 +    public void describe_cluster_name(org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
 -    public void describe_ring(String keyspace, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.describe_ring_call> resultHandler) throws org.apache.thrift.TException;
 +    public void describe_version(org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
 -    public void describe_local_ring(String keyspace, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.describe_local_ring_call> resultHandler) throws org.apache.thrift.TException;
 +    public void describe_ring(String keyspace, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
 -    public void describe_token_map(org.apache.thrift.async.AsyncMethodCallback<AsyncClient.describe_token_map_call> resultHandler) throws org.apache.thrift.TException;
++    public void describe_local_ring(String keyspace, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+ 
 -    public void describe_partitioner(org.apache.thrift.async.AsyncMethodCallback<AsyncClient.describe_partitioner_call> resultHandler) throws org.apache.thrift.TException;
 +    public void describe_token_map(org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
 -    public void describe_snitch(org.apache.thrift.async.AsyncMethodCallback<AsyncClient.describe_snitch_call> resultHandler) throws org.apache.thrift.TException;
 +    public void describe_partitioner(org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
 -    public void describe_keyspace(String keyspace, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.describe_keyspace_call> resultHandler) throws org.apache.thrift.TException;
 +    public void describe_snitch(org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
 -    public void describe_splits(String cfName, String start_token, String end_token, int keys_per_split, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.describe_splits_call> resultHandler) throws org.apache.thrift.TException;
 +    public void describe_keyspace(String keyspace, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
 -    public void trace_next_query(org.apache.thrift.async.AsyncMethodCallback<AsyncClient.trace_next_query_call> resultHandler) throws org.apache.thrift.TException;
 +    public void describe_splits(String cfName, String start_token, String end_token, int keys_per_split, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
 -    public void describe_splits_ex(String cfName, String start_token, String end_token, int keys_per_split, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.describe_splits_ex_call> resultHandler) throws org.apache.thrift.TException;
 +    public void trace_next_query(org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
 -    public void system_add_column_family(CfDef cf_def, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.system_add_column_family_call> resultHandler) throws org.apache.thrift.TException;
 +    public void describe_splits_ex(String cfName, String start_token, String end_token, int keys_per_split, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
 -    public void system_drop_column_family(String column_family, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.system_drop_column_family_call> resultHandler) throws org.apache.thrift.TException;
 +    public void system_add_column_family(CfDef cf_def, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
 -    public void system_add_keyspace(KsDef ks_def, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.system_add_keyspace_call> resultHandler) throws org.apache.thrift.TException;
 +    public void system_drop_column_family(String column_family, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
 -    public void system_drop_keyspace(String keyspace, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.system_drop_keyspace_call> resultHandler) throws org.apache.thrift.TException;
 +    public void system_add_keyspace(KsDef ks_def, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
 -    public void system_update_keyspace(KsDef ks_def, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.system_update_keyspace_call> resultHandler) throws org.apache.thrift.TException;
 +    public void system_drop_keyspace(String keyspace, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
 -    public void system_update_column_family(CfDef cf_def, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.system_update_column_family_call> resultHandler) throws org.apache.thrift.TException;
 +    public void system_update_keyspace(KsDef ks_def, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
 -    public void execute_cql_query(ByteBuffer query, Compression compression, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.execute_cql_query_call> resultHandler) throws org.apache.thrift.TException;
 +    public void system_update_column_family(CfDef cf_def, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
 -    public void execute_cql3_query(ByteBuffer query, Compression compression, ConsistencyLevel consistency, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.execute_cql3_query_call> resultHandler) throws org.apache.thrift.TException;
 +    public void execute_cql_query(ByteBuffer query, Compression compression, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
 -    public void prepare_cql_query(ByteBuffer query, Compression compression, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.prepare_cql_query_call> resultHandler) throws org.apache.thrift.TException;
 +    public void execute_cql3_query(ByteBuffer query, Compression compression, ConsistencyLevel consistency, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
 -    public void prepare_cql3_query(ByteBuffer query, Compression compression, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.prepare_cql3_query_call> resultHandler) throws org.apache.thrift.TException;
 +    public void prepare_cql_query(ByteBuffer query, Compression compression, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
 -    public void execute_prepared_cql_query(int itemId, List<ByteBuffer> values, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.execute_prepared_cql_query_call> resultHandler) throws org.apache.thrift.TException;
 +    public void prepare_cql3_query(ByteBuffer query, Compression compression, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
 -    public void execute_prepared_cql3_query(int itemId, List<ByteBuffer> values, ConsistencyLevel consistency, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.execute_prepared_cql3_query_call> resultHandler) throws org.apache.thrift.TException;
 +    public void execute_prepared_cql_query(int itemId, List<ByteBuffer> values, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
 -    public void set_cql_version(String version, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.set_cql_version_call> resultHandler) throws org.apache.thrift.TException;
 +    public void execute_prepared_cql3_query(int itemId, List<ByteBuffer> values, ConsistencyLevel consistency, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
 +
 +    public void set_cql_version(String version, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
  
    }
  
@@@ -2689,7 -2581,39 +2724,39 @@@
        }
      }
  
 -    public void describe_local_ring(String keyspace, org.apache.thrift.async.AsyncMethodCallback<describe_local_ring_call> resultHandler) throws org.apache.thrift.TException {
++    public void describe_local_ring(String keyspace, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+       checkReady();
+       describe_local_ring_call method_call = new describe_local_ring_call(keyspace, resultHandler, this, ___protocolFactory, ___transport);
+       this.___currentMethod = method_call;
+       ___manager.call(method_call);
+     }
+ 
+     public static class describe_local_ring_call extends org.apache.thrift.async.TAsyncMethodCall {
+       private String keyspace;
 -      public describe_local_ring_call(String keyspace, org.apache.thrift.async.AsyncMethodCallback<describe_local_ring_call> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
++      public describe_local_ring_call(String keyspace, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+         super(client, protocolFactory, transport, resultHandler, false);
+         this.keyspace = keyspace;
+       }
+ 
+       public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+         prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("describe_local_ring", org.apache.thrift.protocol.TMessageType.CALL, 0));
+         describe_local_ring_args args = new describe_local_ring_args();
+         args.setKeyspace(keyspace);
+         args.write(prot);
+         prot.writeMessageEnd();
+       }
+ 
+       public List<TokenRange> getResult() throws InvalidRequestException, org.apache.thrift.TException {
+         if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+           throw new IllegalStateException("Method call not finished!");
+         }
+         org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+         org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+         return (new Client(prot)).recv_describe_local_ring();
+       }
+     }
+ 
 -    public void describe_token_map(org.apache.thrift.async.AsyncMethodCallback<describe_token_map_call> resultHandler) throws org.apache.thrift.TException {
 +    public void describe_token_map(org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
        checkReady();
        describe_token_map_call method_call = new describe_token_map_call(resultHandler, this, ___protocolFactory, ___transport);
        this.___currentMethod = method_call;
@@@ -4031,7 -3839,27 +4099,31 @@@
        }
      }
  
 -    private static class describe_local_ring<I extends Iface> extends org.apache.thrift.ProcessFunction<I, describe_local_ring_args> {
++    public static class describe_local_ring<I extends Iface> extends org.apache.thrift.ProcessFunction<I, describe_local_ring_args> {
+       public describe_local_ring() {
+         super("describe_local_ring");
+       }
+ 
 -      protected describe_local_ring_args getEmptyArgsInstance() {
++      public describe_local_ring_args getEmptyArgsInstance() {
+         return new describe_local_ring_args();
+       }
+ 
 -      protected describe_local_ring_result getResult(I iface, describe_local_ring_args args) throws org.apache.thrift.TException {
++      protected boolean isOneway() {
++        return false;
++      }
++
++      public describe_local_ring_result getResult(I iface, describe_local_ring_args args) throws org.apache.thrift.TException {
+         describe_local_ring_result result = new describe_local_ring_result();
+         try {
+           result.success = iface.describe_local_ring(args.keyspace);
+         } catch (InvalidRequestException ire) {
+           result.ire = ire;
+         }
+         return result;
+       }
+     }
+ 
 -    private static class describe_token_map<I extends Iface> extends org.apache.thrift.ProcessFunction<I, describe_token_map_args> {
 +    public static class describe_token_map<I extends Iface> extends org.apache.thrift.ProcessFunction<I, describe_token_map_args> {
        public describe_token_map() {
          super("describe_token_map");
        }
@@@ -4539,2942 -4287,201 +4631,3000 @@@
  
    }
  
 -  public static class login_args implements org.apache.thrift.TBase<login_args, login_args._Fields>, java.io.Serializable, Cloneable   {
 -    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("login_args");
 +  public static class AsyncProcessor<I extends AsyncIface> extends org.apache.thrift.TBaseAsyncProcessor<I> {
 +    private static final Logger LOGGER = LoggerFactory.getLogger(AsyncProcessor.class.getName());
 +    public AsyncProcessor(I iface) {
 +      super(iface, getProcessMap(new HashMap<String, org.apache.thrift.AsyncProcessFunction<I, ? extends org.apache.thrift.TBase, ?>>()));
 +    }
  
 -    private static final org.apache.thrift.protocol.TField AUTH_REQUEST_FIELD_DESC = new org.apache.thrift.protocol.TField("auth_request", org.apache.thrift.protocol.TType.STRUCT, (short)1);
 +    protected AsyncProcessor(I iface, Map<String,  org.apache.thrift.AsyncProcessFunction<I, ? extends  org.apache.thrift.TBase, ?>> processMap) {
 +      super(iface, getProcessMap(processMap));
 +    }
  
 -    public AuthenticationRequest auth_request; // required
 +    private static <I extends AsyncIface> Map<String,  org.apache.thrift.AsyncProcessFunction<I, ? extends  org.apache.thrift.TBase,?>> getProcessMap(Map<String,  org.apache.thrift.AsyncProcessFunction<I, ? extends  org.apache.thrift.TBase, ?>> processMap) {
 +      processMap.put("login", new login());
 +      processMap.put("set_keyspace", new set_keyspace());
 +      processMap.put("get", new get());
 +      processMap.put("get_slice", new get_slice());
 +      processMap.put("get_count", new get_count());
 +      processMap.put("multiget_slice", new multiget_slice());
 +      processMap.put("multiget_count", new multiget_count());
 +      processMap.put("get_range_slices", new get_range_slices());
 +      processMap.put("get_paged_slice", new get_paged_slice());
 +      processMap.put("get_indexed_slices", new get_indexed_slices());
 +      processMap.put("insert", new insert());
 +      processMap.put("add", new add());
 +      processMap.put("cas", new cas());
 +      processMap.put("remove", new remove());
 +      processMap.put("remove_counter", new remove_counter());
 +      processMap.put("batch_mutate", new batch_mutate());
 +      processMap.put("atomic_batch_mutate", new atomic_batch_mutate());
 +      processMap.put("truncate", new truncate());
 +      processMap.put("describe_schema_versions", new describe_schema_versions());
 +      processMap.put("describe_keyspaces", new describe_keyspaces());
 +      processMap.put("describe_cluster_name", new describe_cluster_name());
 +      processMap.put("describe_version", new describe_version());
 +      processMap.put("describe_ring", new describe_ring());
++      processMap.put("describe_local_ring", new describe_local_ring());
 +      processMap.put("describe_token_map", new describe_token_map());
 +      processMap.put("describe_partitioner", new describe_partitioner());
 +      processMap.put("describe_snitch", new describe_snitch());
 +      processMap.put("describe_keyspace", new describe_keyspace());
 +      processMap.put("describe_splits", new describe_splits());
 +      processMap.put("trace_next_query", new trace_next_query());
 +      processMap.put("describe_splits_ex", new describe_splits_ex());
 +      processMap.put("system_add_column_family", new system_add_column_family());
 +      processMap.put("system_drop_column_family", new system_drop_column_family());
 +      processMap.put("system_add_keyspace", new system_add_keyspace());
 +      processMap.put("system_drop_keyspace", new system_drop_keyspace());
 +      processMap.put("system_update_keyspace", new system_update_keyspace());
 +      processMap.put("system_update_column_family", new system_update_column_family());
 +      processMap.put("execute_cql_query", new execute_cql_query());
 +      processMap.put("execute_cql3_query", new execute_cql3_query());
 +      processMap.put("prepare_cql_query", new prepare_cql_query());
 +      processMap.put("prepare_cql3_query", new prepare_cql3_query());
 +      processMap.put("execute_prepared_cql_query", new execute_prepared_cql_query());
 +      processMap.put("execute_prepared_cql3_query", new execute_prepared_cql3_query());
 +      processMap.put("set_cql_version", new set_cql_version());
 +      return processMap;
 +    }
  
 -    /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
 -    public enum _Fields implements org.apache.thrift.TFieldIdEnum {
 -      AUTH_REQUEST((short)1, "auth_request");
 +    public static class login<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, login_args, Void> {
 +      public login() {
 +        super("login");
 +      }
  
 -      private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 +      public login_args getEmptyArgsInstance() {
 +        return new login_args();
 +      }
  
 -      static {
 -        for (_Fields field : EnumSet.allOf(_Fields.class)) {
 -          byName.put(field.getFieldName(), field);
 -        }
 +      public AsyncMethodCallback<Void> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
 +        final org.apache.thrift.AsyncProcessFunction fcall = this;
 +        return new AsyncMethodCallback<Void>() { 
 +          public void onComplete(Void o) {
 +            login_result result = new login_result();
 +            try {
 +              fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
 +              return;
 +            } catch (Exception e) {
 +              LOGGER.error("Exception writing to internal frame buffer", e);
 +            }
 +            fb.close();
 +          }
 +          public void onError(Exception e) {
 +            byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
 +            org.apache.thrift.TBase msg;
 +            login_result result = new login_result();
 +            if (e instanceof AuthenticationException) {
 +                        result.authnx = (AuthenticationException) e;
 +                        result.setAuthnxIsSet(true);
 +                        msg = result;
 +            }
 +            else             if (e instanceof AuthorizationException) {
 +                        result.authzx = (AuthorizationException) e;
 +                        result.setAuthzxIsSet(true);
 +                        msg = result;
 +            }
 +             else 
 +            {
 +              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
 +              msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
 +            }
 +            try {
 +              fcall.sendResponse(fb,msg,msgType,seqid);
 +              return;
 +            } catch (Exception ex) {
 +              LOGGER.error("Exception writing to internal frame buffer", ex);
 +            }
 +            fb.close();
 +          }
 +        };
        }
  
 -      /**
 -       * Find the _Fields constant that matches fieldId, or null if its not found.
 -       */
 -      public static _Fields findByThriftId(int fieldId) {
 -        switch(fieldId) {
 -          case 1: // AUTH_REQUEST
 -            return AUTH_REQUEST;
 -          default:
 -            return null;
 -        }
 +      protected boolean isOneway() {
 +        return false;
        }
  
 -      /**
 -       * 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;
 +      public void start(I iface, login_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws TException {
 +        iface.login(args.auth_request,resultHandler);
        }
 +    }
  
 -      /**
 -       * Find the _Fields constant that matches name, or null if its not found.
 -       */
 -      public static _Fields findByName(String name) {
 -        return byName.get(name);
 +    public static class set_keyspace<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, set_keyspace_args, Void> {
 +      public set_keyspace() {
 +        super("set_keyspace");
        }
  
 -      private final short _thriftId;
 -      private final String _fieldName;
 +      public set_keyspace_args getEmptyArgsInstance() {
 +        return new set_keyspace_args();
 +      }
  
 -      _Fields(short thriftId, String fieldName) {
 -        _thriftId = thriftId;
 -        _fieldName = fieldName;
 +      public AsyncMethodCallback<Void> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
 +        final org.apache.thrift.AsyncProcessFunction fcall = this;
 +        return new AsyncMethodCallback<Void>() { 
 +          public void onComplete(Void o) {
 +            set_keyspace_result result = new set_keyspace_result();
 +            try {
 +              fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
 +              return;
 +            } catch (Exception e) {
 +              LOGGER.error("Exception writing to internal frame buffer", e);
 +            }
 +            fb.close();
 +          }
 +          public void onError(Exception e) {
 +            byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
 +            org.apache.thrift.TBase msg;
 +            set_keyspace_result result = new set_keyspace_result();
 +            if (e instanceof InvalidRequestException) {
 +                        result.ire = (InvalidRequestException) e;
 +                        result.setIreIsSet(true);
 +                        msg = result;
 +            }
 +             else 
 +            {
 +              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
 +              msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
 +            }
 +            try {
 +              fcall.sendResponse(fb,msg,msgType,seqid);
 +              return;
 +            } catch (Exception ex) {
 +              LOGGER.error("Exception writing to internal frame buffer", ex);
 +            }
 +            fb.close();
 +          }
 +        };
        }
  
 -      public short getThriftFieldId() {
 -        return _thriftId;
 +      protected boolean isOneway() {
 +        return false;
        }
  
 -      public String getFieldName() {
 -        return _fieldName;
 +      public void start(I iface, set_keyspace_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws TException {
 +        iface.set_keyspace(args.keyspace,resultHandler);
        }
      }
  
 -    // isset id assignments
 -
 -    public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
 -    static {
 -      Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
 -      tmpMap.put(_Fields.AUTH_REQUEST, new org.apache.thrift.meta_data.FieldMetaData("auth_request", org.apache.thrift.TFieldRequirementType.REQUIRED, 
 -          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, AuthenticationRequest.class)));
 -      metaDataMap = Collections.unmodifiableMap(tmpMap);
 -      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(login_args.class, metaDataMap);
 -    }
 +    public static class get<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, get_args, ColumnOrSuperColumn> {
 +      public get() {
 +        super("get");
 +      }
  
 -    public login_args() {
 -    }
 +      public get_args getEmptyArgsInstance() {
 +        return new get_args();
 +      }
  
 -    public login_args(
 -      AuthenticationRequest auth_request)
 -    {
 -      this();
 -      this.auth_request = auth_request;
 -    }
 +      public AsyncMethodCallback<ColumnOrSuperColumn> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
 +        final org.apache.thrift.AsyncProcessFunction fcall = this;
 +        return new AsyncMethodCallback<ColumnOrSuperColumn>() { 
 +          public void onComplete(ColumnOrSuperColumn o) {
 +            get_result result = new get_result();
 +            result.success = o;
 +            try {
 +              fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
 +              return;
 +            } catch (Exception e) {
 +              LOGGER.error("Exception writing to internal frame buffer", e);
 +            }
 +            fb.close();
 +          }
 +          public void onError(Exception e) {
 +            byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
 +            org.apache.thrift.TBase msg;
 +            get_result result = new get_result();
 +            if (e instanceof InvalidRequestException) {
 +                        result.ire = (InvalidRequestException) e;
 +                        result.setIreIsSet(true);
 +                        msg = result;
 +            }
 +            else             if (e instanceof NotFoundException) {
 +                        result.nfe = (NotFoundException) e;
 +                        result.setNfeIsSet(true);
 +                        msg = result;
 +            }
 +            else             if (e instanceof UnavailableException) {
 +                        result.ue = (UnavailableException) e;
 +                        result.setUeIsSet(true);
 +                        msg = result;
 +            }
 +            else             if (e instanceof TimedOutException) {
 +                        result.te = (TimedOutException) e;
 +                        result.setTeIsSet(true);
 +                        msg = result;
 +            }
 +             else 
 +            {
 +              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
 +              msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
 +            }
 +            try {
 +              fcall.sendResponse(fb,msg,msgType,seqid);
 +              return;
 +            } catch (Exception ex) {
 +              LOGGER.error("Exception writing to internal frame buffer", ex);
 +            }
 +            fb.close();
 +          }
 +        };
 +      }
  
 -    /**
 -     * Performs a deep copy on <i>other</i>.
 -     */
 -    public login_args(login_args other) {
 -      if (other.isSetAuth_request()) {
 -        this.auth_request = new AuthenticationRequest(other.auth_request);
 +      protected boolean isOneway() {
 +        return false;
        }
 -    }
  
 -    public login_args deepCopy() {
 -      return new login_args(this);
 +      public void start(I iface, get_args args, org.apache.thrift.async.AsyncMethodCallback<ColumnOrSuperColumn> resultHandler) throws TException {
 +        iface.get(args.key, args.column_path, args.consistency_level,resultHandler);
 +      }
      }
  
 -    @Override
 -    public void clear() {
 -      this.auth_request = null;
 -    }
 +    public static class get_slice<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, get_slice_args, List<ColumnOrSuperColumn>> {
 +      public get_slice() {
 +        super("get_slice");
 +      }
  
 -    public AuthenticationRequest getAuth_request() {
 -      return this.auth_request;
 -    }
 +      public get_slice_args getEmptyArgsInstance() {
 +        return new get_slice_args();
 +      }
  
 -    public login_args setAuth_request(AuthenticationRequest auth_request) {
 -      this.auth_request = auth_request;
 -      return this;
 -    }
 +      public AsyncMethodCallback<List<ColumnOrSuperColumn>> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
 +        final org.apache.thrift.AsyncProcessFunction fcall = this;
 +        return new AsyncMethodCallback<List<ColumnOrSuperColumn>>() { 
 +          public void onComplete(List<ColumnOrSuperColumn> o) {
 +            get_slice_result result = new get_slice_result();
 +            result.success = o;
 +            try {
 +              fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
 +              return;
 +            } catch (Exception e) {
 +              LOGGER.error("Exception writing to internal frame buffer", e);
 +            }
 +            fb.close();
 +          }
 +          public void onError(Exception e) {
 +            byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
 +            org.apache.thrift.TBase msg;
 +            get_slice_result result = new get_slice_result();
 +            if (e instanceof InvalidRequestException) {
 +                        result.ire = (InvalidRequestException) e;
 +                        result.setIreIsSet(true);
 +                        msg = result;
 +            }
 +            else             if (e instanceof UnavailableException) {
 +                        result.ue = (UnavailableException) e;
 +                        result.setUeIsSet(true);
 +                        msg = result;
 +            }
 +            else             if (e instanceof TimedOutException) {
 +                        result.te = (TimedOutException) e;
 +                        result.setTeIsSet(true);
 +                        msg = result;
 +            }
 +             else 
 +            {
 +              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
 +              msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
 +            }
 +            try {
 +              fcall.sendResponse(fb,msg,msgType,seqid);
 +              return;
 +            } catch (Exception ex) {
 +              LOGGER.error("Exception writing to internal frame buffer", ex);
 +            }
 +            fb.close();
 +          }
 +        };
 +      }
  
 -    public void unsetAuth_request() {
 -      this.auth_request = null;
 -    }
 +      protected boolean isOneway() {
 +        return false;
 +      }
  
 -    /** Returns true if field auth_request is set (has been assigned a value) and false otherwise */
 -    public boolean isSetAuth_request() {
 -      return this.auth_request != null;
 +      public void start(I iface, get_slice_args args, org.apache.thrift.async.AsyncMethodCallback<List<ColumnOrSuperColumn>> resultHandler) throws TException {
 +        iface.get_slice(args.key, args.column_parent, args.predicate, args.consistency_level,resultHandler);
 +      }
      }
  
 -    public void setAuth_requestIsSet(boolean value) {
 -      if (!value) {
 -        this.auth_request = null;
 +    public static class get_count<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, get_count_args, Integer> {
 +      public get_count() {
 +        super("get_count");
        }
 -    }
  
 -    public void setFieldValue(_Fields field, Object value) {
 -      switch (field) {
 -      case AUTH_REQUEST:
 -        if (value == null) {
 -          unsetAuth_request();
 -        } else {
 -          setAuth_request((AuthenticationRequest)value);
 -        }
 -        break;
 +      public get_count_args getEmptyArgsInstance() {
 +        return new get_count_args();
 +      }
  
 +      public AsyncMethodCallback<Integer> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
 +        final org.apache.thrift.AsyncProcessFunction fcall = this;
 +        return new AsyncMethodCallback<Integer>() { 
 +          public void onComplete(Integer o) {
 +            get_count_result result = new get_count_result();
 +            result.success = o;
 +            result.setSuccessIsSet(true);
 +            try {
 +              fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
 +              return;
 +            } catch (Exception e) {
 +              LOGGER.error("Exception writing to internal frame buffer", e);
 +            }
 +            fb.close();
 +          }
 +          public void onError(Exception e) {
 +            byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
 +            org.apache.thrift.TBase msg;
 +            get_count_result result = new get_count_result();
 +            if (e instanceof InvalidRequestException) {
 +                        result.ire = (InvalidRequestException) e;
 +                        result.setIreIsSet(true);
 +                        msg = result;
 +            }
 +            else             if (e instanceof UnavailableException) {
 +                        result.ue = (UnavailableException) e;
 +                        result.setUeIsSet(true);
 +                        msg = result;
 +            }
 +            else             if (e instanceof TimedOutException) {
 +                        result.te = (TimedOutException) e;
 +                        result.setTeIsSet(true);
 +                        msg = result;
 +            }
 +             else 
 +            {
 +              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
 +              msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
 +            }
 +            try {
 +              fcall.sendResponse(fb,msg,msgType,seqid);
 +              return;
 +            } catch (Exception ex) {
 +              LOGGER.error("Exception writing to internal frame buffer", ex);
 +            }
 +            fb.close();
 +          }
 +        };
        }
 -    }
  
 -    public Object getFieldValue(_Fields field) {
 -      switch (field) {
 -      case AUTH_REQUEST:
 -        return getAuth_request();
 +      protected boolean isOneway() {
 +        return false;
 +      }
  
 +      public void start(I iface, get_count_args args, org.apache.thrift.async.AsyncMethodCallback<Integer> resultHandler) throws TException {
 +        iface.get_count(args.key, args.column_parent, args.predicate, args.consistency_level,resultHandler);
        }
 -      throw new IllegalStateException();
      }
  
 -    /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
 -    public boolean isSet(_Fields field) {
 -      if (field == null) {
 -        throw new IllegalArgumentException();
 +    public static class multiget_slice<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, multiget_slice_args, Map<ByteBuffer,List<ColumnOrSuperColumn>>> {
 +      public multiget_slice() {
 +        super("multiget_slice");
        }
  
 -      switch (field) {
 -      case AUTH_REQUEST:
 -        return isSetAuth_request();
 +      public multiget_slice_args getEmptyArgsInstance() {
 +        return new multiget_slice_args();
        }
 -      throw new IllegalStateException();
 -    }
  
 -    @Override
 -    public boolean equals(Object that) {
 -      if (that == null)
 -        return false;
 -      if (that instanceof login_args)
 -        return this.equals((login_args)that);
 -      return false;
 -    }
 +      public AsyncMethodCallback<Map<ByteBuffer,List<ColumnOrSuperColumn>>> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
 +        final org.apache.thrift.AsyncProcessFunction fcall = this;
 +        return new AsyncMethodCallback<Map<ByteBuffer,List<ColumnOrSuperColumn>>>() { 
 +          public void onComplete(Map<ByteBuffer,List<ColumnOrSuperColumn>> o) {
 +            multiget_slice_result result = new multiget_slice_result();
 +            result.success = o;
 +            try {
 +              fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
 +              return;
 +            } catch (Exception e) {
 +              LOGGER.error("Exception writing to internal frame buffer", e);
 +            }
 +            fb.close();
 +          }
 +          public void onError(Exception e) {
 +            byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
 +            org.apache.thrift.TBase msg;
 +            multiget_slice_result result = new multiget_slice_result();
 +            if (e instanceof InvalidRequestException) {
 +                        result.ire = (InvalidRequestException) e;
 +                        result.setIreIsSet(true);
 +                        msg = result;
 +            }
 +            else             if (e instanceof UnavailableException) {
 +                        result.ue = (UnavailableException) e;
 +                        result.setUeIsSet(true);
 +                        msg = result;
 +            }
 +            else             if (e instanceof TimedOutException) {
 +                        result.te = (TimedOutException) e;
 +                        result.setTeIsSet(true);
 +                        msg = result;
 +            }
 +             else 
 +            {
 +              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
 +              msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
 +            }
 +            try {
 +              fcall.sendResponse(fb,msg,msgType,seqid);
 +              return;
 +            } catch (Exception ex) {
 +              LOGGER.error("Exception writing to internal frame buffer", ex);
 +            }
 +            fb.close();
 +          }
 +        };
 +      }
  
 -    public boolean equals(login_args that) {
 -      if (that == null)
 +      protected boolean isOneway() {
          return false;
 -
 -      boolean this_present_auth_request = true && this.isSetAuth_request();
 -      boolean that_present_auth_request = true && that.isSetAuth_request();
 -      if (this_present_auth_request || that_present_auth_request) {
 -        if (!(this_present_auth_request && that_present_auth_request))
 -          return false;
 -        if (!this.auth_request.equals(that.auth_request))
 -          return false;
        }
  
 -      return true;
 +      public void start(I iface, multiget_slice_args args, org.apache.thrift.async.AsyncMethodCallback<Map<ByteBuffer,List<ColumnOrSuperColumn>>> resultHandler) throws TException {
 +        iface.multiget_slice(args.keys, args.column_parent, args.predicate, args.consistency_level,resultHandler);
 +      }
      }
  
 -    @Override
 -    public int hashCode() {
 -      HashCodeBuilder builder = new HashCodeBuilder();
 +    public static class multiget_count<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, multiget_count_args, Map<ByteBuffer,Integer>> {
 +      public multiget_count() {
 +        super("multiget_count");
 +      }
  
 -      boolean present_auth_request = true && (isSetAuth_request());
 -      builder.append(present_auth_request);
 -      if (present_auth_request)
 +      public multiget_count_args getEmptyArgsInstance() {
 +        return new multiget_count_args();
 +      }
 +
 +      public AsyncMethodCallback<Map<ByteBuffer,Integer>> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
 +        final org.apache.thrift.AsyncProcessFunction fcall = this;
 +        return new AsyncMethodCallback<Map<ByteBuffer,Integer>>() { 
 +          public void onComplete(Map<ByteBuffer,Integer> o) {
 +            multiget_count_result result = new multiget_count_result();
 +            result.success = o;
 +            try {
 +              fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
 +              return;
 +            } catch (Exception e) {
 +              LOGGER.error("Exception writing to internal frame buffer", e);
 +            }
 +            fb.close();
 +          }
 +          public void onError(Exception e) {
 +            byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
 +            org.apache.thrift.TBase msg;
 +            multiget_count_result result = new multiget_count_result();
 +            if (e instanceof InvalidRequestException) {
 +                        result.ire = (InvalidRequestException) e;
 +                        result.setIreIsSet(true);
 +                        msg = result;
 +            }
 +            else             if (e instanceof UnavailableException) {
 +                        result.ue = (UnavailableException) e;
 +                        result.setUeIsSet(true);
 +                        msg = result;
 +            }
 +            else             if (e instanceof TimedOutException) {
 +                        result.te = (TimedOutException) e;
 +                        result.setTeIsSet(true);
 +                        msg = result;
 +            }
 +             else 
 +            {
 +              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
 +              msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
 +            }
 +            try {
 +              fcall.sendResponse(fb,msg,msgType,seqid);
 +              return;
 +            } catch (Exception ex) {
 +              LOGGER.error("Exception writing to internal frame buffer", ex);
 +            }
 +            fb.close();
 +          }
 +        };
 +      }
 +
 +      protected boolean isOneway() {
 +        return false;
 +      }
 +
 +      public void start(I iface, multiget_count_args args, org.apache.thrift.async.AsyncMethodCallback<Map<ByteBuffer,Integer>> resultHandler) throws TException {
 +        iface.multiget_count(args.keys, args.column_parent, args.predicate, args.consistency_level,resultHandler);
 +      }
 +    }
 +
 +    public static class get_range_slices<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, get_range_slices_args, List<KeySlice>> {
 +      public get_range_slices() {
 +        super("get_range_slices");
 +      }
 +
 +      public get_range_slices_args getEmptyArgsInstance() {
 +        return new get_range_slices_args();
 +      }
 +
 +      public AsyncMethodCallback<List<KeySlice>> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
 +        final org.apache.thrift.AsyncProcessFunction fcall = this;
 +        return new AsyncMethodCallback<List<KeySlice>>() { 
 +          public void onComplete(List<KeySlice> o) {
 +            get_range_slices_result result = new get_range_slices_result();
 +            result.success = o;
 +            try {
 +              fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
 +              return;
 +            } catch (Exception e) {
 +              LOGGER.error("Exception writing to internal frame buffer", e);
 +            }
 +            fb.close();
 +          }
 +          public void onError(Exception e) {
 +            byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
 +            org.apache.thrift.TBase msg;
 +            get_range_slices_result result = new get_range_slices_result();
 +            if (e instanceof InvalidRequestException) {
 +                        result.ire = (InvalidRequestException) e;
 +                        result.setIreIsSet(true);
 +                        msg = result;
 +            }
 +            else             if (e instanceof UnavailableException) {
 +                        result.ue = (UnavailableException) e;
 +                        result.setUeIsSet(true);
 +                        msg = result;
 +            }
 +            else             if (e instanceof TimedOutException) {
 +                        result.te = (TimedOutException) e;
 +                        result.setTeIsSet(true);
 +                        msg = result;
 +            }
 +             else 
 +            {
 +              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
 +              msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
 +            }
 +            try {
 +              fcall.sendResponse(fb,msg,msgType,seqid);
 +              return;
 +            } catch (Exception ex) {
 +              LOGGER.error("Exception writing to internal frame buffer", ex);
 +            }
 +            fb.close();
 +          }
 +        };
 +      }
 +
 +      protected boolean isOneway() {
 +        return false;
 +      }
 +
 +      public void start(I iface, get_range_slices_args args, org.apache.thrift.async.AsyncMethodCallback<List<KeySlice>> resultHandler) throws TException {
 +        iface.get_range_slices(args.column_parent, args.predicate, args.range, args.consistency_level,resultHandler);
 +      }
 +    }
 +
 +    public static class get_paged_slice<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, get_paged_slice_args, List<KeySlice>> {
 +      public get_paged_slice() {
 +        super("get_paged_slice");
 +      }
 +
 +      public get_paged_slice_args getEmptyArgsInstance() {
 +        return new get_paged_slice_args();
 +      }
 +
 +      public AsyncMethodCallback<List<KeySlice>> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
 +        final org.apache.thrift.AsyncProcessFunction fcall = this;
 +        return new AsyncMethodCallback<List<KeySlice>>() { 
 +          public void onComplete(List<KeySlice> o) {
 +            get_paged_slice_result result = new get_paged_slice_result();
 +            result.success = o;
 +            try {
 +              fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
 +              return;
 +            } catch (Exception e) {
 +              LOGGER.error("Exception writing to internal frame buffer", e);
 +            }
 +            fb.close();
 +          }
 +          public void onError(Exception e) {
 +            byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
 +            org.apache.thrift.TBase msg;
 +            get_paged_slice_result result = new get_paged_slice_result();
 +            if (e instanceof InvalidRequestException) {
 +                        result.ire = (InvalidRequestException) e;
 +                        result.setIreIsSet(true);
 +                        msg = result;
 +            }
 +            else             if (e instanceof UnavailableException) {
 +                        result.ue = (UnavailableException) e;
 +                        result.setUeIsSet(true);
 +                        msg = result;
 +            }
 +            else             if (e instanceof TimedOutException) {
 +                        result.te = (TimedOutException) e;
 +                        result.setTeIsSet(true);
 +                        msg = result;
 +            }
 +             else 
 +            {
 +              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
 +              msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
 +            }
 +            try {
 +              fcall.sendResponse(fb,msg,msgType,seqid);
 +              return;
 +            } catch (Exception ex) {
 +              LOGGER.error("Exception writing to internal frame buffer", ex);
 +            }
 +            fb.close();
 +          }
 +        };
 +      }
 +
 +      protected boolean isOneway() {
 +        return false;
 +      }
 +
 +      public void start(I iface, get_paged_slice_args args, org.apache.thrift.async.AsyncMethodCallback<List<KeySlice>> resultHandler) throws TException {
 +        iface.get_paged_slice(args.column_family, args.range, args.start_column, args.consistency_level,resultHandler);
 +      }
 +    }
 +
 +    public static class get_indexed_slices<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, get_indexed_slices_args, List<KeySlice>> {
 +      public get_indexed_slices() {
 +        super("get_indexed_slices");
 +      }
 +
 +      public get_indexed_slices_args getEmptyArgsInstance() {
 +        return new get_indexed_slices_args();
 +      }
 +
 +      public AsyncMethodCallback<List<KeySlice>> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
 +        final org.apache.thrift.AsyncProcessFunction fcall = this;
 +        return new AsyncMethodCallback<List<KeySlice>>() { 
 +          public void onComplete(List<KeySlice> o) {
 +            get_indexed_slices_result result = new get_indexed_slices_result();
 +            result.success = o;
 +            try {
 +              fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
 +              return;
 +            } catch (Exception e) {
 +              LOGGER.error("Exception writing to internal frame buffer", e);
 +            }
 +            fb.close();
 +          }
 +          public void onError(Exception e) {
 +            byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
 +            org.apache.thrift.TBase msg;
 +            get_indexed_slices_result result = new get_indexed_slices_result();
 +            if (e instanceof InvalidRequestException) {
 +                        result.ire = (InvalidRequestException) e;
 +                        result.setIreIsSet(true);
 +                        msg = result;
 +            }
 +            else             if (e instanceof UnavailableException) {
 +                        result.ue = (UnavailableException) e;
 +                        result.setUeIsSet(true);
 +                        msg = result;
 +            }
 +            else             if (e instanceof TimedOutException) {
 +                        result.te = (TimedOutException) e;
 +                        result.setTeIsSet(true);
 +                        msg = result;
 +            }
 +             else 
 +            {
 +              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
 +              msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
 +            }
 +            try {
 +              fcall.sendResponse(fb,msg,msgType,seqid);
 +              return;
 +            } catch (Exception ex) {
 +              LOGGER.error("Exception writing to internal frame buffer", ex);
 +            }
 +            fb.close();
 +          }
 +        };
 +      }
 +
 +      protected boolean isOneway() {
 +        return false;
 +      }
 +
 +      public void start(I iface, get_indexed_slices_args args, org.apache.thrift.async.AsyncMethodCallback<List<KeySlice>> resultHandler) throws TException {
 +        iface.get_indexed_slices(args.column_parent, args.index_clause, args.column_predicate, args.consistency_level,resultHandler);
 +      }
 +    }
 +
 +    public static class insert<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, insert_args, Void> {
 +      public insert() {
 +        super("insert");
 +      }
 +
 +      public insert_args getEmptyArgsInstance() {
 +        return new insert_args();
 +      }
 +
 +      public AsyncMethodCallback<Void> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
 +        final org.apache.thrift.AsyncProcessFunction fcall = this;
 +        return new AsyncMethodCallback<Void>() { 
 +          public void onComplete(Void o) {
 +            insert_result result = new insert_result();
 +            try {
 +              fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
 +              return;
 +            } catch (Exception e) {
 +              LOGGER.error("Exception writing to internal frame buffer", e);
 +            }
 +            fb.close();
 +          }
 +          public void onError(Exception e) {
 +            byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
 +            org.apache.thrift.TBase msg;
 +            insert_result result = new insert_result();
 +            if (e instanceof InvalidRequestException) {
 +                        result.ire = (InvalidRequestException) e;
 +                        result.setIreIsSet(true);
 +                        msg = result;
 +            }
 +            else             if (e instanceof UnavailableException) {
 +                        result.ue = (UnavailableException) e;
 +                        result.setUeIsSet(true);
 +                        msg = result;
 +            }
 +            else             if (e instanceof TimedOutException) {
 +                        result.te = (TimedOutException) e;
 +                        result.setTeIsSet(true);
 +                        msg = result;
 +            }
 +             else 
 +            {
 +              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
 +              msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
 +            }
 +            try {
 +              fcall.sendResponse(fb,msg,msgType,seqid);
 +              return;
 +            } catch (Exception ex) {
 +              LOGGER.error("Exception writing to internal frame buffer", ex);
 +            }
 +            fb.close();
 +          }
 +        };
 +      }
 +
 +      protected boolean isOneway() {
 +        return false;
 +      }
 +
 +      public void start(I iface, insert_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws TException {
 +        iface.insert(args.key, args.column_parent, args.column, args.consistency_level,resultHandler);
 +      }
 +    }
 +
 +    public static class add<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, add_args, Void> {
 +      public add() {
 +        super("add");
 +      }
 +
 +      public add_args getEmptyArgsInstance() {
 +        return new add_args();
 +      }
 +
 +      public AsyncMethodCallback<Void> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
 +        final org.apache.thrift.AsyncProcessFunction fcall = this;
 +        return new AsyncMethodCallback<Void>() { 
 +          public void onComplete(Void o) {
 +            add_result result = new add_result();
 +            try {
 +              fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
 +              return;
 +            } catch (Exception e) {
 +              LOGGER.error("Exception writing to internal frame buffer", e);
 +            }
 +            fb.close();
 +          }
 +          public void onError(Exception e) {
 +            byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
 +            org.apache.thrift.TBase msg;
 +            add_result result = new add_result();
 +            if (e instanceof InvalidRequestException) {
 +                        result.ire = (InvalidRequestException) e;
 +                        result.setIreIsSet(true);
 +                        msg = result;
 +            }
 +            else             if (e instanceof UnavailableException) {
 +                        result.ue = (UnavailableException) e;
 +                        result.setUeIsSet(true);
 +                        msg = result;
 +            }
 +            else             if (e instanceof TimedOutException) {
 +                        result.te = (TimedOutException) e;
 +                        result.setTeIsSet(true);
 +                        msg = result;
 +            }
 +             else 
 +            {
 +              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
 +              msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
 +            }
 +            try {
 +              fcall.sendResponse(fb,msg,msgType,seqid);
 +              return;
 +            } catch (Exception ex) {
 +              LOGGER.error("Exception writing to internal frame buffer", ex);
 +            }
 +            fb.close();
 +          }
 +        };
 +      }
 +
 +      protected boolean isOneway() {
 +        return false;
 +      }
 +
 +      public void start(I iface, add_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws TException {
 +        iface.add(args.key, args.column_parent, args.column, args.consistency_level,resultHandler);
 +      }
 +    }
 +
 +    public static class cas<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, cas_args, CASResult> {
 +      public cas() {
 +        super("cas");
 +      }
 +
 +      public cas_args getEmptyArgsInstance() {
 +        return new cas_args();
 +      }
 +
 +      public AsyncMethodCallback<CASResult> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
 +        final org.apache.thrift.AsyncProcessFunction fcall = this;
 +        return new AsyncMethodCallback<CASResult>() { 
 +          public void onComplete(CASResult o) {
 +            cas_result result = new cas_result();
 +            result.success = o;
 +            try {
 +              fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
 +              return;
 +            } catch (Exception e) {
 +              LOGGER.error("Exception writing to internal frame buffer", e);
 +            }
 +            fb.close();
 +          }
 +          public void onError(Exception e) {
 +            byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
 +            org.apache.thrift.TBase msg;
 +            cas_result result = new cas_result();
 +            if (e instanceof InvalidRequestException) {
 +                        result.ire = (InvalidRequestException) e;
 +                        result.setIreIsSet(true);
 +                        msg = result;
 +            }
 +            else             if (e instanceof UnavailableException) {
 +                        result.ue = (UnavailableException) e;
 +                        result.setUeIsSet(true);
 +                        msg = result;
 +            }
 +            else             if (e instanceof TimedOutException) {
 +                        result.te = (TimedOutException) e;
 +                        result.setTeIsSet(true);
 +                        msg = result;
 +            }
 +             else 
 +            {
 +              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
 +              msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
 +            }
 +            try {
 +              fcall.sendResponse(fb,msg,msgType,seqid);
 +              return;
 +            } catch (Exception ex) {
 +              LOGGER.error("Exception writing to internal frame buffer", ex);
 +            }
 +            fb.close();
 +          }
 +        };
 +      }
 +
 +      protected boolean isOneway() {
 +        return false;
 +      }
 +
 +      public void start(I iface, cas_args args, org.apache.thrift.async.AsyncMethodCallback<CASResult> resultHandler) throws TException {
 +        iface.cas(args.key, args.column_family, args.expected, args.updates, args.serial_consistency_level, args.commit_consistency_level,resultHandler);
 +      }
 +    }
 +
 +    public static class remove<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, remove_args, Void> {
 +      public remove() {
 +        super("remove");
 +      }
 +
 +      public remove_args getEmptyArgsInstance() {
 +        return new remove_args();
 +      }
 +
 +      public AsyncMethodCallback<Void> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
 +        final org.apache.thrift.AsyncProcessFunction fcall = this;
 +        return new AsyncMethodCallback<Void>() { 
 +          public void onComplete(Void o) {
 +            remove_result result = new remove_result();
 +            try {
 +              fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
 +              return;
 +            } catch (Exception e) {
 +              LOGGER.error("Exception writing to internal frame buffer", e);
 +            }
 +            fb.close();
 +          }
 +          public void onError(Exception e) {
 +            byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
 +            org.apache.thrift.TBase msg;
 +            remove_result result = new remove_result();
 +            if (e instanceof InvalidRequestException) {
 +                        result.ire = (InvalidRequestException) e;
 +                        result.setIreIsSet(true);
 +                        msg = result;
 +            }
 +            else             if (e instanceof UnavailableException) {
 +                        result.ue = (UnavailableException) e;
 +                        result.setUeIsSet(true);
 +                        msg = result;
 +            }
 +            else             if (e instanceof TimedOutException) {
 +                        result.te = (TimedOutException) e;
 +                        result.setTeIsSet(true);
 +                        msg = result;
 +            }
 +             else 
 +            {
 +              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
 +              msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
 +            }
 +            try {
 +              fcall.sendResponse(fb,msg,msgType,seqid);
 +              return;
 +            } catch (Exception ex) {
 +              LOGGER.error("Exception writing to internal frame buffer", ex);
 +            }
 +            fb.close();
 +          }
 +        };
 +      }
 +
 +      protected boolean isOneway() {
 +        return false;
 +      }
 +
 +      public void start(I iface, remove_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws TException {
 +        iface.remove(args.key, args.column_path, args.timestamp, args.consistency_level,resultHandler);
 +      }
 +    }
 +
 +    public static class remove_counter<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, remove_counter_args, Void> {
 +      public remove_counter() {
 +        super("remove_counter");
 +      }
 +
 +      public remove_counter_args getEmptyArgsInstance() {
 +        return new remove_counter_args();
 +      }
 +
 +      public AsyncMethodCallback<Void> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
 +        final org.apache.thrift.AsyncProcessFunction fcall = this;
 +        return new AsyncMethodCallback<Void>() { 
 +          public void onComplete(Void o) {
 +            remove_counter_result result = new remove_counter_result();
 +            try {
 +              fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
 +              return;
 +            } catch (Exception e) {
 +              LOGGER.error("Exception writing to internal frame buffer", e);
 +            }
 +            fb.close();
 +          }
 +          public void onError(Exception e) {
 +            byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
 +            org.apache.thrift.TBase msg;
 +            remove_counter_result result = new remove_counter_result();
 +            if (e instanceof InvalidRequestException) {
 +                        result.ire = (InvalidRequestException) e;
 +                        result.setIreIsSet(true);
 +                        msg = result;
 +            }
 +            else             if (e instanceof UnavailableException) {
 +                        result.ue = (UnavailableException) e;
 +                        result.setUeIsSet(true);
 +                        msg = result;
 +            }
 +            else             if (e instanceof TimedOutException) {
 +                        result.te = (TimedOutException) e;
 +                        result.setTeIsSet(true);
 +                        msg = result;
 +            }
 +             else 
 +            {
 +              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
 +              msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
 +            }
 +            try {
 +              fcall.sendResponse(fb,msg,msgType,seqid);
 +              return;
 +            } catch (Exception ex) {
 +              LOGGER.error("Exception writing to internal frame buffer", ex);
 +            }
 +            fb.close();
 +          }
 +        };
 +      }
 +
 +      protected boolean isOneway() {
 +        return false;
 +      }
 +
 +      public void start(I iface, remove_counter_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws TException {
 +        iface.remove_counter(args.key, args.path, args.consistency_level,resultHandler);
 +      }
 +    }
 +
 +    public static class batch_mutate<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, batch_mutate_args, Void> {
 +      public batch_mutate() {
 +        super("batch_mutate");
 +      }
 +
 +      public batch_mutate_args getEmptyArgsInstance() {
 +        return new batch_mutate_args();
 +      }
 +
 +      public AsyncMethodCallback<Void> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
 +        final org.apache.thrift.AsyncProcessFunction fcall = this;
 +        return new AsyncMethodCallback<Void>() { 
 +          public void onComplete(Void o) {
 +            batch_mutate_result result = new batch_mutate_result();
 +            try {
 +              fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
 +              return;
 +            } catch (Exception e) {
 +              LOGGER.error("Exception writing to internal frame buffer", e);
 +            }
 +            fb.close();
 +          }
 +          public void onError(Exception e) {
 +            byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
 +            org.apache.thrift.TBase msg;
 +            batch_mutate_result result = new batch_mutate_result();
 +            if (e instanceof InvalidRequestException) {
 +                        result.ire = (InvalidRequestException) e;
 +                        result.setIreIsSet(true);
 +                        msg = result;
 +            }
 +            else             if (e instanceof UnavailableException) {
 +                        result.ue = (UnavailableException) e;
 +                        result.setUeIsSet(true);
 +                        msg = result;
 +            }
 +            else             if (e instanceof TimedOutException) {
 +                        result.te = (TimedOutException) e;
 +                        result.setTeIsSet(true);
 +                        msg = result;
 +            }
 +             else 
 +            {
 +              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
 +              msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
 +            }
 +            try {
 +              fcall.sendResponse(fb,msg,msgType,seqid);
 +              return;
 +            } catch (Exception ex) {
 +              LOGGER.error("Exception writing to internal frame buffer", ex);
 +            }
 +            fb.close();
 +          }
 +        };
 +      }
 +
 +      protected boolean isOneway() {
 +        return false;
 +      }
 +
 +      public void start(I iface, batch_mutate_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws TException {
 +        iface.batch_mutate(args.mutation_map, args.consistency_level,resultHandler);
 +      }
 +    }
 +
 +    public static class atomic_batch_mutate<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, atomic_batch_mutate_args, Void> {
 +      public atomic_batch_mutate() {
 +        super("atomic_batch_mutate");
 +      }
 +
 +      public atomic_batch_mutate_args getEmptyArgsInstance() {
 +        return new atomic_batch_mutate_args();
 +      }
 +
 +      public AsyncMethodCallback<Void> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
 +        final org.apache.thrift.AsyncProcessFunction fcall = this;
 +        return new AsyncMethodCallback<Void>() { 
 +          public void onComplete(Void o) {
 +            atomic_batch_mutate_result result = new atomic_batch_mutate_result();
 +            try {
 +              fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
 +              return;
 +            } catch (Exception e) {
 +              LOGGER.error("Exception writing to internal frame buffer", e);
 +            }
 +            fb.close();
 +          }
 +          public void onError(Exception e) {
 +            byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
 +            org.apache.thrift.TBase msg;
 +            atomic_batch_mutate_result result = new atomic_batch_mutate_result();
 +            if (e instanceof InvalidRequestException) {
 +                        result.ire = (InvalidRequestException) e;
 +                        result.setIreIsSet(true);
 +                        msg = result;
 +            }
 +            else             if (e instanceof UnavailableException) {
 +                        result.ue = (UnavailableException) e;
 +                        result.setUeIsSet(true);
 +                        msg = result;
 +            }
 +            else             if (e instanceof TimedOutException) {
 +                        result.te = (TimedOutException) e;
 +                        result.setTeIsSet(true);
 +                        msg = result;
 +            }
 +             else 
 +            {
 +              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
 +              msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
 +            }
 +            try {
 +              fcall.sendResponse(fb,msg,msgType,seqid);
 +              return;
 +            } catch (Exception ex) {
 +              LOGGER.error("Exception writing to internal frame buffer", ex);
 +            }
 +            fb.close();
 +          }
 +        };
 +      }
 +
 +      protected boolean isOneway() {
 +        return false;
 +      }
 +
 +      public void start(I iface, atomic_batch_mutate_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws TException {
 +        iface.atomic_batch_mutate(args.mutation_map, args.consistency_level,resultHandler);
 +      }
 +    }
 +
 +    public static class truncate<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, truncate_args, Void> {
 +      public truncate() {
 +        super("truncate");
 +      }
 +
 +      public truncate_args getEmptyArgsInstance() {
 +        return new truncate_args();
 +      }
 +
 +      public AsyncMethodCallback<Void> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
 +        final org.apache.thrift.AsyncProcessFunction fcall = this;
 +        return new AsyncMethodCallback<Void>() { 
 +          public void onComplete(Void o) {
 +            truncate_result result = new truncate_result();
 +            try {
 +              fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
 +              return;
 +            } catch (Exception e) {
 +              LOGGER.error("Exception writing to internal frame buffer", e);
 +            }
 +            fb.close();
 +          }
 +          public void onError(Exception e) {
 +            byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
 +            org.apache.thrift.TBase msg;
 +            truncate_result result = new truncate_result();
 +            if (e instanceof InvalidRequestException) {
 +                        result.ire = (InvalidRequestException) e;
 +                        result.setIreIsSet(true);
 +                        msg = result;
 +            }
 +            else             if (e instanceof UnavailableException) {
 +                        result.ue = (UnavailableException) e;
 +                        result.setUeIsSet(true);
 +                        msg = result;
 +            }
 +            else             if (e instanceof TimedOutException) {
 +                        result.te = (TimedOutException) e;
 +                        result.setTeIsSet(true);
 +                        msg = result;
 +            }
 +             else 
 +            {
 +              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
 +              msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
 +            }
 +            try {
 +              fcall.sendResponse(fb,msg,msgType,seqid);
 +              return;
 +            } catch (Exception ex) {
 +              LOGGER.error("Exception writing to internal frame buffer", ex);
 +            }
 +            fb.close();
 +          }
 +        };
 +      }
 +
 +      protected boolean isOneway() {
 +        return false;
 +      }
 +
 +      public void start(I iface, truncate_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws TException {
 +        iface.truncate(args.cfname,resultHandler);
 +      }
 +    }
 +
 +    public static class describe_schema_versions<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, describe_schema_versions_args, Map<String,List<String>>> {
 +      public describe_schema_versions() {
 +        super("describe_schema_versions");
 +      }
 +
 +      public describe_schema_versions_args getEmptyArgsInstance() {
 +        return new describe_schema_versions_args();
 +      }
 +
 +      public AsyncMethodCallback<Map<String,List<String>>> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
 +        final org.apache.thrift.AsyncProcessFunction fcall = this;
 +        return new AsyncMethodCallback<Map<String,List<String>>>() { 
 +          public void onComplete(Map<String,List<String>> o) {
 +            describe_schema_versions_result result = new describe_schema_versions_result();
 +            result.success = o;
 +            try {
 +              fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
 +              return;
 +            } catch (Exception e) {
 +              LOGGER.error("Exception writing to internal frame buffer", e);
 +            }
 +            fb.close();
 +          }
 +          public void onError(Exception e) {
 +            byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
 +            org.apache.thrift.TBase msg;
 +            describe_schema_versions_result result = new describe_schema_versions_result();
 +            if (e instanceof InvalidRequestException) {
 +                        result.ire = (InvalidRequestException) e;
 +                        result.setIreIsSet(true);
 +                        msg = result;
 +            }
 +             else 
 +            {
 +              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
 +              msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
 +            }
 +            try {
 +              fcall.sendResponse(fb,msg,msgType,seqid);
 +              return;
 +            } catch (Exception ex) {
 +              LOGGER.error("Exception writing to internal frame buffer", ex);
 +            }
 +            fb.close();
 +          }
 +        };
 +      }
 +
 +      protected boolean isOneway() {
 +        return false;
 +      }
 +
 +      public void start(I iface, describe_schema_versions_args args, org.apache.thrift.async.AsyncMethodCallback<Map<String,List<String>>> resultHandler) throws TException {
 +        iface.describe_schema_versions(resultHandler);
 +      }
 +    }
 +
 +    public static class describe_keyspaces<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, describe_keyspaces_args, List<KsDef>> {
 +      public describe_keyspaces() {
 +        super("describe_keyspaces");
 +      }
 +
 +      public describe_keyspaces_args getEmptyArgsInstance() {
 +        return new describe_keyspaces_args();
 +      }
 +
 +      public AsyncMethodCallback<List<KsDef>> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
 +        final org.apache.thrift.AsyncProcessFunction fcall = this;
 +        return new AsyncMethodCallback<List<KsDef>>() { 
 +          public void onComplete(List<KsDef> o) {
 +            describe_keyspaces_result result = new describe_keyspaces_result();
 +            result.success = o;
 +            try {
 +              fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
 +              return;
 +            } catch (Exception e) {
 +              LOGGER.error("Exception writing to internal frame buffer", e);
 +            }
 +            fb.close();
 +          }
 +          public void onError(Exception e) {
 +            byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
 +            org.apache.thrift.TBase msg;
 +            describe_keyspaces_result result = new describe_keyspaces_result();
 +            if (e instanceof InvalidRequestException) {
 +                        result.ire = (InvalidRequestException) e;
 +                        result.setIreIsSet(true);
 +                        msg = result;
 +            }
 +             else 
 +            {
 +              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
 +              msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
 +            }
 +            try {
 +              fcall.sendResponse(fb,msg,msgType,seqid);
 +              return;
 +            } catch (Exception ex) {
 +              LOGGER.error("Exception writing to internal frame buffer", ex);
 +            }
 +            fb.close();
 +          }
 +        };
 +      }
 +
 +      protected boolean isOneway() {
 +        return false;
 +      }
 +
 +      publi

<TRUNCATED>

[16/16] git commit: Merge branch 'cassandra-2.0' into trunk

Posted by jb...@apache.org.
Merge branch 'cassandra-2.0' into trunk


Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/e6170fef
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/e6170fef
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/e6170fef

Branch: refs/heads/trunk
Commit: e6170fef991dfad99cb6e094498c5a10fe129592
Parents: c00e05b f2a82ee
Author: Jonathan Ellis <jb...@apache.org>
Authored: Fri Dec 6 09:52:08 2013 -0600
Committer: Jonathan Ellis <jb...@apache.org>
Committed: Fri Dec 6 09:52:08 2013 -0600

----------------------------------------------------------------------
 CHANGES.txt                                     |    1 +
 interface/cassandra.thrift                      |    5 +
 .../org/apache/cassandra/thrift/Cassandra.java  | 2250 +++++++++++++-----
 .../cassandra/thrift/cassandraConstants.java    |    2 +-
 .../hadoop/AbstractColumnFamilyInputFormat.java |   10 +-
 .../cassandra/service/StorageService.java       |   69 +-
 .../cassandra/thrift/CassandraServer.java       |   13 +
 7 files changed, 1728 insertions(+), 622 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/e6170fef/CHANGES.txt
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e6170fef/src/java/org/apache/cassandra/hadoop/AbstractColumnFamilyInputFormat.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/hadoop/AbstractColumnFamilyInputFormat.java
index 9e0a801,6c45a35..746666b
--- a/src/java/org/apache/cassandra/hadoop/AbstractColumnFamilyInputFormat.java
+++ b/src/java/org/apache/cassandra/hadoop/AbstractColumnFamilyInputFormat.java
@@@ -132,8 -127,9 +127,9 @@@ public abstract class AbstractColumnFam
          keyspace = ConfigHelper.getInputKeyspace(context.getConfiguration());
          cfName = ConfigHelper.getInputColumnFamily(context.getConfiguration());
          partitioner = ConfigHelper.getInputPartitioner(context.getConfiguration());
 -        logger.debug("partitioner is " + partitioner);
 +        logger.debug("partitioner is {}", partitioner);
  
+ 
          // cannonical ranges, split into pieces, fetching the splits in parallel
          ExecutorService executor = new ThreadPoolExecutor(0, 128, 60L, TimeUnit.SECONDS, new LinkedBlockingQueue<Runnable>());
          List<InputSplit> splits = new ArrayList<InputSplit>();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e6170fef/src/java/org/apache/cassandra/service/StorageService.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/service/StorageService.java
index c16d906,c341684..e1392c6
--- a/src/java/org/apache/cassandra/service/StorageService.java
+++ b/src/java/org/apache/cassandra/service/StorageService.java
@@@ -38,7 -37,9 +38,8 @@@ import javax.management.ObjectName
  import static java.nio.charset.StandardCharsets.ISO_8859_1;
  
  import com.google.common.annotations.VisibleForTesting;
+ import com.google.common.base.Predicate;
  import com.google.common.collect.*;
 -import com.google.common.util.concurrent.AtomicDouble;
  import com.google.common.util.concurrent.FutureCallback;
  import com.google.common.util.concurrent.Futures;
  import com.google.common.util.concurrent.Uninterruptibles;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e6170fef/src/java/org/apache/cassandra/thrift/CassandraServer.java
----------------------------------------------------------------------


[07/16] add describe_local_ring for Hadoop's benefit patch by Piotr Kołaczkowski; reviewed by jbellis for CASSANDRA-6268

Posted by jb...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/86e949f3/interface/thrift/gen-java/org/apache/cassandra/thrift/CfDef.java
----------------------------------------------------------------------
diff --git a/interface/thrift/gen-java/org/apache/cassandra/thrift/CfDef.java b/interface/thrift/gen-java/org/apache/cassandra/thrift/CfDef.java
index 60aa07f..059d630 100644
--- a/interface/thrift/gen-java/org/apache/cassandra/thrift/CfDef.java
+++ b/interface/thrift/gen-java/org/apache/cassandra/thrift/CfDef.java
@@ -32,10 +32,15 @@ import java.util.ArrayList;
 import java.util.Map;
 import java.util.HashMap;
 import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
 import java.util.EnumSet;
 import java.util.Collections;
 import java.util.BitSet;
 import java.nio.ByteBuffer;
+import java.util.Arrays;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 public class CfDef implements org.apache.thrift.TBase<CfDef, CfDef._Fields>, java.io.Serializable, Cloneable {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("CfDef");
@@ -62,6 +67,7 @@ public class CfDef implements org.apache.thrift.TBase<CfDef, CfDef._Fields>, jav
   private static final org.apache.thrift.protocol.TField BLOOM_FILTER_FP_CHANCE_FIELD_DESC = new org.apache.thrift.protocol.TField("bloom_filter_fp_chance", org.apache.thrift.protocol.TType.DOUBLE, (short)33);
   private static final org.apache.thrift.protocol.TField CACHING_FIELD_DESC = new org.apache.thrift.protocol.TField("caching", org.apache.thrift.protocol.TType.STRING, (short)34);
   private static final org.apache.thrift.protocol.TField DCLOCAL_READ_REPAIR_CHANCE_FIELD_DESC = new org.apache.thrift.protocol.TField("dclocal_read_repair_chance", org.apache.thrift.protocol.TType.DOUBLE, (short)37);
+  private static final org.apache.thrift.protocol.TField POPULATE_IO_CACHE_ON_FLUSH_FIELD_DESC = new org.apache.thrift.protocol.TField("populate_io_cache_on_flush", org.apache.thrift.protocol.TType.BOOL, (short)38);
   private static final org.apache.thrift.protocol.TField ROW_CACHE_SIZE_FIELD_DESC = new org.apache.thrift.protocol.TField("row_cache_size", org.apache.thrift.protocol.TType.DOUBLE, (short)9);
   private static final org.apache.thrift.protocol.TField KEY_CACHE_SIZE_FIELD_DESC = new org.apache.thrift.protocol.TField("key_cache_size", org.apache.thrift.protocol.TType.DOUBLE, (short)11);
   private static final org.apache.thrift.protocol.TField ROW_CACHE_SAVE_PERIOD_IN_SECONDS_FIELD_DESC = new org.apache.thrift.protocol.TField("row_cache_save_period_in_seconds", org.apache.thrift.protocol.TType.I32, (short)19);
@@ -72,7 +78,6 @@ public class CfDef implements org.apache.thrift.TBase<CfDef, CfDef._Fields>, jav
   private static final org.apache.thrift.protocol.TField MERGE_SHARDS_CHANCE_FIELD_DESC = new org.apache.thrift.protocol.TField("merge_shards_chance", org.apache.thrift.protocol.TType.DOUBLE, (short)25);
   private static final org.apache.thrift.protocol.TField ROW_CACHE_PROVIDER_FIELD_DESC = new org.apache.thrift.protocol.TField("row_cache_provider", org.apache.thrift.protocol.TType.STRING, (short)27);
   private static final org.apache.thrift.protocol.TField ROW_CACHE_KEYS_TO_SAVE_FIELD_DESC = new org.apache.thrift.protocol.TField("row_cache_keys_to_save", org.apache.thrift.protocol.TType.I32, (short)31);
-  private static final org.apache.thrift.protocol.TField POPULATE_IO_CACHE_ON_FLUSH_FIELD_DESC = new org.apache.thrift.protocol.TField("populate_io_cache_on_flush", org.apache.thrift.protocol.TType.BOOL, (short)38);
 
   public String keyspace; // required
   public String name; // required
@@ -331,16 +336,16 @@ public class CfDef implements org.apache.thrift.TBase<CfDef, CfDef._Fields>, jav
   private static final int __REPLICATE_ON_WRITE_ISSET_ID = 5;
   private static final int __BLOOM_FILTER_FP_CHANCE_ISSET_ID = 6;
   private static final int __DCLOCAL_READ_REPAIR_CHANCE_ISSET_ID = 7;
-  private static final int __ROW_CACHE_SIZE_ISSET_ID = 8;
-  private static final int __KEY_CACHE_SIZE_ISSET_ID = 9;
-  private static final int __ROW_CACHE_SAVE_PERIOD_IN_SECONDS_ISSET_ID = 10;
-  private static final int __KEY_CACHE_SAVE_PERIOD_IN_SECONDS_ISSET_ID = 11;
-  private static final int __MEMTABLE_FLUSH_AFTER_MINS_ISSET_ID = 12;
-  private static final int __MEMTABLE_THROUGHPUT_IN_MB_ISSET_ID = 13;
-  private static final int __MEMTABLE_OPERATIONS_IN_MILLIONS_ISSET_ID = 14;
-  private static final int __MERGE_SHARDS_CHANCE_ISSET_ID = 15;
-  private static final int __ROW_CACHE_KEYS_TO_SAVE_ISSET_ID = 16;
-  private static final int __POPULATE_IO_CACHE_ISSET_ID = 17;
+  private static final int __POPULATE_IO_CACHE_ON_FLUSH_ISSET_ID = 8;
+  private static final int __ROW_CACHE_SIZE_ISSET_ID = 9;
+  private static final int __KEY_CACHE_SIZE_ISSET_ID = 10;
+  private static final int __ROW_CACHE_SAVE_PERIOD_IN_SECONDS_ISSET_ID = 11;
+  private static final int __KEY_CACHE_SAVE_PERIOD_IN_SECONDS_ISSET_ID = 12;
+  private static final int __MEMTABLE_FLUSH_AFTER_MINS_ISSET_ID = 13;
+  private static final int __MEMTABLE_THROUGHPUT_IN_MB_ISSET_ID = 14;
+  private static final int __MEMTABLE_OPERATIONS_IN_MILLIONS_ISSET_ID = 15;
+  private static final int __MERGE_SHARDS_CHANCE_ISSET_ID = 16;
+  private static final int __ROW_CACHE_KEYS_TO_SAVE_ISSET_ID = 17;
   private BitSet __isset_bit_vector = new BitSet(18);
 
   public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
@@ -375,7 +380,7 @@ public class CfDef implements org.apache.thrift.TBase<CfDef, CfDef._Fields>, jav
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
     tmpMap.put(_Fields.REPLICATE_ON_WRITE, new org.apache.thrift.meta_data.FieldMetaData("replicate_on_write", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL)));
-    tmpMap.put(_Fields.KEY_VALIDATION_CLASS, new org.apache.thrift.meta_data.FieldMetaData("key_validation_class", org.apache.thrift.TFieldRequirementType.OPTIONAL,
+    tmpMap.put(_Fields.KEY_VALIDATION_CLASS, new org.apache.thrift.meta_data.FieldMetaData("key_validation_class", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
     tmpMap.put(_Fields.KEY_ALIAS, new org.apache.thrift.meta_data.FieldMetaData("key_alias", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING        , true)));
@@ -395,9 +400,9 @@ public class CfDef implements org.apache.thrift.TBase<CfDef, CfDef._Fields>, jav
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
     tmpMap.put(_Fields.DCLOCAL_READ_REPAIR_CHANCE, new org.apache.thrift.meta_data.FieldMetaData("dclocal_read_repair_chance", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.DOUBLE)));
-    tmpMap.put(_Fields.POPULATE_IO_CACHE_ON_FLUSH, new org.apache.thrift.meta_data.FieldMetaData("populate_io_cache_on_flush", org.apache.thrift.TFieldRequirementType.OPTIONAL,
+    tmpMap.put(_Fields.POPULATE_IO_CACHE_ON_FLUSH, new org.apache.thrift.meta_data.FieldMetaData("populate_io_cache_on_flush", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL)));
-    tmpMap.put(_Fields.ROW_CACHE_SIZE, new org.apache.thrift.meta_data.FieldMetaData("row_cache_size", org.apache.thrift.TFieldRequirementType.OPTIONAL,
+    tmpMap.put(_Fields.ROW_CACHE_SIZE, new org.apache.thrift.meta_data.FieldMetaData("row_cache_size", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.DOUBLE)));
     tmpMap.put(_Fields.KEY_CACHE_SIZE, new org.apache.thrift.meta_data.FieldMetaData("key_cache_size", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.DOUBLE)));
@@ -429,6 +434,7 @@ public class CfDef implements org.apache.thrift.TBase<CfDef, CfDef._Fields>, jav
     this.caching = "keys_only";
 
     this.dclocal_read_repair_chance = 0;
+
   }
 
   public CfDef(
@@ -485,6 +491,7 @@ public class CfDef implements org.apache.thrift.TBase<CfDef, CfDef._Fields>, jav
     }
     if (other.isSetKey_alias()) {
       this.key_alias = org.apache.thrift.TBaseHelper.copyBinary(other.key_alias);
+;
     }
     if (other.isSetCompaction_strategy()) {
       this.compaction_strategy = other.compaction_strategy;
@@ -575,10 +582,11 @@ public class CfDef implements org.apache.thrift.TBase<CfDef, CfDef._Fields>, jav
     setBloom_filter_fp_chanceIsSet(false);
     this.bloom_filter_fp_chance = 0.0;
     this.caching = "keys_only";
+
     this.dclocal_read_repair_chance = 0;
+
     setPopulate_io_cache_on_flushIsSet(false);
     this.populate_io_cache_on_flush = false;
-
     setRow_cache_sizeIsSet(false);
     this.row_cache_size = 0.0;
     setKey_cache_sizeIsSet(false);
@@ -1154,16 +1162,6 @@ public class CfDef implements org.apache.thrift.TBase<CfDef, CfDef._Fields>, jav
     return this;
   }
 
-  public boolean getPopulate_io_cache_on_flush() {
-    return this.populate_io_cache_on_flush;
-  }
-
-  public CfDef setPopulate_io_cache_on_flush(boolean populate_io_cache_on_flush) {
-    this.populate_io_cache_on_flush = populate_io_cache_on_flush;
-    setPopulate_io_cache_on_flushIsSet(true);
-    return this;
-  }
-
   public void unsetDclocal_read_repair_chance() {
     __isset_bit_vector.clear(__DCLOCAL_READ_REPAIR_CHANCE_ISSET_ID);
   }
@@ -1177,17 +1175,27 @@ public class CfDef implements org.apache.thrift.TBase<CfDef, CfDef._Fields>, jav
     __isset_bit_vector.set(__DCLOCAL_READ_REPAIR_CHANCE_ISSET_ID, value);
   }
 
+  public boolean isPopulate_io_cache_on_flush() {
+    return this.populate_io_cache_on_flush;
+  }
+
+  public CfDef setPopulate_io_cache_on_flush(boolean populate_io_cache_on_flush) {
+    this.populate_io_cache_on_flush = populate_io_cache_on_flush;
+    setPopulate_io_cache_on_flushIsSet(true);
+    return this;
+  }
+
   public void unsetPopulate_io_cache_on_flush() {
-    __isset_bit_vector.clear(__POPULATE_IO_CACHE_ISSET_ID);
+    __isset_bit_vector.clear(__POPULATE_IO_CACHE_ON_FLUSH_ISSET_ID);
   }
 
   /** Returns true if field populate_io_cache_on_flush is set (has been assigned a value) and false otherwise */
   public boolean isSetPopulate_io_cache_on_flush() {
-    return __isset_bit_vector.get(__POPULATE_IO_CACHE_ISSET_ID);
+    return __isset_bit_vector.get(__POPULATE_IO_CACHE_ON_FLUSH_ISSET_ID);
   }
 
   public void setPopulate_io_cache_on_flushIsSet(boolean value) {
-    __isset_bit_vector.set(__POPULATE_IO_CACHE_ISSET_ID, value);
+    __isset_bit_vector.set(__POPULATE_IO_CACHE_ON_FLUSH_ISSET_ID, value);
   }
 
   /**
@@ -1663,7 +1671,7 @@ public class CfDef implements org.apache.thrift.TBase<CfDef, CfDef._Fields>, jav
       if (value == null) {
         unsetPopulate_io_cache_on_flush();
       } else {
-        setPopulate_io_cache_on_flush((Boolean) value);
+        setPopulate_io_cache_on_flush((Boolean)value);
       }
       break;
 
@@ -1819,7 +1827,7 @@ public class CfDef implements org.apache.thrift.TBase<CfDef, CfDef._Fields>, jav
       return Double.valueOf(getDclocal_read_repair_chance());
 
     case POPULATE_IO_CACHE_ON_FLUSH:
-      return Boolean.valueOf(getPopulate_io_cache_on_flush());
+      return Boolean.valueOf(isPopulate_io_cache_on_flush());
 
     case ROW_CACHE_SIZE:
       return Double.valueOf(getRow_cache_size());
@@ -2146,10 +2154,10 @@ public class CfDef implements org.apache.thrift.TBase<CfDef, CfDef._Fields>, jav
     boolean this_present_populate_io_cache_on_flush = true && this.isSetPopulate_io_cache_on_flush();
     boolean that_present_populate_io_cache_on_flush = true && that.isSetPopulate_io_cache_on_flush();
     if (this_present_populate_io_cache_on_flush || that_present_populate_io_cache_on_flush) {
-        if (!(this_present_populate_io_cache_on_flush && that_present_populate_io_cache_on_flush))
-            return false;
-        if (this.populate_io_cache_on_flush != that.populate_io_cache_on_flush)
-            return false;
+      if (!(this_present_populate_io_cache_on_flush && that_present_populate_io_cache_on_flush))
+        return false;
+      if (this.populate_io_cache_on_flush != that.populate_io_cache_on_flush)
+        return false;
     }
 
     boolean this_present_row_cache_size = true && this.isSetRow_cache_size();
@@ -2362,7 +2370,7 @@ public class CfDef implements org.apache.thrift.TBase<CfDef, CfDef._Fields>, jav
     boolean present_populate_io_cache_on_flush = true && (isSetPopulate_io_cache_on_flush());
     builder.append(present_populate_io_cache_on_flush);
     if (present_populate_io_cache_on_flush)
-        builder.append(populate_io_cache_on_flush);
+      builder.append(populate_io_cache_on_flush);
 
     boolean present_row_cache_size = true && (isSetRow_cache_size());
     builder.append(present_row_cache_size);
@@ -2973,10 +2981,10 @@ public class CfDef implements org.apache.thrift.TBase<CfDef, CfDef._Fields>, jav
           if (field.type == org.apache.thrift.protocol.TType.BOOL) {
             this.populate_io_cache_on_flush = iprot.readBool();
             setPopulate_io_cache_on_flushIsSet(true);
-          } else {
+          } else { 
             org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
           }
-          break;  
+          break;
         case 9: // ROW_CACHE_SIZE
           if (field.type == org.apache.thrift.protocol.TType.DOUBLE) {
             this.row_cache_size = iprot.readDouble();
@@ -3479,9 +3487,9 @@ public class CfDef implements org.apache.thrift.TBase<CfDef, CfDef._Fields>, jav
     }
     if (isSetPopulate_io_cache_on_flush()) {
       if (!first) sb.append(", ");
-        sb.append("populate_io_cache_on_flush:");
-        sb.append(this.populate_io_cache_on_flush);
-        first = false;
+      sb.append("populate_io_cache_on_flush:");
+      sb.append(this.populate_io_cache_on_flush);
+      first = false;
     }
     if (isSetRow_cache_size()) {
       if (!first) sb.append(", ");

http://git-wip-us.apache.org/repos/asf/cassandra/blob/86e949f3/interface/thrift/gen-java/org/apache/cassandra/thrift/Constants.java
----------------------------------------------------------------------
diff --git a/interface/thrift/gen-java/org/apache/cassandra/thrift/Constants.java b/interface/thrift/gen-java/org/apache/cassandra/thrift/Constants.java
index 1315a5b..98a7ce9 100644
--- a/interface/thrift/gen-java/org/apache/cassandra/thrift/Constants.java
+++ b/interface/thrift/gen-java/org/apache/cassandra/thrift/Constants.java
@@ -44,6 +44,6 @@ import org.slf4j.LoggerFactory;
 
 public class Constants {
 
-  public static final String VERSION = "19.36.1";
+  public static final String VERSION = "19.36.2";
 
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/86e949f3/src/java/org/apache/cassandra/hadoop/AbstractColumnFamilyInputFormat.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/hadoop/AbstractColumnFamilyInputFormat.java b/src/java/org/apache/cassandra/hadoop/AbstractColumnFamilyInputFormat.java
index 8fd66ab..cca2734 100644
--- a/src/java/org/apache/cassandra/hadoop/AbstractColumnFamilyInputFormat.java
+++ b/src/java/org/apache/cassandra/hadoop/AbstractColumnFamilyInputFormat.java
@@ -19,12 +19,7 @@ package org.apache.cassandra.hadoop;
 
 import java.io.IOException;
 import java.net.InetAddress;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Random;
+import java.util.*;
 import java.util.concurrent.Callable;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Future;
@@ -130,6 +125,7 @@ public abstract class AbstractColumnFamilyInputFormat<K, Y> extends InputFormat<
         partitioner = ConfigHelper.getInputPartitioner(context.getConfiguration());
         logger.debug("partitioner is " + partitioner);
 
+
         // cannonical ranges, split into pieces, fetching the splits in parallel
         ExecutorService executor = new ThreadPoolExecutor(0, 128, 60L, TimeUnit.SECONDS, new LinkedBlockingQueue<Runnable>());
         List<InputSplit> splits = new ArrayList<InputSplit>();
@@ -326,7 +322,7 @@ public abstract class AbstractColumnFamilyInputFormat<K, Y> extends InputFormat<
         List<TokenRange> map;
         try
         {
-            map = client.describe_ring(ConfigHelper.getInputKeyspace(conf));
+            map = client.describe_local_ring(ConfigHelper.getInputKeyspace(conf));
         }
         catch (InvalidRequestException e)
         {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/86e949f3/src/java/org/apache/cassandra/service/StorageService.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/StorageService.java b/src/java/org/apache/cassandra/service/StorageService.java
index c9326ae..b85f478 100644
--- a/src/java/org/apache/cassandra/service/StorageService.java
+++ b/src/java/org/apache/cassandra/service/StorageService.java
@@ -34,6 +34,7 @@ import javax.management.Notification;
 import javax.management.NotificationBroadcasterSupport;
 import javax.management.ObjectName;
 
+import com.google.common.base.Predicate;
 import com.google.common.collect.*;
 
 import com.google.common.util.concurrent.AtomicDouble;
@@ -1080,15 +1081,61 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
 
     public Map<Range<Token>, List<InetAddress>> getRangeToAddressMap(String keyspace)
     {
+        return getRangeToAddressMap(keyspace, tokenMetadata.sortedTokens());
+    }
+
+    public Map<Range<Token>, List<InetAddress>> getRangeToAddressMapInLocalDC(String keyspace)
+    {
+        Predicate<InetAddress> isLocalDC = new Predicate<InetAddress>()
+        {
+            public boolean apply(InetAddress address)
+            {
+                return isLocalDC(address);
+            }
+        };
+
+        Map<Range<Token>, List<InetAddress>> origMap = getRangeToAddressMap(keyspace, getTokensInLocalDC());
+        Map<Range<Token>, List<InetAddress>> filteredMap = Maps.newHashMap();
+        for (Map.Entry<Range<Token>, List<InetAddress>> entry : origMap.entrySet())
+        {
+            List<InetAddress> endpointsInLocalDC = Lists.newArrayList(Collections2.filter(entry.getValue(), isLocalDC));
+            filteredMap.put(entry.getKey(), endpointsInLocalDC);
+        }
+
+        return filteredMap;
+    }
+
+    private List<Token> getTokensInLocalDC()
+    {
+        List<Token> filteredTokens = Lists.newArrayList();
+        for (Token token : tokenMetadata.sortedTokens())
+        {
+            InetAddress endpoint = tokenMetadata.getEndpoint(token);
+            if (isLocalDC(endpoint))
+                filteredTokens.add(token);
+        }
+        return filteredTokens;
+    }
+
+    private boolean isLocalDC(InetAddress targetHost)
+    {
+        String remoteDC = DatabaseDescriptor.getEndpointSnitch().getDatacenter(targetHost);
+        String localDC = DatabaseDescriptor.getEndpointSnitch().getDatacenter(FBUtilities.getBroadcastAddress());
+        return remoteDC.equals(localDC);
+    }
+
+    private Map<Range<Token>, List<InetAddress>> getRangeToAddressMap(String keyspace, List<Token> sortedTokens)
+    {
         // some people just want to get a visual representation of things. Allow null and set it to the first
         // non-system table.
         if (keyspace == null)
             keyspace = Schema.instance.getNonSystemTables().get(0);
 
-        List<Range<Token>> ranges = getAllRanges(tokenMetadata.sortedTokens());
+        List<Range<Token>> ranges = getAllRanges(sortedTokens);
         return constructRangeToEndpointMap(keyspace, ranges);
     }
 
+
     /**
      * The same as {@code describeRing(String)} but converts TokenRange to the String for JMX compatibility
      *
@@ -1126,13 +1173,31 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
      */
     public List<TokenRange> describeRing(String keyspace) throws InvalidRequestException
     {
+        return describeRing(keyspace, false);
+    }
+
+    /**
+     * The same as {@code describeRing(String)} but considers only the part of the ring formed by nodes in the local DC.
+     */
+    public List<TokenRange> describeLocalRing(String keyspace) throws InvalidRequestException
+    {
+        return describeRing(keyspace, true);
+    }
+
+    private List<TokenRange> describeRing(String keyspace, boolean includeOnlyLocalDC) throws InvalidRequestException
+    {
         if (keyspace == null || Table.open(keyspace).getReplicationStrategy() instanceof LocalStrategy)
             throw new InvalidRequestException("There is no ring for the keyspace: " + keyspace);
 
         List<TokenRange> ranges = new ArrayList<TokenRange>();
         Token.TokenFactory tf = getPartitioner().getTokenFactory();
 
-        for (Map.Entry<Range<Token>, List<InetAddress>> entry : getRangeToAddressMap(keyspace).entrySet())
+        Map<Range<Token>, List<InetAddress>> rangeToAddressMap =
+                includeOnlyLocalDC
+                        ? getRangeToAddressMapInLocalDC(keyspace)
+                        : getRangeToAddressMap(keyspace);
+
+        for (Map.Entry<Range<Token>, List<InetAddress>> entry : rangeToAddressMap.entrySet())
         {
             Range range = entry.getKey();
             List<InetAddress> addresses = entry.getValue();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/86e949f3/src/java/org/apache/cassandra/thrift/CassandraServer.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/thrift/CassandraServer.java b/src/java/org/apache/cassandra/thrift/CassandraServer.java
index 883ab5a..6a35285 100644
--- a/src/java/org/apache/cassandra/thrift/CassandraServer.java
+++ b/src/java/org/apache/cassandra/thrift/CassandraServer.java
@@ -1183,6 +1183,19 @@ public class CassandraServer implements Cassandra.Iface
         }
     }
 
+    @Override
+    public List<TokenRange> describe_local_ring(String keyspace) throws InvalidRequestException, TException
+    {
+        try
+        {
+            return StorageService.instance.describeLocalRing(keyspace);
+        }
+        catch (RequestValidationException e)
+        {
+            throw ThriftConversion.toThrift(e);
+        }
+    }
+
     public Map<String, String> describe_token_map() throws InvalidRequestException
     {
         return StorageService.instance.getTokenToEndpointMap();


[12/16] git commit: Merge branch 'cassandra-1.2' into cassandra-2.0

Posted by jb...@apache.org.
Merge branch 'cassandra-1.2' into cassandra-2.0


Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/f2a82ee7
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/f2a82ee7
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/f2a82ee7

Branch: refs/heads/trunk
Commit: f2a82ee7311697aec2d440be8a89aebeaf4dab90
Parents: f54f4e2 86e949f
Author: Jonathan Ellis <jb...@apache.org>
Authored: Fri Dec 6 09:50:37 2013 -0600
Committer: Jonathan Ellis <jb...@apache.org>
Committed: Fri Dec 6 09:51:59 2013 -0600

----------------------------------------------------------------------
 CHANGES.txt                                     |    1 +
 interface/cassandra.thrift                      |    5 +
 .../org/apache/cassandra/thrift/Cassandra.java  | 2250 +++++++++++++-----
 .../cassandra/thrift/cassandraConstants.java    |    2 +-
 .../hadoop/AbstractColumnFamilyInputFormat.java |   10 +-
 .../cassandra/service/StorageService.java       |   69 +-
 .../cassandra/thrift/CassandraServer.java       |   13 +
 7 files changed, 1728 insertions(+), 622 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/f2a82ee7/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index 43d0e9d,3b1f2f6..fc512cd
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -19,45 -14,10 +19,46 @@@ Merged from 1.2
   * Warn when collection read has > 65K elements (CASSANDRA-5428)
   * Fix cache persistence when both row and key cache are enabled 
     (CASSANDRA-6413)
+  * (Hadoop) add describe_local_ring (CASSANDRA-6268)
  
  
 -1.2.12
 +2.0.3
 + * Fix FD leak on slice read path (CASSANDRA-6275)
 + * Cancel read meter task when closing SSTR (CASSANDRA-6358)
 + * free off-heap IndexSummary during bulk (CASSANDRA-6359)
 + * Recover from IOException in accept() thread (CASSANDRA-6349)
 + * Improve Gossip tolerance of abnormally slow tasks (CASSANDRA-6338)
 + * Fix trying to hint timed out counter writes (CASSANDRA-6322)
 + * Allow restoring specific columnfamilies from archived CL (CASSANDRA-4809)
 + * Avoid flushing compaction_history after each operation (CASSANDRA-6287)
 + * Fix repair assertion error when tombstones expire (CASSANDRA-6277)
 + * Skip loading corrupt key cache (CASSANDRA-6260)
 + * Fixes for compacting larger-than-memory rows (CASSANDRA-6274)
 + * Compact hottest sstables first and optionally omit coldest from
 +   compaction entirely (CASSANDRA-6109)
 + * Fix modifying column_metadata from thrift (CASSANDRA-6182)
 + * cqlsh: fix LIST USERS output (CASSANDRA-6242)
 + * Add IRequestSink interface (CASSANDRA-6248)
 + * Update memtable size while flushing (CASSANDRA-6249)
 + * Provide hooks around CQL2/CQL3 statement execution (CASSANDRA-6252)
 + * Require Permission.SELECT for CAS updates (CASSANDRA-6247)
 + * New CQL-aware SSTableWriter (CASSANDRA-5894)
 + * Reject CAS operation when the protocol v1 is used (CASSANDRA-6270)
 + * Correctly throw error when frame too large (CASSANDRA-5981)
 + * Fix serialization bug in PagedRange with 2ndary indexes (CASSANDRA-6299)
 + * Fix CQL3 table validation in Thrift (CASSANDRA-6140)
 + * Fix bug missing results with IN clauses (CASSANDRA-6327)
 + * Fix paging with reversed slices (CASSANDRA-6343)
 + * Set minTimestamp correctly to be able to drop expired sstables (CASSANDRA-6337)
 + * Support NaN and Infinity as float literals (CASSANDRA-6003)
 + * Remove RF from nodetool ring output (CASSANDRA-6289)
 + * Fix attempting to flush empty rows (CASSANDRA-6374)
 + * Fix potential out of bounds exception when paging (CASSANDRA-6333)
 +Merged from 1.2:
 + * Optimize FD phi calculation (CASSANDRA-6386)
 + * Improve initial FD phi estimate when starting up (CASSANDRA-6385)
 + * Don't list CQL3 table in CLI describe even if named explicitely 
 +   (CASSANDRA-5750)
   * Invalidate row cache when dropping CF (CASSANDRA-6351)
   * add non-jamm path for cached statements (CASSANDRA-6293)
   * (Hadoop) Require CFRR batchSize to be at least 2 (CASSANDRA-6114)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f2a82ee7/interface/cassandra.thrift
----------------------------------------------------------------------


[03/16] git commit: add describe_local_ring for Hadoop's benefit patch by Piotr Kołaczkowski; reviewed by jbellis for CASSANDRA-6268

Posted by jb...@apache.org.
add describe_local_ring for Hadoop's benefit
patch by Piotr Kołaczkowski; reviewed by jbellis for CASSANDRA-6268


Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/86e949f3
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/86e949f3
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/86e949f3

Branch: refs/heads/cassandra-1.2
Commit: 86e949f321cb1ce3b0bc0c9ed387218cc8a2b7b9
Parents: 57613dc
Author: Jonathan Ellis <jb...@apache.org>
Authored: Thu Dec 5 20:49:19 2013 -0600
Committer: Jonathan Ellis <jb...@apache.org>
Committed: Thu Dec 5 20:49:19 2013 -0600

----------------------------------------------------------------------
 CHANGES.txt                                     |    1 +
 interface/cassandra.thrift                      |    7 +-
 .../org/apache/cassandra/thrift/Cassandra.java  | 2336 ++++++++++++------
 .../org/apache/cassandra/thrift/CfDef.java      |   88 +-
 .../org/apache/cassandra/thrift/Constants.java  |    2 +-
 .../hadoop/AbstractColumnFamilyInputFormat.java |   10 +-
 .../cassandra/service/StorageService.java       |   69 +-
 .../cassandra/thrift/CassandraServer.java       |   13 +
 8 files changed, 1720 insertions(+), 806 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/86e949f3/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 166c566..3b1f2f6 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -14,6 +14,7 @@
  * Warn when collection read has > 65K elements (CASSANDRA-5428)
  * Fix cache persistence when both row and key cache are enabled 
    (CASSANDRA-6413)
+ * (Hadoop) add describe_local_ring (CASSANDRA-6268)
 
 
 1.2.12

http://git-wip-us.apache.org/repos/asf/cassandra/blob/86e949f3/interface/cassandra.thrift
----------------------------------------------------------------------
diff --git a/interface/cassandra.thrift b/interface/cassandra.thrift
index c64b2d7..1fc7201 100644
--- a/interface/cassandra.thrift
+++ b/interface/cassandra.thrift
@@ -55,7 +55,7 @@ namespace rb CassandraThrift
 # An effort should be made not to break forward-client-compatibility either
 # (e.g. one should avoid removing obsolete fields from the IDL), but no
 # guarantees in this respect are made by the Cassandra project.
-const string VERSION = "19.36.1"
+const string VERSION = "19.36.2"
 
 
 #
@@ -718,6 +718,11 @@ service Cassandra {
   list<TokenRange> describe_ring(1:required string keyspace)
                    throws (1:InvalidRequestException ire),
 
+
+  /** same as describe_ring, but considers only nodes in the local DC */
+  list<TokenRange> describe_local_ring(1:required string keyspace)
+                   throws (1:InvalidRequestException ire),
+
   /** get the mapping between token->node ip
       without taking replication into consideration
       https://issues.apache.org/jira/browse/CASSANDRA-4092 */


[15/16] git commit: Merge branch 'cassandra-1.2' into cassandra-2.0

Posted by jb...@apache.org.
Merge branch 'cassandra-1.2' into cassandra-2.0


Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/f2a82ee7
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/f2a82ee7
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/f2a82ee7

Branch: refs/heads/cassandra-2.0
Commit: f2a82ee7311697aec2d440be8a89aebeaf4dab90
Parents: f54f4e2 86e949f
Author: Jonathan Ellis <jb...@apache.org>
Authored: Fri Dec 6 09:50:37 2013 -0600
Committer: Jonathan Ellis <jb...@apache.org>
Committed: Fri Dec 6 09:51:59 2013 -0600

----------------------------------------------------------------------
 CHANGES.txt                                     |    1 +
 interface/cassandra.thrift                      |    5 +
 .../org/apache/cassandra/thrift/Cassandra.java  | 2250 +++++++++++++-----
 .../cassandra/thrift/cassandraConstants.java    |    2 +-
 .../hadoop/AbstractColumnFamilyInputFormat.java |   10 +-
 .../cassandra/service/StorageService.java       |   69 +-
 .../cassandra/thrift/CassandraServer.java       |   13 +
 7 files changed, 1728 insertions(+), 622 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/f2a82ee7/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index 43d0e9d,3b1f2f6..fc512cd
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -19,45 -14,10 +19,46 @@@ Merged from 1.2
   * Warn when collection read has > 65K elements (CASSANDRA-5428)
   * Fix cache persistence when both row and key cache are enabled 
     (CASSANDRA-6413)
+  * (Hadoop) add describe_local_ring (CASSANDRA-6268)
  
  
 -1.2.12
 +2.0.3
 + * Fix FD leak on slice read path (CASSANDRA-6275)
 + * Cancel read meter task when closing SSTR (CASSANDRA-6358)
 + * free off-heap IndexSummary during bulk (CASSANDRA-6359)
 + * Recover from IOException in accept() thread (CASSANDRA-6349)
 + * Improve Gossip tolerance of abnormally slow tasks (CASSANDRA-6338)
 + * Fix trying to hint timed out counter writes (CASSANDRA-6322)
 + * Allow restoring specific columnfamilies from archived CL (CASSANDRA-4809)
 + * Avoid flushing compaction_history after each operation (CASSANDRA-6287)
 + * Fix repair assertion error when tombstones expire (CASSANDRA-6277)
 + * Skip loading corrupt key cache (CASSANDRA-6260)
 + * Fixes for compacting larger-than-memory rows (CASSANDRA-6274)
 + * Compact hottest sstables first and optionally omit coldest from
 +   compaction entirely (CASSANDRA-6109)
 + * Fix modifying column_metadata from thrift (CASSANDRA-6182)
 + * cqlsh: fix LIST USERS output (CASSANDRA-6242)
 + * Add IRequestSink interface (CASSANDRA-6248)
 + * Update memtable size while flushing (CASSANDRA-6249)
 + * Provide hooks around CQL2/CQL3 statement execution (CASSANDRA-6252)
 + * Require Permission.SELECT for CAS updates (CASSANDRA-6247)
 + * New CQL-aware SSTableWriter (CASSANDRA-5894)
 + * Reject CAS operation when the protocol v1 is used (CASSANDRA-6270)
 + * Correctly throw error when frame too large (CASSANDRA-5981)
 + * Fix serialization bug in PagedRange with 2ndary indexes (CASSANDRA-6299)
 + * Fix CQL3 table validation in Thrift (CASSANDRA-6140)
 + * Fix bug missing results with IN clauses (CASSANDRA-6327)
 + * Fix paging with reversed slices (CASSANDRA-6343)
 + * Set minTimestamp correctly to be able to drop expired sstables (CASSANDRA-6337)
 + * Support NaN and Infinity as float literals (CASSANDRA-6003)
 + * Remove RF from nodetool ring output (CASSANDRA-6289)
 + * Fix attempting to flush empty rows (CASSANDRA-6374)
 + * Fix potential out of bounds exception when paging (CASSANDRA-6333)
 +Merged from 1.2:
 + * Optimize FD phi calculation (CASSANDRA-6386)
 + * Improve initial FD phi estimate when starting up (CASSANDRA-6385)
 + * Don't list CQL3 table in CLI describe even if named explicitely 
 +   (CASSANDRA-5750)
   * Invalidate row cache when dropping CF (CASSANDRA-6351)
   * add non-jamm path for cached statements (CASSANDRA-6293)
   * (Hadoop) Require CFRR batchSize to be at least 2 (CASSANDRA-6114)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f2a82ee7/interface/cassandra.thrift
----------------------------------------------------------------------