You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by al...@apache.org on 2014/09/25 03:33:47 UTC

[1/5] Remove cassandra-cli

Repository: cassandra
Updated Branches:
  refs/heads/trunk 24b18a9e5 -> 380273826


http://git-wip-us.apache.org/repos/asf/cassandra/blob/38027382/test/pig/org/apache/cassandra/pig/ThriftColumnFamilyTest.java
----------------------------------------------------------------------
diff --git a/test/pig/org/apache/cassandra/pig/ThriftColumnFamilyTest.java b/test/pig/org/apache/cassandra/pig/ThriftColumnFamilyTest.java
index 8903297..167668c 100644
--- a/test/pig/org/apache/cassandra/pig/ThriftColumnFamilyTest.java
+++ b/test/pig/org/apache/cassandra/pig/ThriftColumnFamilyTest.java
@@ -20,27 +20,20 @@ package org.apache.cassandra.pig;
 
 import java.io.IOException;
 import java.nio.ByteBuffer;
-import java.nio.charset.CharacterCodingException;
 import java.util.Iterator;
 
-import org.apache.cassandra.cli.CliMain;
 import org.apache.cassandra.exceptions.ConfigurationException;
-import org.apache.cassandra.thrift.AuthenticationException;
-import org.apache.cassandra.thrift.AuthorizationException;
 import org.apache.cassandra.thrift.Cassandra;
 import org.apache.cassandra.thrift.ColumnOrSuperColumn;
 import org.apache.cassandra.thrift.ColumnPath;
 import org.apache.cassandra.thrift.ConsistencyLevel;
-import org.apache.cassandra.thrift.InvalidRequestException;
 import org.apache.cassandra.thrift.NotFoundException;
-import org.apache.cassandra.thrift.TimedOutException;
-import org.apache.cassandra.thrift.UnavailableException;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.pig.data.DataBag;
 import org.apache.pig.data.DataByteArray;
 import org.apache.pig.data.Tuple;
 import org.apache.thrift.TException;
-import org.apache.thrift.transport.TTransportException;
+
 import org.junit.Assert;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -48,171 +41,159 @@ import org.junit.Test;
 public class ThriftColumnFamilyTest extends PigTestBase
 {    
     private static String[] statements = {
-            "create keyspace thriftKs with placement_strategy = 'org.apache.cassandra.locator.SimpleStrategy' and" +
-            " strategy_options={replication_factor:1};",
-            "use thriftKs;",
-
-            "create column family SomeApp " +
-                    " with comparator = UTF8Type " +
-                    " and default_validation_class = UTF8Type " +
-                    " and key_validation_class = UTF8Type " +
-                    " and column_metadata = [{column_name: name, validation_class: UTF8Type, index_type: KEYS}, " +
-                    "{column_name: vote_type, validation_class: UTF8Type}, " +
-                    "{column_name: rating, validation_class: Int32Type}, " +
-                    "{column_name: score, validation_class: LongType}, " +
-                    "{column_name: percent, validation_class: FloatType}, " +
-                    "{column_name: atomic_weight, validation_class: DoubleType}, " +
-                    "{column_name: created, validation_class: DateType},]; ",
-
-             "create column family CopyOfSomeApp " +
-                    "with key_validation_class = UTF8Type " +
-                    "and default_validation_class = UTF8Type " +
-                    "and comparator = UTF8Type " +
-                    "and column_metadata = " +
-                    "[ " +
-                        "{column_name: name, validation_class: UTF8Type, index_type: KEYS}, " +
-                        "{column_name: vote_type, validation_class: UTF8Type}, " +
-                        "{column_name: rating, validation_class: Int32Type}, " +
-                        "{column_name: score, validation_class: LongType}, " +
-                        "{column_name: percent, validation_class: FloatType}, " +
-                        "{column_name: atomic_weight, validation_class: DoubleType}, " +
-                        "{column_name: created, validation_class: DateType}, " +
-                    "];",
-
-             "set SomeApp['foo']['name'] = 'User Foo';",
-             "set SomeApp['foo']['vote_type'] = 'like';",
-             "set SomeApp['foo']['rating'] = 8;",
-             "set SomeApp['foo']['score'] = 125000;",
-             "set SomeApp['foo']['percent'] = '85.0';",
-             "set SomeApp['foo']['atomic_weight'] = '2.7182818284590451';",
-             "set SomeApp['foo']['created'] = 1335890877;",
-
-             "set SomeApp['bar']['name'] = 'User Bar';",
-             "set SomeApp['bar']['vote_type'] = 'like';",
-             "set SomeApp['bar']['rating'] = 9;",
-             "set SomeApp['bar']['score'] = 15000;",
-             "set SomeApp['bar']['percent'] = '35.0';",
-             "set SomeApp['bar']['atomic_weight'] = '3.1415926535897931';",
-             "set SomeApp['bar']['created'] = 1335890877;",
-
-             "set SomeApp['baz']['name'] = 'User Baz';",
-             "set SomeApp['baz']['vote_type'] = 'dislike';",
-             "set SomeApp['baz']['rating'] = 3;",
-             "set SomeApp['baz']['score'] = 512000;",
-             "set SomeApp['baz']['percent'] = '95.3';",
-             "set SomeApp['baz']['atomic_weight'] = '1.61803399';",
-             "set SomeApp['baz']['created'] = 1335890877;",
-             "set SomeApp['baz']['extra1'] = 'extra1';",
-             "set SomeApp['baz']['extra2'] = 'extra2';",
-             "set SomeApp['baz']['extra3'] = 'extra3';",
-
-             "set SomeApp['qux']['name'] = 'User Qux';",
-             "set SomeApp['qux']['vote_type'] = 'dislike';",
-             "set SomeApp['qux']['rating'] = 2;",
-             "set SomeApp['qux']['score'] = 12000;",
-             "set SomeApp['qux']['percent'] = '64.7';",
-             "set SomeApp['qux']['atomic_weight'] = '0.660161815846869';",
-             "set SomeApp['qux']['created'] = 1335890877;",
-             "set SomeApp['qux']['extra1'] = 'extra1';",
-             "set SomeApp['qux']['extra2'] = 'extra2';",
-             "set SomeApp['qux']['extra3'] = 'extra3';",
-             "set SomeApp['qux']['extra4'] = 'extra4';",
-             "set SomeApp['qux']['extra5'] = 'extra5';",
-             "set SomeApp['qux']['extra6'] = 'extra6';",
-             "set SomeApp['qux']['extra7'] = 'extra7';",
-
-             "create column family U8 with " +
-                     "key_validation_class = UTF8Type and " +
-                     "comparator = UTF8Type;",
-                     
-             "create column family Bytes with " +
-                      "key_validation_class = BytesType and " +
-                      "comparator = UTF8Type;",
-
-             "set U8['foo']['x'] = ascii('Z');",
-             "set Bytes[ascii('foo')]['x'] = ascii('Z');",
-
-             "create column family CC with " +
-                       "key_validation_class = UTF8Type and " +
-                       "default_validation_class=CounterColumnType " +
-                       "and comparator=UTF8Type;",
-
-             "incr CC['chuck']['kick'];",
-             "incr CC['chuck']['kick'];",
-             "incr CC['chuck']['kick'];",
-             "incr CC['chuck']['fist'];",
-
-             "create column family Compo " +
-                       "with key_validation_class = UTF8Type " +
-                       "and default_validation_class = UTF8Type " +
-                       "and comparator = 'CompositeType(UTF8Type,UTF8Type)';",
-
-             "set Compo['punch']['bruce:lee'] = 'ouch';",
-             "set Compo['punch']['bruce:bruce'] = 'hunh?';",
-             "set Compo['kick']['bruce:lee'] = 'oww';",
-             "set Compo['kick']['bruce:bruce'] = 'watch it, mate';",
-
-             "create column family CompoInt " +
-                       "with key_validation_class = UTF8Type " +
-                       "and default_validation_class = UTF8Type " +
-                       "and comparator = 'CompositeType(LongType,LongType)';",
-
-            "set CompoInt['clock']['1:0'] = 'z';",
-            "set CompoInt['clock']['1:30'] = 'zzzz';",
-            "set CompoInt['clock']['2:30'] = 'daddy?';",
-            "set CompoInt['clock']['6:30'] = 'coffee...';",
-
-            "create column family CompoIntCopy " +
-                        "with key_validation_class = UTF8Type " +
-                        "and default_validation_class = UTF8Type " +
-                        "and comparator = 'CompositeType(LongType,LongType)';",
-
-            "create column family CompoKey " +
-                        "with key_validation_class = 'CompositeType(UTF8Type,LongType)' " +
-                        "and default_validation_class = UTF8Type " +
-                        "and comparator = LongType;",
-
-            "set CompoKey['clock:10']['1'] = 'z';",
-            "set CompoKey['clock:20']['1'] = 'zzzz';",
-            "set CompoKey['clock:30']['2'] = 'daddy?';",
-            "set CompoKey['clock:40']['6'] = 'coffee...';",
-
-            "create column family CompoKeyCopy " +
-                        "with key_validation_class = 'CompositeType(UTF8Type,LongType)' " +
-                        "and default_validation_class = UTF8Type " +
-                        "and comparator = LongType;"
+            "DROP KEYSPACE IF EXISTS thrift_ks",
+            "CREATE KEYSPACE thrift_ks WITH replication = {'class': 'SimpleStrategy', 'replication_factor': 1};",
+            "USE thrift_ks;",
+
+            "CREATE TABLE some_app (" +
+            "key text PRIMARY KEY," +
+            "name text," +
+            "vote_type text," +
+            "rating int," +
+            "score bigint," +
+            "percent float," +
+            "atomic_weight double," +
+            "created timestamp)" +
+            " WITH COMPACT STORAGE;",
+
+            "CREATE INDEX ON some_app(name);",
+
+            "INSERT INTO some_app (key, name, vote_type, rating, score, percent, atomic_weight, created) " +
+                    "VALUES ('foo', 'User Foo', 'like', 8, 125000, 85.0, 2.7182818284590451, 1335890877);",
+
+            "INSERT INTO some_app (key, name, vote_type, rating, score, percent, atomic_weight, created) " +
+                    "VALUES ('bar', 'User Bar', 'like', 9, 15000, 35.0, 3.1415926535897931, 1335890877);",
+
+            "INSERT INTO some_app (key, name, vote_type, rating, score, percent, atomic_weight, created) " +
+                    "VALUES ('baz', 'User Baz', 'dislike', 3, 512000, 95.3, 1.61803399, 1335890877);",
+
+            "INSERT INTO some_app (key, name, vote_type, rating, score, percent, atomic_weight, created) " +
+                    "VALUES ('qux', 'User Qux', 'dislike', 2, 12000, 64.7, 0.660161815846869, 1335890877);",
+
+            "CREATE TABLE copy_of_some_app (" +
+            "key text PRIMARY KEY," +
+            "name text," +
+            "vote_type text," +
+            "rating int," +
+            "score bigint," +
+            "percent float," +
+            "atomic_weight double," +
+            "created timestamp)" +
+            " WITH COMPACT STORAGE;",
+
+            "CREATE INDEX ON copy_of_some_app(name);",
+
+            "CREATE TABLE u8 (" +
+            "key text," +
+            "column1 text," +
+            "value blob," +
+            "PRIMARY KEY (key, column1))" +
+            " WITH COMPACT STORAGE",
+
+            "INSERT INTO u8 (key, column1, value) VALUES ('foo', 'x', asciiAsBlob('Z'))",
+
+            "CREATE TABLE bytes (" +
+            "key blob," +
+            "column1 text," +
+            "value blob," +
+            "PRIMARY KEY (key, column1))" +
+            " WITH COMPACT STORAGE",
+
+            "INSERT INTO bytes (key, column1, value) VALUES (asciiAsBlob('foo'), 'x', asciiAsBlob('Z'))",
+
+            "CREATE TABLE cc (key text, name text, value counter, PRIMARY KEY (key, name)) WITH COMPACT STORAGE",
+
+            "UPDATE cc SET value = value + 3 WHERE key = 'chuck' AND name = 'kick'",
+            "UPDATE cc SET value = value + 1 WHERE key = 'chuck' AND name = 'fist'",
+
+            "CREATE TABLE compo (" +
+            "key text," +
+            "column1 text," +
+            "column2 text," +
+            "value text," +
+            "PRIMARY KEY (key, column1, column2))" +
+            " WITH COMPACT STORAGE",
+
+            "INSERT INTO compo (key, column1, column2, value) VALUES ('punch', 'bruce', 'lee', 'ouch');",
+            "INSERT INTO compo (key, column1, column2, value) VALUES ('punch', 'bruce', 'bruce', 'hunh?');",
+            "INSERT INTO compo (key, column1, column2, value) VALUES ('kick', 'bruce', 'lee', 'oww');",
+            "INSERT INTO compo (key, column1, column2, value) VALUES ('kick', 'bruce', 'bruce', 'watch it, mate');",
+
+            "CREATE TABLE compo_int (" +
+            "key text," +
+            "column1 bigint," +
+            "column2 bigint," +
+            "value text," +
+            "PRIMARY KEY (key, column1, column2))" +
+            " WITH COMPACT STORAGE",
+
+            "INSERT INTO compo_int (key, column1, column2, value) VALUES ('clock', 1, 0, 'z');",
+            "INSERT INTO compo_int (key, column1, column2, value) VALUES ('clock', 1, 30, 'zzzz');",
+            "INSERT INTO compo_int (key, column1, column2, value) VALUES ('clock', 2, 30, 'daddy?');",
+            "INSERT INTO compo_int (key, column1, column2, value) VALUES ('clock', 6, 30, 'coffee...');",
+
+            "CREATE TABLE compo_int_copy (" +
+            "key text," +
+            "column1 bigint," +
+            "column2 bigint," +
+            "value text," +
+            "PRIMARY KEY (key, column1, column2))" +
+            " WITH COMPACT STORAGE",
+
+            "CREATE TABLE compo_key (" +
+            "key text," +
+            "column1 bigint," +
+            "column2 bigint," +
+            "value text," +
+            "PRIMARY KEY ((key, column1), column2))" +
+            " WITH COMPACT STORAGE",
+
+            "INSERT INTO compo_key (key, column1, column2, value) VALUES ('clock', 10, 1, 'z');",
+            "INSERT INTO compo_key (key, column1, column2, value) VALUES ('clock', 20, 1, 'zzzz');",
+            "INSERT INTO compo_key (key, column1, column2, value) VALUES ('clock', 30, 2, 'daddy?');",
+            "INSERT INTO compo_key (key, column1, column2, value) VALUES ('clock', 40, 6, 'coffee...');",
+
+            "CREATE TABLE compo_key_copy (" +
+            "key text," +
+            "column1 bigint," +
+            "column2 bigint," +
+            "value text," +
+            "PRIMARY KEY ((key, column1), column2))" +
+            " WITH COMPACT STORAGE",
     };
 
-    private static String[] deleteCopyOfSomeAppTableData = { "use thriftKs;",
-            "DEL CopyOfSomeApp ['foo']",
-            "DEL CopyOfSomeApp ['bar']",
-            "DEL CopyOfSomeApp ['baz']",
-            "DEL CopyOfSomeApp ['qux']"
+    private static String[] deleteCopyOfSomeAppTableData = {
+            "use thrift_ks;",
+            "DELETE FROM copy_of_some_app WHERE key = 'foo';",
+            "DELETE FROM copy_of_some_app WHERE key = 'bar';",
+            "DELETE FROM copy_of_some_app WHERE key = 'baz';",
+            "DELETE FROM copy_of_some_app WHERE key = 'qux';",
     };
 
     @BeforeClass
-    public static void setup() throws TTransportException, IOException, InterruptedException, ConfigurationException,
-                                      AuthenticationException, AuthorizationException, InvalidRequestException, UnavailableException, TimedOutException, TException, NotFoundException, CharacterCodingException, ClassNotFoundException, NoSuchFieldException, IllegalAccessException, InstantiationException
+    public static void setup() throws IOException, InterruptedException, ConfigurationException, TException,
+        ClassNotFoundException, NoSuchFieldException, IllegalAccessException, InstantiationException
     {
         startCassandra();
-        setupDataByCli(statements);
+        executeCQLStatements(statements);
         startHadoopCluster();
     }
 
     @Test
-    public void testCqlNativeStorage() throws IOException, ClassNotFoundException, TException, TimedOutException, NotFoundException, InvalidRequestException, NoSuchFieldException, UnavailableException, IllegalAccessException, InstantiationException, AuthenticationException, AuthorizationException
+    public void testCqlNativeStorage() throws IOException, ClassNotFoundException, TException, NoSuchFieldException,
+        IllegalAccessException, InstantiationException
     {
         //regular thrift column families
-        //input_cql=select * from "SomeApp" where token(key) > ? and token(key) <= ?
-        cqlStorageTest("data = load 'cql://thriftKs/SomeApp?" + defaultParameters + nativeParameters + "&input_cql=select%20*%20from%20%22SomeApp%22%20where%20token(key)%20%3E%20%3F%20and%20token(key)%20%3C%3D%20%3F' using CqlNativeStorage();");
+        //input_cql=select * from "some_app" where token(key) > ? and token(key) <= ?
+        cqlStorageTest("data = load 'cql://thrift_ks/some_app?" + defaultParameters + nativeParameters + "&input_cql=select%20*%20from%20%22some_app%22%20where%20token(key)%20%3E%20%3F%20and%20token(key)%20%3C%3D%20%3F' using CqlNativeStorage();");
 
-        //Test counter colun family
-        //input_cql=select * from "CC" where token(key) > ? and token(key) <= ?
-        cqlStorageCounterTableTest("cc_data = load 'cql://thriftKs/CC?" + defaultParameters + nativeParameters + "&input_cql=select%20*%20from%20%22CC%22%20where%20token(key)%20%3E%20%3F%20and%20token(key)%20%3C%3D%20%3F' using CqlNativeStorage();");
+        //Test counter column family
+        //input_cql=select * from "cc" where token(key) > ? and token(key) <= ?
+        cqlStorageCounterTableTest("cc_data = load 'cql://thrift_ks/cc?" + defaultParameters + nativeParameters + "&input_cql=select%20*%20from%20%22cc%22%20where%20token(key)%20%3E%20%3F%20and%20token(key)%20%3C%3D%20%3F' using CqlNativeStorage();");
 
         //Test composite column family
-        //input_cql=select * from "Compo" where token(key) > ? and token(key) <= ?
-        cqlStorageCompositeTableTest("compo_data = load 'cql://thriftKs/Compo?" + defaultParameters + nativeParameters + "&input_cql=select%20*%20from%20%22Compo%22%20where%20token(key)%20%3E%20%3F%20and%20token(key)%20%3C%3D%20%3F' using CqlNativeStorage();");
+        //input_cql=select * from "compo" where token(key) > ? and token(key) <= ?
+        cqlStorageCompositeTableTest("compo_data = load 'cql://thrift_ks/compo?" + defaultParameters + nativeParameters + "&input_cql=select%20*%20from%20%22compo%22%20where%20token(key)%20%3E%20%3F%20and%20token(key)%20%3C%3D%20%3F' using CqlNativeStorage();");
     }
 
     private void cqlStorageTest(String initialQuery) throws IOException
@@ -321,14 +302,12 @@ public class ThriftColumnFamilyTest extends PigTestBase
     }
 
     @Test
-    public void testCassandraStorageSchema() throws IOException, ClassNotFoundException, TException, TimedOutException, NotFoundException, InvalidRequestException, NoSuchFieldException, UnavailableException, IllegalAccessException, InstantiationException
+    public void testCassandraStorageSchema() throws IOException, ClassNotFoundException, TException, NoSuchFieldException,
+        IllegalAccessException, InstantiationException
     {
         //results: (qux,(atomic_weight,0.660161815846869),(created,1335890877),(name,User Qux),(percent,64.7),
-        //(rating,2),(score,12000),(vote_type,dislike),{(extra1,extra1),
-        //(extra2,extra2),(extra3,extra3),
-        //(extra4,extra4),(extra5,extra5),
-        //(extra6,extra6),(extra7,extra7)})
-        pig.registerQuery("rows = LOAD 'cassandra://thriftKs/SomeApp?" + defaultParameters + "' USING CassandraStorage();");
+        //(rating,2),(score,12000),(vote_type,dislike))
+        pig.registerQuery("rows = LOAD 'cassandra://thrift_ks/some_app?" + defaultParameters + "' USING CassandraStorage();");
 
         //schema: {key: chararray,atomic_weight: (name: chararray,value: double),created: (name: chararray,value: long),
         //name: (name: chararray,value: chararray),percent: (name: chararray,value: float),
@@ -358,48 +337,39 @@ public class ThriftColumnFamilyTest extends PigTestBase
                 column = (Tuple) t.get(7);
                 Assert.assertEquals(column.get(0), "vote_type");
                 Assert.assertEquals(column.get(1), "dislike");
-                DataBag columns = (DataBag) t.get(8);
-                Iterator<Tuple> iter = columns.iterator();
-                int i = 0;
-                while(iter.hasNext())
-                {
-                    i++;
-                    column = iter.next();
-                    Assert.assertEquals(column.get(0), "extra"+i);
-                }
-                Assert.assertEquals(7, columns.size());
             }
-
         }
     }
 
     @Test
-    public void testCassandraStorageFullCopy() throws IOException, ClassNotFoundException, TException, TimedOutException, NotFoundException, InvalidRequestException, NoSuchFieldException, UnavailableException, IllegalAccessException, InstantiationException, AuthenticationException, AuthorizationException
+    public void testCassandraStorageFullCopy() throws IOException, ClassNotFoundException, TException, NoSuchFieldException,
+        IllegalAccessException, InstantiationException
     {
         pig.setBatchOn();
-        pig.registerQuery("rows = LOAD 'cassandra://thriftKs/SomeApp?" + defaultParameters + "' USING CassandraStorage();");
+        pig.registerQuery("rows = LOAD 'cassandra://thrift_ks/some_app?" + defaultParameters + "' USING CassandraStorage();");
         //full copy
-        pig.registerQuery("STORE rows INTO 'cassandra://thriftKs/CopyOfSomeApp?" + defaultParameters + "' USING CassandraStorage();");
+        pig.registerQuery("STORE rows INTO 'cassandra://thrift_ks/copy_of_some_app?" + defaultParameters + "' USING CassandraStorage();");
         pig.executeBatch();
-        Assert.assertEquals("User Qux", getColumnValue("thriftKs", "CopyOfSomeApp", "name", "qux", "UTF8Type"));
-        Assert.assertEquals("dislike", getColumnValue("thriftKs", "CopyOfSomeApp", "vote_type", "qux", "UTF8Type"));
-        Assert.assertEquals("64.7", getColumnValue("thriftKs", "CopyOfSomeApp", "percent", "qux", "FloatType"));
+        Assert.assertEquals("User Qux", getColumnValue("thrift_ks", "copy_of_some_app", "name", "qux", "UTF8Type"));
+        Assert.assertEquals("dislike", getColumnValue("thrift_ks", "copy_of_some_app", "vote_type", "qux", "UTF8Type"));
+        Assert.assertEquals("64.7", getColumnValue("thrift_ks", "copy_of_some_app", "percent", "qux", "FloatType"));
     }
 
     @Test
-    public void testCassandraStorageSigleTupleCopy() throws IOException, ClassNotFoundException, TException, TimedOutException, NotFoundException, InvalidRequestException, NoSuchFieldException, UnavailableException, IllegalAccessException, InstantiationException, AuthenticationException, AuthorizationException
+    public void testCassandraStorageSigleTupleCopy() throws IOException, ClassNotFoundException, TException,
+        NoSuchFieldException, IllegalAccessException, InstantiationException
     {
-        executeCliStatements(deleteCopyOfSomeAppTableData);
+        executeCQLStatements(deleteCopyOfSomeAppTableData);
         pig.setBatchOn();
-        pig.registerQuery("rows = LOAD 'cassandra://thriftKs/SomeApp?" + defaultParameters + "' USING CassandraStorage();");
+        pig.registerQuery("rows = LOAD 'cassandra://thrift_ks/some_app?" + defaultParameters + "' USING CassandraStorage();");
         //sigle tuple
         pig.registerQuery("onecol = FOREACH rows GENERATE key, percent;");
-        pig.registerQuery("STORE onecol INTO 'cassandra://thriftKs/CopyOfSomeApp?" + defaultParameters + "' USING CassandraStorage();");
+        pig.registerQuery("STORE onecol INTO 'cassandra://thrift_ks/copy_of_some_app?" + defaultParameters + "' USING CassandraStorage();");
         pig.executeBatch();
         String value = null;
         try
         {
-            value = getColumnValue("thriftKs", "CopyOfSomeApp", "name", "qux", "UTF8Type");
+            value = getColumnValue("thrift_ks", "copy_of_some_app", "name", "qux", "UTF8Type");
         }
         catch (NotFoundException e)
         {
@@ -409,7 +379,7 @@ public class ThriftColumnFamilyTest extends PigTestBase
             Assert.fail();
         try
         {
-            value = getColumnValue("thriftKs", "CopyOfSomeApp", "vote_type", "qux", "UTF8Type");
+            value = getColumnValue("thrift_ks", "copy_of_some_app", "vote_type", "qux", "UTF8Type");
         }
         catch (NotFoundException e)
         {
@@ -417,23 +387,24 @@ public class ThriftColumnFamilyTest extends PigTestBase
         }
         if (value != null)
             Assert.fail();
-        Assert.assertEquals("64.7", getColumnValue("thriftKs", "CopyOfSomeApp", "percent", "qux", "FloatType"));
+        Assert.assertEquals("64.7", getColumnValue("thrift_ks", "copy_of_some_app", "percent", "qux", "FloatType"));
     }
 
     @Test
-    public void testCassandraStorageBagOnlyCopy() throws IOException, ClassNotFoundException, TException, TimedOutException, NotFoundException, InvalidRequestException, NoSuchFieldException, UnavailableException, IllegalAccessException, InstantiationException, AuthenticationException, AuthorizationException
+    public void testCassandraStorageBagOnlyCopy() throws IOException, ClassNotFoundException, TException,
+        NoSuchFieldException, IllegalAccessException, InstantiationException
     {
-        executeCliStatements(deleteCopyOfSomeAppTableData);
+        executeCQLStatements(deleteCopyOfSomeAppTableData);
         pig.setBatchOn();
-        pig.registerQuery("rows = LOAD 'cassandra://thriftKs/SomeApp?" + defaultParameters + "' USING CassandraStorage();");
+        pig.registerQuery("rows = LOAD 'cassandra://thrift_ks/some_app?" + defaultParameters + "' USING CassandraStorage();");
         //bag only
         pig.registerQuery("other = FOREACH rows GENERATE key, columns;");
-        pig.registerQuery("STORE other INTO 'cassandra://thriftKs/CopyOfSomeApp?" + defaultParameters + "' USING CassandraStorage();");
+        pig.registerQuery("STORE other INTO 'cassandra://thrift_ks/copy_of_some_app?" + defaultParameters + "' USING CassandraStorage();");
         pig.executeBatch();
         String value = null;
         try
         {
-            value = getColumnValue("thriftKs", "CopyOfSomeApp", "name", "qux", "UTF8Type");
+            value = getColumnValue("thrift_ks", "copy_of_some_app", "name", "qux", "UTF8Type");
         }
         catch (NotFoundException e)
         {
@@ -443,7 +414,7 @@ public class ThriftColumnFamilyTest extends PigTestBase
             Assert.fail();
         try
         {
-            value = getColumnValue("thriftKs", "CopyOfSomeApp", "vote_type", "qux", "UTF8Type");
+            value = getColumnValue("thrift_ks", "copy_of_some_app", "vote_type", "qux", "UTF8Type");
         }
         catch (NotFoundException e)
         {
@@ -453,7 +424,7 @@ public class ThriftColumnFamilyTest extends PigTestBase
             Assert.fail();
         try
         {
-            value = getColumnValue("thriftKs", "CopyOfSomeApp", "percent", "qux", "FloatType");
+            value = getColumnValue("thrift_ks", "copy_of_some_app", "percent", "qux", "FloatType");
         }
         catch (NotFoundException e)
         {
@@ -461,27 +432,27 @@ public class ThriftColumnFamilyTest extends PigTestBase
         }
         if (value != null)
             Assert.fail();
-        Assert.assertEquals("extra1", getColumnValue("thriftKs", "CopyOfSomeApp", "extra1", "qux", "UTF8Type"));
     }
 
     @Test
-    public void testCassandraStorageFilter() throws IOException, ClassNotFoundException, TException, TimedOutException, NotFoundException, InvalidRequestException, NoSuchFieldException, UnavailableException, IllegalAccessException, InstantiationException, AuthenticationException, AuthorizationException
+    public void testCassandraStorageFilter() throws IOException, ClassNotFoundException, TException, NoSuchFieldException,
+        IllegalAccessException, InstantiationException
     {
-        executeCliStatements(deleteCopyOfSomeAppTableData);
+        executeCQLStatements(deleteCopyOfSomeAppTableData);
         pig.setBatchOn();
-        pig.registerQuery("rows = LOAD 'cassandra://thriftKs/SomeApp?" + defaultParameters + "' USING CassandraStorage();");
+        pig.registerQuery("rows = LOAD 'cassandra://thrift_ks/some_app?" + defaultParameters + "' USING CassandraStorage();");
 
         //filter
         pig.registerQuery("likes = FILTER rows by vote_type.value eq 'like' and rating.value > 5;");
-        pig.registerQuery("STORE likes INTO 'cassandra://thriftKs/CopyOfSomeApp?" + defaultParameters + "' USING CassandraStorage();");
+        pig.registerQuery("STORE likes INTO 'cassandra://thrift_ks/copy_of_some_app?" + defaultParameters + "' USING CassandraStorage();");
         pig.executeBatch();
 
-        Assert.assertEquals("like", getColumnValue("thriftKs", "CopyOfSomeApp", "vote_type", "bar", "UTF8Type"));
-        Assert.assertEquals("like", getColumnValue("thriftKs", "CopyOfSomeApp", "vote_type", "foo", "UTF8Type"));
+        Assert.assertEquals("like", getColumnValue("thrift_ks", "copy_of_some_app", "vote_type", "bar", "UTF8Type"));
+        Assert.assertEquals("like", getColumnValue("thrift_ks", "copy_of_some_app", "vote_type", "foo", "UTF8Type"));
         String value = null;
         try
         {
-            value = getColumnValue("thriftKs", "CopyOfSomeApp", "vote_type", "qux", "UTF8Type");
+            value = getColumnValue("thrift_ks", "copy_of_some_app", "vote_type", "qux", "UTF8Type");
         }
         catch (NotFoundException e)
         {
@@ -491,7 +462,7 @@ public class ThriftColumnFamilyTest extends PigTestBase
             Assert.fail();
         try
         {
-            value = getColumnValue("thriftKs", "CopyOfSomeApp", "vote_type", "baz", "UTF8Type");
+            value = getColumnValue("thrift_ks", "copy_of_some_app", "vote_type", "baz", "UTF8Type");
         }
         catch (NotFoundException e)
         {
@@ -500,19 +471,18 @@ public class ThriftColumnFamilyTest extends PigTestBase
         if (value != null)
             Assert.fail();
 
-        executeCliStatements(deleteCopyOfSomeAppTableData);
+        executeCQLStatements(deleteCopyOfSomeAppTableData);
         pig.setBatchOn();
-        pig.registerQuery("rows = LOAD 'cassandra://thriftKs/SomeApp?" + defaultParameters + "' USING CassandraStorage();");
-        pig.registerQuery("dislikes_extras = FILTER rows by vote_type.value eq 'dislike' AND COUNT(columns) > 0;");
-        pig.registerQuery("STORE dislikes_extras INTO 'cassandra://thriftKs/CopyOfSomeApp?" + defaultParameters + "' USING CassandraStorage();");
-        pig.registerQuery("visible = FILTER rows BY COUNT(columns) == 0;");
+        pig.registerQuery("rows = LOAD 'cassandra://thrift_ks/some_app?" + defaultParameters + "' USING CassandraStorage();");
+        pig.registerQuery("dislikes_extras = FILTER rows by vote_type.value eq 'dislike';");
+        pig.registerQuery("STORE dislikes_extras INTO 'cassandra://thrift_ks/copy_of_some_app?" + defaultParameters + "' USING CassandraStorage();");
         pig.executeBatch();
-        Assert.assertEquals("dislike", getColumnValue("thriftKs", "CopyOfSomeApp", "vote_type", "baz", "UTF8Type"));
-        Assert.assertEquals("dislike", getColumnValue("thriftKs", "CopyOfSomeApp", "vote_type", "qux", "UTF8Type"));
+        Assert.assertEquals("dislike", getColumnValue("thrift_ks", "copy_of_some_app", "vote_type", "baz", "UTF8Type"));
+        Assert.assertEquals("dislike", getColumnValue("thrift_ks", "copy_of_some_app", "vote_type", "qux", "UTF8Type"));
         value = null;
         try
         {
-            value = getColumnValue("thriftKs", "CopyOfSomeApp", "vote_type", "bar", "UTF8Type");
+            value = getColumnValue("thrift_ks", "copy_of_some_app", "vote_type", "bar", "UTF8Type");
         }
         catch (NotFoundException e)
         {
@@ -522,7 +492,7 @@ public class ThriftColumnFamilyTest extends PigTestBase
             Assert.fail();
         try
         {
-            value = getColumnValue("thriftKs", "CopyOfSomeApp", "vote_type", "foo", "UTF8Type");
+            value = getColumnValue("thrift_ks", "copy_of_some_app", "vote_type", "foo", "UTF8Type");
         }
         catch (NotFoundException e)
         {
@@ -533,11 +503,11 @@ public class ThriftColumnFamilyTest extends PigTestBase
     }
 
     @Test
-    public void testCassandraStorageJoin() throws IOException, ClassNotFoundException, TException, TimedOutException, NotFoundException, InvalidRequestException, NoSuchFieldException, UnavailableException, IllegalAccessException, InstantiationException, AuthenticationException, AuthorizationException
+    public void testCassandraStorageJoin() throws IOException, ClassNotFoundException, TException, IllegalAccessException, InstantiationException
     {
         //test key types with a join
-        pig.registerQuery("U8 = load 'cassandra://thriftKs/U8?" + defaultParameters + "' using CassandraStorage();");
-        pig.registerQuery("Bytes = load 'cassandra://thriftKs/Bytes?" + defaultParameters + "' using CassandraStorage();");
+        pig.registerQuery("U8 = load 'cassandra://thrift_ks/u8?" + defaultParameters + "' using CassandraStorage();");
+        pig.registerQuery("Bytes = load 'cassandra://thrift_ks/bytes?" + defaultParameters + "' using CassandraStorage();");
 
         //cast key to chararray
         pig.registerQuery("b = foreach Bytes generate (chararray)key, columns;");
@@ -585,28 +555,24 @@ public class ThriftColumnFamilyTest extends PigTestBase
     }
 
     @Test
-    public void testCassandraStorageCounterCF() throws IOException, ClassNotFoundException, TException, TimedOutException, NotFoundException, InvalidRequestException, NoSuchFieldException, UnavailableException, IllegalAccessException, InstantiationException, AuthenticationException, AuthorizationException
+    public void testCassandraStorageCounterCF() throws IOException, ClassNotFoundException, TException, NoSuchFieldException,
+        IllegalAccessException, InstantiationException
     {
-        pig.registerQuery("rows = LOAD 'cassandra://thriftKs/SomeApp?" + defaultParameters + "' USING CassandraStorage();");
-
         //Test counter column family support
-        pig.registerQuery("CC = load 'cassandra://thriftKs/CC?" + defaultParameters + "' using CassandraStorage();");
+        pig.registerQuery("CC = load 'cassandra://thrift_ks/cc?" + defaultParameters + "' using CassandraStorage();");
         pig.registerQuery("total_hits = foreach CC generate key, SUM(columns.value);");
         //(chuck,4)
-        Iterator<Tuple> it = pig.openIterator("total_hits");
-        if (it.hasNext()) {
-            Tuple t = it.next();
-            Assert.assertEquals(t.get(0), "chuck");
-            Assert.assertEquals(t.get(1), 4l);
-        }
+        Tuple t = pig.openIterator("total_hits").next();
+        Assert.assertEquals(t.get(0), "chuck");
+        Assert.assertEquals(t.get(1), 4l);
     }
 
-    /** This test case fails due to antlr lib conflicts, Cassandra2.1 uses 3.2, Hive1.2 uses 3.4 */
-    //@Test
-    public void testCassandraStorageCompositeColumnCF() throws IOException, ClassNotFoundException, TException, TimedOutException, NotFoundException, InvalidRequestException, NoSuchFieldException, UnavailableException, IllegalAccessException, InstantiationException, AuthenticationException, AuthorizationException
+    @Test
+    public void testCassandraStorageCompositeColumnCF() throws IOException, ClassNotFoundException, TException,
+        NoSuchFieldException, IllegalAccessException, InstantiationException
     {
         //Test CompositeType
-        pig.registerQuery("compo = load 'cassandra://thriftKs/Compo?" + defaultParameters + "' using CassandraStorage();");
+        pig.registerQuery("compo = load 'cassandra://thrift_ks/compo?" + defaultParameters + "' using CassandraStorage();");
         pig.registerQuery("compo = foreach compo generate key as method, flatten(columns);");
         pig.registerQuery("lee = filter compo by columns::name == ('bruce','lee');");
 
@@ -626,7 +592,7 @@ public class ThriftColumnFamilyTest extends PigTestBase
                 Assert.assertEquals(t.get(2), "ouch");
         }
         Assert.assertEquals(count, 2);
-        pig.registerQuery("night = load 'cassandra://thriftKs/CompoInt?" + defaultParameters + "' using CassandraStorage();");
+        pig.registerQuery("night = load 'cassandra://thrift_ks/compo_int?" + defaultParameters + "' using CassandraStorage();");
         pig.registerQuery("night = foreach night generate flatten(columns);");
         pig.registerQuery("night = foreach night generate (int)columns::name.$0+(double)columns::name.$1/60 as hour, columns::value as noise;");
 
@@ -641,10 +607,10 @@ public class ThriftColumnFamilyTest extends PigTestBase
             Assert.assertEquals(t.get(1), "daddy?");
         }
         pig.setBatchOn();
-        pig.registerQuery("compo_int_rows = LOAD 'cassandra://thriftKs/CompoInt?" + defaultParameters + "' using CassandraStorage();");
-        pig.registerQuery("STORE compo_int_rows INTO 'cassandra://thriftKs/CompoIntCopy?" + defaultParameters + "' using CassandraStorage();");
+        pig.registerQuery("compo_int_rows = LOAD 'cassandra://thrift_ks/compo_int?" + defaultParameters + "' using CassandraStorage();");
+        pig.registerQuery("STORE compo_int_rows INTO 'cassandra://thrift_ks/compo_int_copy?" + defaultParameters + "' using CassandraStorage();");
         pig.executeBatch();
-        pig.registerQuery("compocopy_int_rows = LOAD 'cassandra://thriftKs/CompoIntCopy?" + defaultParameters + "' using CassandraStorage();");
+        pig.registerQuery("compocopy_int_rows = LOAD 'cassandra://thrift_ks/compo_int_copy?" + defaultParameters + "' using CassandraStorage();");
         //(clock,{((1,0),z),((1,30),zzzz),((2,30),daddy?),((6,30),coffee...)})
         it = pig.openIterator("compocopy_int_rows");
         count = 0;
@@ -652,11 +618,9 @@ public class ThriftColumnFamilyTest extends PigTestBase
             Tuple t = it.next();
             Assert.assertEquals(t.get(0), "clock");
             DataBag columns = (DataBag) t.get(1);
-            Iterator<Tuple> iter = columns.iterator();
-            while (iter.hasNext())
+            for (Tuple t1 : columns)
             {
-                count ++;
-                Tuple t1 = iter.next();
+                count++;
                 Tuple inner = (Tuple) t1.get(0);
                 if ((Long) inner.get(0) == 1L && (Long) inner.get(1) == 0L)
                     Assert.assertEquals(t1.get(1), "z");
@@ -672,10 +636,11 @@ public class ThriftColumnFamilyTest extends PigTestBase
     }
 
     @Test
-    public void testCassandraStorageCompositeKeyCF() throws IOException, ClassNotFoundException, TException, TimedOutException, NotFoundException, InvalidRequestException, NoSuchFieldException, UnavailableException, IllegalAccessException, InstantiationException, AuthenticationException, AuthorizationException
+    public void testCassandraStorageCompositeKeyCF() throws IOException, ClassNotFoundException, TException,
+        NoSuchFieldException, IllegalAccessException, InstantiationException
     {
         //Test CompositeKey
-        pig.registerQuery("compokeys = load 'cassandra://thriftKs/CompoKey?" + defaultParameters + "' using CassandraStorage();");
+        pig.registerQuery("compokeys = load 'cassandra://thrift_ks/compo_key?" + defaultParameters + "' using CassandraStorage();");
         pig.registerQuery("compokeys = filter compokeys by key.$1 == 40;");
         //((clock,40),{(6,coffee...)})
         Iterator<Tuple> it = pig.openIterator("compokeys");
@@ -694,10 +659,10 @@ public class ThriftColumnFamilyTest extends PigTestBase
             }
         }
         pig.setBatchOn();
-        pig.registerQuery("compo_key_rows = LOAD 'cassandra://thriftKs/CompoKey?" + defaultParameters + "' using CassandraStorage();");
-        pig.registerQuery("STORE compo_key_rows INTO 'cassandra://thriftKs/CompoKeyCopy?" + defaultParameters + "' using CassandraStorage();");
+        pig.registerQuery("compo_key_rows = LOAD 'cassandra://thrift_ks/compo_key?" + defaultParameters + "' using CassandraStorage();");
+        pig.registerQuery("STORE compo_key_rows INTO 'cassandra://thrift_ks/compo_key_copy?" + defaultParameters + "' using CassandraStorage();");
         pig.executeBatch();
-        pig.registerQuery("compo_key_copy_rows = LOAD 'cassandra://thriftKs/CompoKeyCopy?" + defaultParameters + "' using CassandraStorage();");
+        pig.registerQuery("compo_key_copy_rows = LOAD 'cassandra://thrift_ks/compo_key_copy?" + defaultParameters + "' using CassandraStorage();");
         //((clock,10),{(1,z)})
         //((clock,20),{(1,zzzz)})
         //((clock,30),{(2,daddy?)})
@@ -753,11 +718,10 @@ public class ThriftColumnFamilyTest extends PigTestBase
                 }
             }
         }
-        Assert.assertEquals(count, 4);
+        Assert.assertEquals(4, count);
     }
 
-    private String getColumnValue(String ks, String cf, String colName, String key, String validator)
-    throws AuthenticationException, AuthorizationException, InvalidRequestException, UnavailableException, TimedOutException, TException, NotFoundException, IOException
+    private String getColumnValue(String ks, String cf, String colName, String key, String validator) throws TException, IOException
     {
         Cassandra.Client client = getClient();
         client.set_keyspace(ks);
@@ -770,17 +734,4 @@ public class ThriftColumnFamilyTest extends PigTestBase
         ColumnOrSuperColumn got = client.get(key_user_id, cp, ConsistencyLevel.ONE);
         return parseType(validator).getString(got.getColumn().value);
     }
-
-    private void executeCliStatements(String[] statements) throws CharacterCodingException, ClassNotFoundException, TException, TimedOutException, NotFoundException, InvalidRequestException, NoSuchFieldException, UnavailableException, IllegalAccessException, InstantiationException
-    {
-        CliMain.connect("127.0.0.1", 9170);
-        try
-        {
-            for (String stmt : statements)
-                CliMain.processStatement(stmt);
-        }
-        catch (Exception e)
-        {
-        }
-    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/38027382/test/unit/org/apache/cassandra/cli/CliTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cli/CliTest.java b/test/unit/org/apache/cassandra/cli/CliTest.java
deleted file mode 100644
index 568ebf3..0000000
--- a/test/unit/org/apache/cassandra/cli/CliTest.java
+++ /dev/null
@@ -1,341 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.cassandra.cli;
-
-import org.apache.cassandra.SchemaLoader;
-import org.apache.cassandra.config.KSMetaData;
-import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.config.Schema;
-import org.apache.cassandra.exceptions.ConfigurationException;
-import org.apache.cassandra.locator.SimpleStrategy;
-import org.apache.cassandra.service.EmbeddedCassandraService;
-import org.apache.cassandra.thrift.*;
-import org.apache.thrift.TException;
-import org.junit.BeforeClass;
-import org.junit.Test;
-
-import java.io.ByteArrayOutputStream;
-import java.io.IOException;
-import java.io.PrintStream;
-import java.util.regex.Pattern;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
-public class CliTest
-{
-    private static final String KEYSPACE1 = "CliTest";
-    private static final String CF_STANDARD1 = "Standard1";
-
-    // please add new statements here so they could be auto-runned by this test.
-    private String[] statements = {
-        "use " + KEYSPACE1,
-        "create column family SecondaryIndicesWithoutIdxName" +
-                " with comparator = UTF8Type" +
-                " and default_validation_class = UTF8Type" +
-                " and column_metadata = [{column_name: profileId, validation_class: UTF8Type, index_type: KEYS}];",
-        "update column family SecondaryIndicesWithoutIdxName" +
-                " with column_metadata = " +
-                "[{column_name: profileId, validation_class: UTF8Type, index_type: KEYS}," +
-                 "{column_name: postedDate, validation_class: LongType}];",
-        "create column family 123 with comparator=UTF8Type and column_metadata=[{ column_name:world, validation_class:IntegerType, index_type:0, index_name:IdxName }, " +
-                                                                               "{ column_name:world2, validation_class:LongType, index_type:KEYS, index_name:LongIdxName}, " +
-                                                                               "{ column_name:617070, validation_class:UTF8Type, index_type:KEYS }, " +
-                                                                               "{ column_name:'-617071', validation_class:UTF8Type, index_type:KEYS }," +
-                                                                               "{ column_name:time_spent_uuid, validation_class:TimeUUIDType}] and default_validation_class=UTF8Type;",
-        "assume 123 keys as utf8;",
-        "set 123[hello][world] = 123848374878933948398384;",
-        "set 123[hello][test_quote] = 'value\\'';",
-        "set 123['k\\'ey'][VALUE] = 'VAL';",
-        "set 123['k\\'ey'][VALUE] = 'VAL\\'';",
-        "set 123[hello][-31337] = 'some string value';",
-        "list 123;",
-        "list 123[:];",
-        "list 123[456:];",
-        "list 123 limit 5;",
-        "list 123[12:15] limit 20;",
-        "list 123[12:15] columns 2;",
-        "list 123 columns 2 reversed;",
-        "list 123 limit 10 columns 2 reversed;",
-        "get 123[hello][-31337];",
-        "get 123[hello][world];",
-        "get 123[hello][test_quote];",
-        "get 123['k\\'ey'][VALUE]",
-        "set 123[hello][-31337] = -23876;",
-        "set 123[hello][world2] = 15;",
-        "get 123 where world2 = long(15);",
-        "get 123 where world2 = long(15);",
-        "get 123 where world2 = long(15);",
-        "del 123[utf8('hello')][utf8('world')];",
-        "del 123[hello][world2];",
-        "set 123['hello'][time_spent_uuid] = timeuuid(a8098c1a-f86e-11da-bd1a-00112444be1e);",
-        "create column family CF2 with comparator=IntegerType and default_validation_class=AsciiType;",
-        "assume CF2 keys as utf8;",
-        "set CF2['key'][98349387493847748398334] = 'some text';",
-        "get CF2['key'][98349387493847748398334];",
-        "set CF2['key'][98349387493] = 'some text other';",
-        "get CF2['key'][98349387493];",
-        "create column family CF3 with comparator=UTF8Type and column_metadata=[{column_name:'big world', validation_class:LongType, index_type:KEYS, index_name:WorldIdx}];",
-        "assume CF3 keys as utf8;",
-        "set CF3['hello']['big world'] = 3748;",
-        "get CF3['hello']['big world'];",
-        "list CF3;",
-        "list CF3[:];",
-        "list CF3[h:];",
-        "list CF3 limit 10;",
-        "list CF3[h:] limit 10;",
-        "create column family CF4 with comparator=IntegerType and column_metadata=[{column_name:9999, validation_class:LongType}];",
-        "assume CF4 keys as utf8;",
-        "set CF4['hello'][9999] = 1234;",
-        "get CF4['hello'][9999];",
-        "get CF4['hello'][9999] as Long;",
-        "get CF4['hello'][9999] as Bytes;",
-        "set CF4['hello'][9999] = Long(1234);",
-        "get CF4['hello'][9999];",
-        "get CF4['hello'][9999] as Long;",
-        "del CF4['hello'][9999];",
-        "get CF4['hello'][9999];",
-        "create column family sCf1 with column_type=Super and comparator=IntegerType and subcomparator=LongType and column_metadata=[{column_name:9999, validation_class:LongType}];",
-        "assume sCf1 keys as utf8;",
-        "set sCf1['hello'][1][9999] = 1234;",
-        "get sCf1['hello'][1][9999];",
-        "get sCf1['hello'][1][9999] as Long;",
-        "get sCf1['hello'][1][9999] as Bytes;",
-        "set sCf1['hello'][1][9999] = Long(1234);",
-        "set sCf1['hello'][-1][-12] = Long(5678);",
-        "get sCf1['hello'][-1][-12];",
-        "set sCf1['hello'][-1][-12] = -340897;",
-        "set sCf1['hello'][-1][-12] = integer(-340897);",
-        "get sCf1['hello'][1][9999];",
-        "get sCf1['hello'][1][9999] as Long;",
-        "del sCf1['hello'][1][9999];",
-        "get sCf1['hello'][1][9999];",
-        "set sCf1['hello'][1][9999] = Long(1234);",
-        "del sCf1['hello'][9999];",
-        "get sCf1['hello'][1][9999];",
-        "create column family 'Counter1' with comparator=UTF8Type and default_validation_class=CounterColumnType;",
-        "assume Counter1 keys as utf8;",
-        "incr Counter1['hello']['cassandra'];",
-        "incr Counter1['hello']['cassandra'] by 3;",
-        "incr Counter1['hello']['cassandra'] by -2;",
-        "decr Counter1['hello']['cassandra'];",
-        "decr Counter1['hello']['cassandra'] by 3;",
-        "decr Counter1['hello']['cassandra'] by -2;",
-        "get Counter1['hello']['cassandra'];",
-        "get Counter1['hello'];",
-        "truncate 123;",
-        "drop index on '123'.world2;",
-        "drop index on '123'.617070;",
-        "drop index on '123'.'-617071';",
-        "drop index on CF3.'big world';",
-        "update keyspace " + KEYSPACE1 + " with durable_writes = false;",
-        "assume 123 comparator as utf8;",
-        "assume 123 sub_comparator as integer;",
-        "assume 123 validator as lexicaluuid;",
-        "assume 123 keys as timeuuid;",
-        "create column family CF7;",
-        "assume CF7 keys as utf8;",
-        "set CF7[1][timeuuid()] = utf8(test1);",
-        "set CF7[2][lexicaluuid()] = utf8('hello world!');",
-        "set CF7[3][lexicaluuid(550e8400-e29b-41d4-a716-446655440000)] = utf8(test2);",
-        "set CF7[key2][timeuuid()] = utf8(test3);",
-        "assume CF7 comparator as lexicaluuid;",
-        "assume CF7 keys as utf8;",
-        "list CF7;",
-        "get CF7[3];",
-        "get CF7[3][lexicaluuid(550e8400-e29b-41d4-a716-446655440000)];",
-        "get sCf1['hello'][1][9999];",
-        "set sCf1['hello'][1][9999] = 938;",
-        "set sCf1['hello'][1][9999] = 938 with ttl = 30;",
-        "set sCf1['hello'][1][9999] = 938 with ttl = 560;",
-        "count sCf1[hello];",
-        "count sCf1[utf8('hello')];",
-        "count sCf1[utf8('hello')][integer(1)];",
-        "count sCf1[hello][1];",
-        "list sCf1;",
-        "del sCf1['hello'][1][9999];",
-        "assume sCf1 comparator as utf8;",
-        "create column family CF8;",
-        "drop column family cF8;",
-        "create keyspace TESTIN;",
-        "drop keyspace tesTIN;",
-        "update column family 123 with comparator=UTF8Type and column_metadata=[];",
-        "drop column family 123;",
-        "create column family myCF with column_type='Super' and comparator='UTF8Type' AND subcomparator='UTF8Type' AND default_validation_class=AsciiType;",
-        "assume myCF keys as utf8;",
-        "create column family Countries with comparator=UTF8Type and column_metadata=[ {column_name: name, validation_class: UTF8Type} ];",
-        "set Countries[11][name] = USA;",
-        "get Countries[11][name];",
-        "update column family Countries with compaction_strategy = 'org.apache.cassandra.db.compaction.SizeTieredCompactionStrategy';",
-        "create column family Cities with compaction_strategy = 'org.apache.cassandra.db.compaction.SizeTieredCompactionStrategy' and compaction_strategy_options = {min_sstable_size:1024};",
-        "set myCF['key']['scName']['firstname'] = 'John';",
-        "get myCF['key']['scName']",
-        "assume CF3 keys as utf8;",
-        "use " + KEYSPACE1 + ";",
-        "update keyspace " + KEYSPACE1 + " with placement_strategy='org.apache.cassandra.locator.NetworkTopologyStrategy';",
-        "update keyspace " + KEYSPACE1 + " with strategy_options={DC1:3, DC2:4, DC5:1};",
-        "describe cluster;",
-        "help describe cluster;",
-        "show cluster name",
-        "show api version",
-        "help help",
-        "help connect",
-        "help use",
-        "help describe",
-        "HELP exit",
-        "help QUIT",
-        "help show cluster name",
-        "help show keyspaces",
-        "help show schema",
-        "help show api version",
-        "help create keyspace",
-        "HELP update KEYSPACE",
-        "HELP CREATE column FAMILY",
-        "HELP UPDATE COLUMN family",
-        "HELP drop keyspace",
-        "help drop column family",
-        "HELP GET",
-        "HELP set",
-        "HELP DEL",
-        "HELP count",
-        "HELP list",
-        "HELP TRUNCATE",
-        "help assume",
-        "HELP",
-        "?",
-        "show schema",
-        "show schema " + KEYSPACE1
-    };
-
-    @BeforeClass
-    public static void defineSchema() throws ConfigurationException
-    {
-        SchemaLoader.prepareServer();
-        SchemaLoader.startGossiper();
-        SchemaLoader.createKeyspace(KEYSPACE1, true, false,
-                                    SimpleStrategy.class,
-                                    KSMetaData.optsWithRF(1),
-                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD1));
-    }
-
-    @Test
-    public void testCli() throws IOException, TException, TimedOutException, NotFoundException, SchemaDisagreementException, NoSuchFieldException, InvalidRequestException, UnavailableException, InstantiationException, IllegalAccessException
-    {
-        Schema.instance.clear(); // Schema are now written on disk and will be reloaded
-        new EmbeddedCassandraService().start();
-
-        // new error/output streams for CliSessionState
-        ByteArrayOutputStream errStream = new ByteArrayOutputStream();
-        ByteArrayOutputStream outStream = new ByteArrayOutputStream();
-
-        // checking if we can connect to the running cassandra node on localhost
-        CliMain.connect("127.0.0.1", DatabaseDescriptor.getRpcPort());
-
-        // setting new output stream
-        CliMain.sessionState.setOut(new PrintStream(outStream));
-        CliMain.sessionState.setErr(new PrintStream(errStream));
-
-        // re-creating keyspace for tests
-        try
-        {
-            // dropping in case it exists e.g. could be left from previous run
-            CliMain.processStatement(String.format("drop keyspace %s;", KEYSPACE1));
-        }
-        catch (Exception e)
-        {
-            // TODO check before drop so we don't have this fragile ignored exception block
-        }
-        CliMain.processStatement(String.format("create keyspace %s;", KEYSPACE1));
-
-        for (String statement : statements)
-        {
-            errStream.reset();
-            // System.out.println("Executing statement: " + statement);
-            CliMain.processStatement(statement);
-            String result = outStream.toString();
-            // System.out.println("Result:\n" + result);
-            if (statement.startsWith("show schema"))
-                assertEquals(errStream.toString() + "processing" + statement,
-                             "\nWARNING: CQL3 tables are intentionally omitted from 'show schema' output." + String.format("%n")
-                             + "See https://issues.apache.org/jira/browse/CASSANDRA-4377 for details.\n" + String.format("%n"),
-                             errStream.toString());
-            else
-                assertEquals(errStream.toString() + " processing " + statement, "", errStream.toString());
-
-            if (statement.startsWith("drop ") || statement.startsWith("create ") || statement.startsWith("update "))
-            {
-                assert Pattern.compile("(.{8})-(.{4})-(.{4})-(.{4})-(.{12}).*", Pattern.DOTALL).matcher(result).matches()
-                       : String.format("\"%s\" failed: %s", statement, result);
-            }
-            else if (statement.startsWith("set "))
-            {
-                 assertTrue(result.contains("Value inserted."));
-                 assertTrue(result.contains("Elapsed time:"));
-            }
-            else if (statement.startsWith("incr "))
-            {
-                 assertTrue(result.contains("Value incremented."));
-            }
-            else if (statement.startsWith("decr "))
-            {
-                 assertTrue(result.contains("Value decremented."));
-            }
-            else if (statement.startsWith("get "))
-            {
-                if (statement.contains("where"))
-                {
-                    assertTrue(result.startsWith("-------------------" + System.getProperty("line.separator") + "RowKey:"));
-                }
-                else if (statement.contains("Counter"))
-                {
-                    assertTrue(result.startsWith("=> (counter=") || result.startsWith("Value was not found"));
-                }
-                else
-                {
-                    assertTrue(result.startsWith("=> (name=") || result.startsWith("Value was not found"));
-                }
-                assertTrue(result.contains("Elapsed time:"));
-            }
-            else if (statement.startsWith("truncate "))
-            {
-                assertTrue(result.contains(" truncated."));
-            }
-            else if (statement.startsWith("assume "))
-            {
-                assertTrue(result.contains("successfully."));
-            }
-
-            outStream.reset(); // reset stream so we have only output from next statement all the time
-            errStream.reset(); // no errors to the end user.
-        }
-    }
-
-    @Test
-    public void testEscape()
-    {
-        //escaped is the string read from the cli.
-        String escaped = "backspace \\b tab \\t linefeed \\n form feed \\f carriage return \\r duble quote \\\" " +
-                "single quote \\' backslash \\\\";
-        String unescaped = "backspace \b tab \t linefeed \n form feed \f carriage return \r duble quote \" " +
-                "single quote ' backslash \\";
-        // when read from the cli may have single quotes around it
-        assertEquals(unescaped, CliUtils.unescapeSQLString("'" + escaped + "'"));
-        assertEquals(escaped, CliUtils.escapeSQLString(unescaped));
-    }
-}


[4/5] Remove cassandra-cli

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/38027382/src/java/org/apache/cassandra/cli/CliClient.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cli/CliClient.java b/src/java/org/apache/cassandra/cli/CliClient.java
deleted file mode 100644
index 703d4c1..0000000
--- a/src/java/org/apache/cassandra/cli/CliClient.java
+++ /dev/null
@@ -1,3230 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.cassandra.cli;
-
-import java.io.File;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.PrintStream;
-import java.net.InetAddress;
-import java.net.UnknownHostException;
-import java.nio.ByteBuffer;
-import java.nio.charset.CharacterCodingException;
-import java.nio.charset.StandardCharsets;
-import java.util.*;
-
-import com.google.common.base.Predicate;
-import com.google.common.collect.Collections2;
-import com.google.common.collect.Iterables;
-
-import org.apache.cassandra.serializers.MarshalException;
-import org.apache.commons.lang3.StringUtils;
-
-import org.antlr.runtime.tree.Tree;
-import org.apache.cassandra.auth.IAuthenticator;
-import org.apache.cassandra.exceptions.RequestValidationException;
-import org.apache.cassandra.db.ColumnFamilyStoreMBean;
-import org.apache.cassandra.db.compaction.CompactionManagerMBean;
-import org.apache.cassandra.db.compaction.OperationType;
-import org.apache.cassandra.db.marshal.*;
-import org.apache.cassandra.io.util.FileUtils;
-import org.apache.cassandra.locator.SimpleSnitch;
-import org.apache.cassandra.thrift.*;
-import org.apache.cassandra.tools.NodeProbe;
-import org.apache.cassandra.utils.ByteBufferUtil;
-import org.apache.cassandra.utils.FBUtilities;
-import org.apache.cassandra.utils.UUIDGen;
-import org.apache.thrift.TBaseHelper;
-import org.apache.thrift.TException;
-import org.codehaus.jackson.*;
-import org.yaml.snakeyaml.Yaml;
-import org.yaml.snakeyaml.constructor.Constructor;
-
-// Cli Client Side Library
-public class CliClient
-{
-    /**
-     * Available value conversion functions
-     * Used by convertValueByFunction(Tree functionCall) method
-     */
-    public enum Function
-    {
-        BYTES         (BytesType.instance),
-        INTEGER       (IntegerType.instance),
-        LONG          (LongType.instance),
-        INT           (Int32Type.instance),
-        LEXICALUUID   (LexicalUUIDType.instance),
-        TIMEUUID      (TimeUUIDType.instance),
-        UTF8          (UTF8Type.instance),
-        ASCII         (AsciiType.instance),
-        DOUBLE        (DoubleType.instance),
-        COUNTERCOLUMN (CounterColumnType.instance);
-
-        private AbstractType<?> validator;
-
-        Function(AbstractType<?> validator)
-        {
-            this.validator = validator;
-        }
-
-        public AbstractType<?> getValidator()
-        {
-            return this.validator;
-        }
-
-        public static String getFunctionNames()
-        {
-            Function[] functions = Function.values();
-            StringBuilder functionNames = new StringBuilder();
-
-            for (int i = 0; i < functions.length; i++)
-            {
-                StringBuilder currentName = new StringBuilder(functions[i].name().toLowerCase());
-                functionNames.append(currentName.append(((i != functions.length-1) ? ", " : ".")));
-            }
-
-            return functionNames.toString();
-        }
-    }
-
-    /*
-     * the <i>add keyspace</i> command requires a list of arguments,
-     *  this enum defines which arguments are valid
-     */
-    private enum AddKeyspaceArgument {
-        PLACEMENT_STRATEGY,
-        STRATEGY_OPTIONS,
-        DURABLE_WRITES
-    }
-
-    /*
-        * the <i>add column family</i> command requires a list of arguments,
-        *  this enum defines which arguments are valid.
-        */
-    protected enum ColumnFamilyArgument
-    {
-        COLUMN_TYPE,
-        COMPARATOR,
-        SUBCOMPARATOR,
-        COMMENT,
-        READ_REPAIR_CHANCE,
-        DCLOCAL_READ_REPAIR_CHANCE,
-        GC_GRACE,
-        COLUMN_METADATA,
-        MEMTABLE_OPERATIONS,
-        MEMTABLE_THROUGHPUT,
-        DEFAULT_VALIDATION_CLASS,
-        MIN_COMPACTION_THRESHOLD,
-        MAX_COMPACTION_THRESHOLD,
-        REPLICATE_ON_WRITE,
-        KEY_VALIDATION_CLASS,
-        COMPACTION_STRATEGY,
-        COMPACTION_STRATEGY_OPTIONS,
-        COMPRESSION_OPTIONS,
-        BLOOM_FILTER_FP_CHANCE,
-        INDEX_INTERVAL,
-        MEMTABLE_FLUSH_PERIOD_IN_MS,
-        CACHING,
-        CELLS_PER_ROW_TO_CACHE,
-        DEFAULT_TIME_TO_LIVE,
-        SPECULATIVE_RETRY,
-        POPULATE_IO_CACHE_ON_FLUSH,
-    }
-
-    private static final String DEFAULT_PLACEMENT_STRATEGY = "org.apache.cassandra.locator.NetworkTopologyStrategy";
-    private static final String NEWLINE = System.getProperty("line.separator");
-    private static final String TAB = "  ";
-
-    private final Cassandra.Client thriftClient;
-    private final CliSessionState sessionState;
-    private String keySpace = null;
-    private String username = null;
-    private final Map<String, KsDef> keyspacesMap = new HashMap<String, KsDef>();
-    private final Map<String, Map<String, CfDef>> cql3KeyspacesMap = new HashMap<String, Map<String, CfDef>>();
-    private final Map<String, AbstractType<?>> cfKeysComparators;
-    private ConsistencyLevel consistencyLevel = ConsistencyLevel.ONE;
-    private final CfAssumptions assumptions = new CfAssumptions();
-    private CliUserHelp help;
-    public CliClient(CliSessionState cliSessionState, Cassandra.Client thriftClient)
-    {
-        this.sessionState = cliSessionState;
-        this.thriftClient = thriftClient;
-        this.cfKeysComparators = new HashMap<String, AbstractType<?>>();
-        assumptions.readAssumptions();
-    }
-
-    private CliUserHelp getHelp()
-    {
-        if (help == null)
-            help = loadHelp();
-        return help;
-    }
-
-    private CliUserHelp loadHelp()
-    {
-        final InputStream is = CliClient.class.getClassLoader().getResourceAsStream("org/apache/cassandra/cli/CliHelp.yaml");
-        assert is != null;
-
-        try
-        {
-            final Constructor constructor = new Constructor(CliUserHelp.class);
-            final Yaml yaml = new Yaml(constructor);
-            return (CliUserHelp) yaml.load(is);
-        }
-        finally
-        {
-            FileUtils.closeQuietly(is);
-        }
-    }
-
-    public void printBanner()
-    {
-        sessionState.out.println("Welcome to Cassandra CLI version " + FBUtilities.getReleaseVersionString() + "\n");
-
-        sessionState.out.println("The CLI is deprecated and will be removed in Cassandra 3.0.  Consider migrating to cqlsh.");
-        sessionState.out.println("CQL is fully backwards compatible with Thrift data; see http://www.datastax.com/dev/blog/thrift-to-cql3\n");
-
-        sessionState.out.println(getHelp().banner);
-    }
-
-    private void printCQL3TablesWarning(String cmd)
-    {
-        sessionState.err.println("\nWARNING: CQL3 tables are intentionally omitted from '" + cmd + "' output.");
-        sessionState.err.println("See https://issues.apache.org/jira/browse/CASSANDRA-4377 for details.\n");
-    }
-
-    // Execute a CLI Statement
-    public void executeCLIStatement(String statement) throws CharacterCodingException, TException, TimedOutException, NotFoundException, NoSuchFieldException, InvalidRequestException, UnavailableException, InstantiationException, IllegalAccessException
-    {
-        Tree tree = CliCompiler.compileQuery(statement);
-        try
-        {
-            switch (tree.getType())
-            {
-                case CliParser.NODE_EXIT:
-                    cleanupAndExit();
-                    break;
-                case CliParser.NODE_THRIFT_GET:
-                    executeGet(tree);
-                    break;
-                case CliParser.NODE_THRIFT_GET_WITH_CONDITIONS:
-                    executeGetWithConditions(tree);
-                    break;
-                case CliParser.NODE_HELP:
-                    executeHelp(tree);
-                    break;
-                case CliParser.NODE_THRIFT_SET:
-                    executeSet(tree);
-                    break;
-                case CliParser.NODE_THRIFT_DEL:
-                    executeDelete(tree);
-                    break;
-                case CliParser.NODE_THRIFT_COUNT:
-                    executeCount(tree);
-                    break;
-                case CliParser.NODE_ADD_KEYSPACE:
-                    executeAddKeySpace(tree.getChild(0));
-                    break;
-                case CliParser.NODE_ADD_COLUMN_FAMILY:
-                    executeAddColumnFamily(tree.getChild(0));
-                    break;
-                case CliParser.NODE_UPDATE_KEYSPACE:
-                    executeUpdateKeySpace(tree.getChild(0));
-                    break;
-                case CliParser.NODE_UPDATE_COLUMN_FAMILY:
-                    executeUpdateColumnFamily(tree.getChild(0));
-                    break;
-                case CliParser.NODE_DEL_COLUMN_FAMILY:
-                    executeDelColumnFamily(tree);
-                    break;
-                case CliParser.NODE_DEL_KEYSPACE:
-                    executeDelKeySpace(tree);
-                    break;
-                case CliParser.NODE_SHOW_CLUSTER_NAME:
-                    executeShowClusterName();
-                    break;
-                case CliParser.NODE_SHOW_VERSION:
-                    executeShowVersion();
-                    break;
-                case CliParser.NODE_SHOW_KEYSPACES:
-                    executeShowKeySpaces();
-                    break;
-                case CliParser.NODE_SHOW_SCHEMA:
-                    executeShowSchema(tree);
-                    break;
-                case CliParser.NODE_DESCRIBE:
-                    executeDescribe(tree);
-                    break;
-                case CliParser.NODE_DESCRIBE_CLUSTER:
-                    executeDescribeCluster();
-                    break;
-                case CliParser.NODE_USE_TABLE:
-                    executeUseKeySpace(tree);
-                    break;
-                case CliParser.NODE_TRACE_NEXT_QUERY:
-                    executeTraceNextQuery();
-                    break;
-                case CliParser.NODE_CONNECT:
-                    executeConnect(tree);
-                    break;
-                case CliParser.NODE_LIST:
-                    executeList(tree);
-                    break;
-                case CliParser.NODE_TRUNCATE:
-                    executeTruncate(tree.getChild(0).getText());
-                    break;
-                case CliParser.NODE_ASSUME:
-                    executeAssumeStatement(tree);
-                    break;
-                case CliParser.NODE_CONSISTENCY_LEVEL:
-                    executeConsistencyLevelStatement(tree);
-                    break;
-                case CliParser.NODE_THRIFT_INCR:
-                    executeIncr(tree, 1L);
-                    break;
-                case CliParser.NODE_THRIFT_DECR:
-                    executeIncr(tree, -1L);
-                    break;
-                case CliParser.NODE_DROP_INDEX:
-                    executeDropIndex(tree);
-                    break;
-
-                case CliParser.NODE_NO_OP:
-                    // comment lines come here; they are treated as no ops.
-                    break;
-                default:
-                    sessionState.err.println("Invalid Statement (Type: " + tree.getType() + ")");
-                    if (sessionState.batch)
-                        System.exit(2);
-                    break;
-            }
-        }
-        catch (SchemaDisagreementException e)
-        {
-            throw new RuntimeException("schema does not match across nodes, (try again later).", e);
-        }
-    }
-
-    private void cleanupAndExit()
-    {
-        CliMain.disconnect();
-        assumptions.writeAssumptions();
-        System.exit(0);
-    }
-
-    public KsDef getKSMetaData(String keyspace)
-            throws NotFoundException, InvalidRequestException, TException
-    {
-        // Lazily lookup keyspace meta-data.
-        if (!(keyspacesMap.containsKey(keyspace)))
-        {
-            KsDef ksDef = thriftClient.describe_keyspace(keyspace);
-            keyspacesMap.put(keyspace, ksDef);
-            cql3KeyspacesMap.put(keyspace, loadCql3Defs(thriftClient, ksDef));
-            assumptions.replayAssumptions(keyspace);
-        }
-
-        return keyspacesMap.get(keyspace);
-    }
-
-    public static Map<String, CfDef> loadCql3Defs(Cassandra.Client thriftClient, KsDef thriftKs)
-    {
-        try
-        {
-            return loadCql3DefsUnchecked(thriftClient, thriftKs);
-        }
-        catch (Exception e)
-        {
-            throw new RuntimeException(e);
-        }
-    }
-
-    public static Map<String, CfDef> loadCql3DefsUnchecked(Cassandra.Client thriftClient, KsDef thriftKs) throws Exception
-    {
-        Map<String, CfDef> cql3Defs = new HashMap<String, CfDef>();
-
-        String query = "SELECT columnfamily_name, comparator, default_validator, key_validator FROM system.schema_columnfamilies WHERE keyspace_name='%s'";
-        String formatted = String.format(query, thriftKs.name);
-        CqlResult result = thriftClient.execute_cql3_query(ByteBufferUtil.bytes(formatted),
-                                                           Compression.NONE,
-                                                           ConsistencyLevel.ONE);
-        outer:
-        for (CqlRow row : result.rows)
-        {
-            Column rawName = row.columns.get(0);
-            assert ByteBufferUtil.string(ByteBuffer.wrap(rawName.getName())).equals("columnfamily_name");
-            String name = ByteBufferUtil.string(ByteBuffer.wrap(rawName.getValue()));
-
-            Column rawComparator = row.columns.get(1);
-            assert ByteBufferUtil.string(ByteBuffer.wrap(rawComparator.getName())).equals("comparator");
-            String comparator = ByteBufferUtil.string(ByteBuffer.wrap(rawComparator.getValue()));
-
-            Column rawValidator = row.columns.get(2);
-            assert ByteBufferUtil.string(ByteBuffer.wrap(rawValidator.getName())).equals("default_validator");
-            String validator = ByteBufferUtil.string(ByteBuffer.wrap(rawValidator.getValue()));
-
-            Column rawKeyValidator = row.columns.get(3);
-            assert ByteBufferUtil.string(ByteBuffer.wrap(rawKeyValidator.getName())).equals("key_validator");
-            String keyValidator = ByteBufferUtil.string(ByteBuffer.wrap(rawKeyValidator.getValue()));
-
-            for (CfDef cf_def : thriftKs.cf_defs)
-            {
-                if (cf_def.name.equals(name))
-                    continue outer;
-            }
-
-            CfDef thriftDef = new CfDef(thriftKs.name, name)
-                              .setComparator_type(comparator)
-                              .setDefault_validation_class(validator)
-                              .setKey_validation_class(keyValidator)
-                              .setColumn_metadata(Collections.<ColumnDef>emptyList());
-            cql3Defs.put(name, thriftDef);
-        }
-
-        return cql3Defs;
-    }
-
-    private void executeHelp(Tree tree)
-    {
-        if (tree.getChildCount() > 0)
-        {
-            String token = tree.getChild(0).getText();
-            for (CliCommandHelp ch : getHelp().commands)
-            {
-                if (token.equals(ch.name))
-                {
-                    sessionState.out.println(ch.help);
-                    break;
-                }
-            }
-        }
-        else
-        {
-            sessionState.out.println(getHelp().help);
-        }
-    }
-
-    private void executeCount(Tree statement)
-            throws TException, InvalidRequestException, UnavailableException, TimedOutException
-    {
-        if (!CliMain.isConnected() || !hasKeySpace())
-            return;
-
-        Tree columnFamilySpec = statement.getChild(0);
-
-        String columnFamily = CliCompiler.getColumnFamily(columnFamilySpec, currentCfDefs());
-        int columnSpecCnt = CliCompiler.numColumnSpecifiers(columnFamilySpec);
-
-        ColumnParent colParent = new ColumnParent(columnFamily).setSuper_column((ByteBuffer) null);
-
-        if (columnSpecCnt != 0)
-        {
-            Tree columnTree = columnFamilySpec.getChild(2);
-
-            byte[] superColumn = (columnTree.getType() == CliParser.FUNCTION_CALL)
-                                  ? convertValueByFunction(columnTree, null, null).array()
-                                  : columnNameAsByteArray(CliCompiler.getColumn(columnFamilySpec, 0), columnFamily);
-
-            colParent = new ColumnParent(columnFamily).setSuper_column(superColumn);
-        }
-
-        SliceRange range = new SliceRange(ByteBufferUtil.EMPTY_BYTE_BUFFER, ByteBufferUtil.EMPTY_BYTE_BUFFER, false, Integer.MAX_VALUE);
-        SlicePredicate predicate = new SlicePredicate().setColumn_names(null).setSlice_range(range);
-
-        int count = thriftClient.get_count(getKeyAsBytes(columnFamily, columnFamilySpec.getChild(1)), colParent, predicate, consistencyLevel);
-        sessionState.out.printf("%d cells%n", count);
-    }
-
-    private Iterable<CfDef> currentCfDefs()
-    {
-        return Iterables.concat(keyspacesMap.get(keySpace).cf_defs, cql3KeyspacesMap.get(keySpace).values());
-    }
-
-    private void executeDelete(Tree statement)
-            throws TException, InvalidRequestException, UnavailableException, TimedOutException
-    {
-        if (!CliMain.isConnected() || !hasKeySpace())
-            return;
-
-        Tree columnFamilySpec = statement.getChild(0);
-
-        String columnFamily = CliCompiler.getColumnFamily(columnFamilySpec, currentCfDefs());
-        CfDef cfDef = getCfDef(columnFamily);
-
-        ByteBuffer key = getKeyAsBytes(columnFamily, columnFamilySpec.getChild(1));
-        int columnSpecCnt = CliCompiler.numColumnSpecifiers(columnFamilySpec);
-
-        byte[] superColumnName = null;
-        byte[] columnName = null;
-
-        if ((columnSpecCnt < 0) || (columnSpecCnt > 2))
-        {
-            sessionState.out.println("Invalid row, super column, or column specification.");
-            return;
-        }
-
-        long startTime = System.nanoTime();
-
-        Tree columnTree = (columnSpecCnt >= 1)
-                           ? columnFamilySpec.getChild(2)
-                           : null;
-
-        Tree subColumnTree = (columnSpecCnt == 2)
-                              ? columnFamilySpec.getChild(3)
-                              : null;
-
-        if (columnSpecCnt == 1)
-        {
-            assert columnTree != null;
-
-            byte[] columnNameBytes = (columnTree.getType() == CliParser.FUNCTION_CALL)
-                                      ? convertValueByFunction(columnTree, null, null).array()
-                                      : columnNameAsByteArray(CliCompiler.getColumn(columnFamilySpec, 0), cfDef);
-
-            boolean isSuper = cfDef.column_type.equals("Super");
-            if (isSuper)
-                superColumnName = columnNameBytes;
-            else
-                columnName = columnNameBytes;
-        }
-        else if (columnSpecCnt == 2)
-        {
-            assert columnTree != null;
-            assert subColumnTree != null;
-
-            // keyspace.cf['key']['column']['column']
-            superColumnName = (columnTree.getType() == CliParser.FUNCTION_CALL)
-                                      ? convertValueByFunction(columnTree, null, null).array()
-                                      : columnNameAsByteArray(CliCompiler.getColumn(columnFamilySpec, 0), cfDef);
-
-            columnName = (subColumnTree.getType() == CliParser.FUNCTION_CALL)
-                                         ? convertValueByFunction(subColumnTree, null, null).array()
-                                         : subColumnNameAsByteArray(CliCompiler.getColumn(columnFamilySpec, 1), cfDef);
-        }
-
-        ColumnPath path = new ColumnPath(columnFamily);
-        if (superColumnName != null)
-            path.setSuper_column(superColumnName);
-
-        if (columnName != null)
-            path.setColumn(columnName);
-
-        if (isCounterCF(cfDef))
-        {
-            thriftClient.remove_counter(key, path, consistencyLevel);
-        }
-        else
-        {
-            thriftClient.remove(key, path, FBUtilities.timestampMicros(), consistencyLevel);
-        }
-        sessionState.out.println(String.format("%s removed.", (columnSpecCnt == 0) ? "row" : "cell"));
-        elapsedTime(startTime);
-    }
-
-    private void doSlice(String keyspace, ByteBuffer key, String columnFamily, byte[] superColumnName, int limit)
-            throws InvalidRequestException, UnavailableException, TimedOutException, TException, NotFoundException
-    {
-
-        long startTime = System.nanoTime();
-        ColumnParent parent = new ColumnParent(columnFamily);
-        if(superColumnName != null)
-            parent.setSuper_column(superColumnName);
-
-        SliceRange range = new SliceRange(ByteBufferUtil.EMPTY_BYTE_BUFFER, ByteBufferUtil.EMPTY_BYTE_BUFFER, false, limit);
-        SlicePredicate predicate = new SlicePredicate().setColumn_names(null).setSlice_range(range);
-
-        CfDef cfDef = getCfDef(columnFamily);
-        boolean isSuperCF = cfDef.column_type.equals("Super");
-
-        List<ColumnOrSuperColumn> columns = thriftClient.get_slice(key, parent, predicate, consistencyLevel);
-        AbstractType<?> validator;
-
-        // Print out super columns or columns.
-        for (ColumnOrSuperColumn cosc : columns)
-        {
-            if (cosc.isSetSuper_column())
-            {
-                SuperColumn superColumn = cosc.super_column;
-
-                sessionState.out.printf("=> (super_column=%s,", formatColumnName(keyspace, columnFamily, superColumn.name));
-                for (Column col : superColumn.getColumns())
-                {
-                    validator = getValidatorForValue(cfDef, col.getName());
-                    sessionState.out.printf("%n     (name=%s, value=%s, timestamp=%d%s)", formatSubcolumnName(keyspace, columnFamily, col.name),
-                                                    validator.getString(col.value), col.timestamp,
-                                                    col.isSetTtl() ? String.format(", ttl=%d", col.getTtl()) : "");
-                }
-
-                sessionState.out.println(")");
-            }
-            else if (cosc.isSetColumn())
-            {
-                Column column = cosc.column;
-                validator = getValidatorForValue(cfDef, column.getName());
-
-                String formattedName = isSuperCF
-                                       ? formatSubcolumnName(keyspace, columnFamily, column.name)
-                                       : formatColumnName(keyspace, columnFamily, column.name);
-
-                sessionState.out.printf("=> (name=%s, value=%s, timestamp=%d%s)%n",
-                                        formattedName,
-                                        validator.getString(column.value),
-                                        column.timestamp,
-                                        column.isSetTtl() ? String.format(", ttl=%d", column.getTtl()) : "");
-            }
-            else if (cosc.isSetCounter_super_column())
-            {
-                CounterSuperColumn superColumn = cosc.counter_super_column;
-
-                sessionState.out.printf("=> (super_column=%s,", formatColumnName(keyspace, columnFamily, superColumn.name));
-                for (CounterColumn col : superColumn.getColumns())
-                {
-                    sessionState.out.printf("%n     (counter=%s, value=%s)", formatSubcolumnName(keyspace, columnFamily, col.name), col.value);
-                }
-                sessionState.out.println(")");
-            }
-            else // cosc.isSetCounter_column()
-            {
-                CounterColumn column = cosc.counter_column;
-                String formattedName = isSuperCF
-                                       ? formatSubcolumnName(keyspace, columnFamily, column.name)
-                                       : formatColumnName(keyspace, columnFamily, column.name);
-
-                sessionState.out.printf("=> (counter=%s, value=%s)%n", formattedName, column.value);
-            }
-        }
-
-        sessionState.out.println("Returned " + columns.size() + " results.");
-        elapsedTime(startTime);
-    }
-
-    private AbstractType<?> getFormatType(String compareWith)
-    {
-        Function function;
-
-        try
-        {
-            function = Function.valueOf(compareWith.toUpperCase());
-        }
-        catch (IllegalArgumentException e)
-        {
-            try
-            {
-                return TypeParser.parse(compareWith);
-            }
-            catch (RequestValidationException ce)
-            {
-                String message = String.format("Unknown comparator '%s'. Available functions: %s", compareWith, Function.getFunctionNames());
-                throw new RuntimeException(message, e);
-            }
-        }
-
-        return function.getValidator();
-    }
-
-    // Execute GET statement
-    private void executeGet(Tree statement)
-            throws TException, NotFoundException, InvalidRequestException, UnavailableException, TimedOutException
-    {
-        if (!CliMain.isConnected() || !hasKeySpace())
-            return;
-        long startTime = System.nanoTime();
-        Tree columnFamilySpec = statement.getChild(0);
-        String columnFamily = CliCompiler.getColumnFamily(columnFamilySpec, currentCfDefs());
-        ByteBuffer key = getKeyAsBytes(columnFamily, columnFamilySpec.getChild(1));
-        int columnSpecCnt = CliCompiler.numColumnSpecifiers(columnFamilySpec);
-        CfDef cfDef = getCfDef(columnFamily);
-        boolean isSuper = cfDef.column_type.equals("Super");
-
-        byte[] superColumnName = null;
-        ByteBuffer columnName;
-
-        Tree typeTree = null;
-        Tree limitTree = null;
-
-        int limit = 1000000;
-
-        if (statement.getChildCount() >= 2)
-        {
-            if (statement.getChild(1).getType() == CliParser.CONVERT_TO_TYPE)
-            {
-                typeTree = statement.getChild(1).getChild(0);
-                if (statement.getChildCount() == 3)
-                    limitTree = statement.getChild(2).getChild(0);
-            }
-            else
-            {
-                limitTree = statement.getChild(1).getChild(0);
-            }
-        }
-
-        if (limitTree != null)
-        {
-            limit = Integer.parseInt(limitTree.getText());
-
-            if (limit == 0)
-            {
-                throw new IllegalArgumentException("LIMIT should be greater than zero.");
-            }
-        }
-
-        // keyspace.cf['key'] -- row slice
-        if (columnSpecCnt == 0)
-        {
-            doSlice(keySpace, key, columnFamily, superColumnName, limit);
-            return;
-        }
-        // keyspace.cf['key']['column'] -- slice of a super, or get of a standard
-        else if (columnSpecCnt == 1)
-        {
-            columnName = getColumnName(columnFamily, columnFamilySpec.getChild(2));
-
-            if (isSuper)
-            {
-                superColumnName = columnName.array();
-                doSlice(keySpace, key, columnFamily, superColumnName, limit);
-                return;
-            }
-        }
-        // keyspace.cf['key']['column']['column'] -- get of a sub-column
-        else if (columnSpecCnt == 2)
-        {
-            superColumnName = getColumnName(columnFamily, columnFamilySpec.getChild(2)).array();
-            columnName = getSubColumnName(columnFamily, columnFamilySpec.getChild(3));
-        }
-        // The parser groks an arbitrary number of these so it is possible to get here.
-        else
-        {
-            sessionState.out.println("Invalid row, super column, or column specification.");
-            return;
-        }
-
-        AbstractType<?> validator = getValidatorForValue(cfDef, TBaseHelper.byteBufferToByteArray(columnName));
-
-        // Perform a get()
-        ColumnPath path = new ColumnPath(columnFamily);
-        if(superColumnName != null) path.setSuper_column(superColumnName);
-        path.setColumn(columnName);
-
-        if (isCounterCF(cfDef))
-        {
-            doGetCounter(key, path);
-            elapsedTime(startTime);
-            return;
-        }
-
-        Column column;
-        try
-        {
-            column = thriftClient.get(key, path, consistencyLevel).column;
-        }
-        catch (NotFoundException e)
-        {
-            sessionState.out.println("Value was not found");
-            elapsedTime(startTime);
-            return;
-        }
-
-        byte[] columnValue = column.getValue();
-        String valueAsString;
-
-        // we have ^(CONVERT_TO_TYPE <type>) inside of GET statement
-        // which means that we should try to represent byte[] value according
-        // to specified type
-        if (typeTree != null)
-        {
-            // .getText() will give us <type>
-            String typeName = CliUtils.unescapeSQLString(typeTree.getText());
-            // building AbstractType from <type>
-            AbstractType<?> valueValidator = getFormatType(typeName);
-
-            // setting value for output
-            valueAsString = valueValidator.getString(ByteBuffer.wrap(columnValue));
-            // updating column value validator class
-            updateColumnMetaData(cfDef, columnName, valueValidator.toString());
-        }
-        else
-        {
-            valueAsString = (validator == null) ? new String(columnValue, StandardCharsets.UTF_8) : validator.getString(ByteBuffer.wrap(columnValue));
-        }
-
-        String formattedColumnName = isSuper
-                                     ? formatSubcolumnName(keySpace, columnFamily, column.name)
-                                     : formatColumnName(keySpace, columnFamily, column.name);
-
-        // print results
-        sessionState.out.printf("=> (name=%s, value=%s, timestamp=%d%s)%n",
-                                formattedColumnName,
-                                valueAsString,
-                                column.timestamp,
-                                column.isSetTtl() ? String.format(", ttl=%d", column.getTtl()) : "");
-        elapsedTime(startTime);
-    }
-
-    private void doGetCounter(ByteBuffer key, ColumnPath path)
-            throws TException, NotFoundException, InvalidRequestException, UnavailableException, TimedOutException
-    {
-        boolean isSuper = path.super_column != null;
-
-        CounterColumn column;
-        try
-        {
-            column = thriftClient.get(key, path, consistencyLevel).counter_column;
-        }
-        catch (NotFoundException e)
-        {
-            sessionState.out.println("Value was not found");
-            return;
-        }
-
-        String formattedColumnName = isSuper
-                                     ? formatSubcolumnName(keySpace, path.column_family, column.name)
-                                     : formatColumnName(keySpace, path.column_family, column.name);
-
-        // print results
-        sessionState.out.printf("=> (counter=%s, value=%d)%n",
-                                formattedColumnName,
-                                column.value);
-    }
-
-    /**
-     * Process get operation with conditions (using Thrift get_indexed_slices method)
-     * @param statement - tree representation of the current statement
-     * Format: ^(NODE_THRIFT_GET_WITH_CONDITIONS cf ^(CONDITIONS ^(CONDITION >= column1 value1) ...) ^(NODE_LIMIT int)*)
-     */
-    private void executeGetWithConditions(Tree statement)
-    {
-        if (!CliMain.isConnected() || !hasKeySpace())
-            return;
-
-        long startTime = System.nanoTime();
-
-        IndexClause clause = new IndexClause();
-        String columnFamily = CliCompiler.getColumnFamily(statement, currentCfDefs());
-        // ^(CONDITIONS ^(CONDITION $column $value) ...)
-        Tree conditions = statement.getChild(1);
-
-        // fetching column family definition
-        CfDef columnFamilyDef = getCfDef(columnFamily);
-
-        // fetching all columns
-        SlicePredicate predicate = new SlicePredicate();
-        SliceRange sliceRange = new SliceRange();
-        sliceRange.setStart(new byte[0]).setFinish(new byte[0]);
-        predicate.setSlice_range(sliceRange);
-
-        for (int i = 0; i < conditions.getChildCount(); i++)
-        {
-            // ^(CONDITION operator $column $value)
-            Tree condition = conditions.getChild(i);
-
-            // =, >, >=, <, <=
-            String operator = condition.getChild(0).getText();
-            String columnNameString  = CliUtils.unescapeSQLString(condition.getChild(1).getText());
-            // it could be a basic string or function call
-            Tree valueTree = condition.getChild(2);
-
-            try
-            {
-                ByteBuffer value;
-                ByteBuffer columnName = columnNameAsBytes(columnNameString, columnFamily);
-
-                if (valueTree.getType() == CliParser.FUNCTION_CALL)
-                {
-                    value = convertValueByFunction(valueTree, columnFamilyDef, columnName);
-                }
-                else
-                {
-                    String valueString = CliUtils.unescapeSQLString(valueTree.getText());
-                    value = columnValueAsBytes(columnName, columnFamily, valueString);
-                }
-
-                // index operator from string
-                IndexOperator idxOperator = CliUtils.getIndexOperator(operator);
-                // adding new index expression into index clause
-                clause.addToExpressions(new IndexExpression(columnName, idxOperator, value));
-            }
-            catch (Exception e)
-            {
-                throw new RuntimeException(e);
-            }
-        }
-
-        List<KeySlice> slices;
-        clause.setStart_key(new byte[] {});
-
-        // when we have ^(NODE_LIMIT Integer)
-        if (statement.getChildCount() == 3)
-        {
-            Tree limitNode = statement.getChild(2);
-            int limitValue = Integer.parseInt(limitNode.getChild(0).getText());
-
-            if (limitValue == 0)
-            {
-                throw new IllegalArgumentException("LIMIT should be greater than zero.");
-            }
-
-            clause.setCount(limitValue);
-        }
-
-        try
-        {
-            ColumnParent parent = new ColumnParent(columnFamily);
-            slices = thriftClient.get_indexed_slices(parent, clause, predicate, consistencyLevel);
-            printSliceList(columnFamilyDef, slices);
-        }
-        catch (InvalidRequestException e)
-        {
-            throw new RuntimeException(e);
-        }
-        catch (Exception e)
-        {
-            throw new RuntimeException(e);
-        }
-        elapsedTime(startTime);
-    }
-
-    // Execute SET statement
-    private void executeSet(Tree statement)
-        throws TException, InvalidRequestException, UnavailableException, TimedOutException
-    {
-        if (!CliMain.isConnected() || !hasKeySpace())
-            return;
-
-        long startTime = System.nanoTime();
-        // ^(NODE_COLUMN_ACCESS <cf> <key> <column>)
-        Tree columnFamilySpec = statement.getChild(0);
-        Tree keyTree = columnFamilySpec.getChild(1); // could be a function or regular text
-
-        String columnFamily = CliCompiler.getColumnFamily(columnFamilySpec, currentCfDefs());
-        CfDef cfDef = getCfDef(columnFamily);
-        int columnSpecCnt = CliCompiler.numColumnSpecifiers(columnFamilySpec);
-        String value = CliUtils.unescapeSQLString(statement.getChild(1).getText());
-        Tree valueTree = statement.getChild(1);
-
-        byte[] superColumnName = null;
-        ByteBuffer columnName;
-
-        // keyspace.cf['key']
-        if (columnSpecCnt == 0)
-        {
-            sessionState.err.println("No cell name specified, (type 'help;' or '?' for help on syntax).");
-            return;
-        }
-        // keyspace.cf['key']['column'] = 'value'
-        else if (columnSpecCnt == 1)
-        {
-            // get the column name
-            if (cfDef.column_type.equals("Super"))
-            {
-                sessionState.out.println("Column family " + columnFamily + " may only contain SuperColumns");
-                return;
-            }
-            columnName = getColumnName(columnFamily, columnFamilySpec.getChild(2));
-        }
-        // keyspace.cf['key']['super_column']['column'] = 'value'
-        else
-        {
-            assert (columnSpecCnt == 2) : "serious parsing error (this is a bug).";
-
-            superColumnName = getColumnName(columnFamily, columnFamilySpec.getChild(2)).array();
-            columnName = getSubColumnName(columnFamily, columnFamilySpec.getChild(3));
-        }
-
-        ByteBuffer columnValueInBytes;
-
-        switch (valueTree.getType())
-        {
-        case CliParser.FUNCTION_CALL:
-            columnValueInBytes = convertValueByFunction(valueTree, cfDef, columnName, true);
-            break;
-        default:
-            columnValueInBytes = columnValueAsBytes(columnName, columnFamily, value);
-        }
-
-        ColumnParent parent = new ColumnParent(columnFamily);
-        if(superColumnName != null)
-            parent.setSuper_column(superColumnName);
-
-        Column columnToInsert = new Column(columnName).setValue(columnValueInBytes).setTimestamp(FBUtilities.timestampMicros());
-
-        // children count = 3 mean that we have ttl in arguments
-        if (statement.getChildCount() == 3)
-        {
-            String ttl = statement.getChild(2).getText();
-
-            try
-            {
-                columnToInsert.setTtl(Integer.parseInt(ttl));
-            }
-            catch (NumberFormatException e)
-            {
-                sessionState.err.println(String.format("TTL '%s' is invalid, should be a positive integer.", ttl));
-                return;
-            }
-            catch (Exception e)
-            {
-                throw new RuntimeException(e);
-            }
-        }
-
-        // do the insert
-        thriftClient.insert(getKeyAsBytes(columnFamily, keyTree), parent, columnToInsert, consistencyLevel);
-        sessionState.out.println("Value inserted.");
-        elapsedTime(startTime);
-    }
-
-    // Execute INCR statement
-    private void executeIncr(Tree statement, long multiplier)
-            throws TException, NotFoundException, InvalidRequestException, UnavailableException, TimedOutException
-    {
-        if (!CliMain.isConnected() || !hasKeySpace())
-            return;
-
-        Tree columnFamilySpec = statement.getChild(0);
-
-        String columnFamily = CliCompiler.getColumnFamily(columnFamilySpec, currentCfDefs());
-        ByteBuffer key = getKeyAsBytes(columnFamily, columnFamilySpec.getChild(1));
-        int columnSpecCnt = CliCompiler.numColumnSpecifiers(columnFamilySpec);
-
-        byte[] superColumnName = null;
-        ByteBuffer columnName;
-
-        // keyspace.cf['key']['column'] -- incr standard
-        if (columnSpecCnt == 1)
-        {
-            columnName = getColumnName(columnFamily, columnFamilySpec.getChild(2));
-        }
-        // keyspace.cf['key']['column']['column'] -- incr super
-        else if (columnSpecCnt == 2)
-        {
-            superColumnName = getColumnName(columnFamily, columnFamilySpec.getChild(2)).array();
-            columnName = getSubColumnName(columnFamily, columnFamilySpec.getChild(3));
-        }
-        // The parser groks an arbitrary number of these so it is possible to get here.
-        else
-        {
-            sessionState.out.println("Invalid row, super column, or column specification.");
-            return;
-        }
-
-        ColumnParent parent = new ColumnParent(columnFamily);
-        if(superColumnName != null)
-            parent.setSuper_column(superColumnName);
-
-        long value = 1L;
-
-        // children count = 3 mean that we have by in arguments
-        if (statement.getChildCount() == 2)
-        {
-            String byValue = statement.getChild(1).getText();
-
-            try
-            {
-                value = Long.parseLong(byValue);
-            }
-            catch (NumberFormatException e)
-            {
-                sessionState.err.println(String.format("'%s' is an invalid value, should be an integer.", byValue));
-                return;
-            }
-            catch (Exception e)
-            {
-                throw new RuntimeException(e);
-            }
-        }
-
-        CounterColumn columnToInsert = new CounterColumn(columnName, multiplier * value);
-
-        // do the insert
-        thriftClient.add(key, parent, columnToInsert, consistencyLevel);
-        sessionState.out.printf("Value %s%n", multiplier < 0 ? "decremented." : "incremented.");
-    }
-
-    private void executeShowClusterName() throws TException
-    {
-        if (!CliMain.isConnected())
-            return;
-
-        sessionState.out.println(thriftClient.describe_cluster_name());
-    }
-
-    /**
-     * Add a keyspace
-     * @param statement - a token tree representing current statement
-     */
-    private void executeAddKeySpace(Tree statement)
-    {
-
-        if (!CliMain.isConnected())
-            return;
-
-        // first value is the keyspace name, after that it is all key=value
-        String keyspaceName = CliUtils.unescapeSQLString(statement.getChild(0).getText());
-        KsDef ksDef = new KsDef(keyspaceName, DEFAULT_PLACEMENT_STRATEGY, new LinkedList<CfDef>());
-
-        try
-        {
-            String mySchemaVersion = thriftClient.system_add_keyspace(updateKsDefAttributes(statement, ksDef));
-            sessionState.out.println(mySchemaVersion);
-
-            keyspacesMap.put(keyspaceName, thriftClient.describe_keyspace(keyspaceName));
-        }
-        catch (InvalidRequestException e)
-        {
-            throw new RuntimeException(e);
-        }
-        catch (Exception e)
-        {
-            throw new RuntimeException(e);
-        }
-    }
-
-
-    /**
-     * Add a column family
-     * @param statement - a token tree representing current statement
-     */
-    private void executeAddColumnFamily(Tree statement)
-    {
-        if (!CliMain.isConnected() || !hasKeySpace())
-            return;
-
-        // first value is the column family name, after that it is all key=value
-        CfDef cfDef = new CfDef(keySpace, CliUtils.unescapeSQLString(statement.getChild(0).getText()));
-
-        try
-        {
-            String mySchemaVersion = thriftClient.system_add_column_family(updateCfDefAttributes(statement, cfDef));
-            sessionState.out.println(mySchemaVersion);
-            keyspacesMap.put(keySpace, thriftClient.describe_keyspace(keySpace));
-        }
-        catch (InvalidRequestException e)
-        {
-            throw new RuntimeException(e);
-        }
-        catch (Exception e)
-        {
-            throw new RuntimeException(e);
-        }
-    }
-
-    /**
-     * Update existing keyspace identified by name
-     * @param statement - tree represeting statement
-     */
-    private void executeUpdateKeySpace(Tree statement)
-    {
-        if (!CliMain.isConnected())
-            return;
-
-        try
-        {
-            String keyspaceName = CliCompiler.getKeySpace(statement, thriftClient.describe_keyspaces());
-
-            KsDef currentKsDef = getKSMetaData(keyspaceName);
-            KsDef updatedKsDef = updateKsDefAttributes(statement, currentKsDef);
-
-            String mySchemaVersion = thriftClient.system_update_keyspace(updatedKsDef);
-            sessionState.out.println(mySchemaVersion);
-            keyspacesMap.remove(keyspaceName);
-            getKSMetaData(keyspaceName);
-        }
-        catch (InvalidRequestException e)
-        {
-            throw new RuntimeException(e);
-        }
-        catch (Exception e)
-        {
-            throw new RuntimeException(e);
-        }
-    }
-
-    /**
-     * Update existing column family identified by name
-     * @param statement - tree represeting statement
-     */
-    private void executeUpdateColumnFamily(Tree statement)
-    {
-        if (!CliMain.isConnected() || !hasKeySpace())
-            return;
-
-        String cfName = CliCompiler.getColumnFamily(statement, currentCfDefs());
-
-        try
-        {
-            // request correct cfDef from the server (we let that call include CQL3 cf even though
-            // they can't be modified by thrift because the error message that will be thrown by
-            // system_update_column_family will be more useful)
-            CfDef cfDef = getCfDef(thriftClient.describe_keyspace(this.keySpace), cfName, true);
-
-            if (cfDef == null)
-                throw new RuntimeException("Column Family " + cfName + " was not found in the current keyspace.");
-
-            String mySchemaVersion = thriftClient.system_update_column_family(updateCfDefAttributes(statement, cfDef));
-            sessionState.out.println(mySchemaVersion);
-            keyspacesMap.put(keySpace, thriftClient.describe_keyspace(keySpace));
-        }
-        catch (InvalidRequestException e)
-        {
-            throw new RuntimeException(e);
-        }
-        catch (Exception e)
-        {
-            throw new RuntimeException(e);
-        }
-    }
-
-    /**
-     * Used to update keyspace definition attributes
-     * @param statement - ANTRL tree representing current statement
-     * @param ksDefToUpdate - keyspace definition to update
-     * @return ksDef - updated keyspace definition
-     */
-    private KsDef updateKsDefAttributes(Tree statement, KsDef ksDefToUpdate)
-    {
-        KsDef ksDef = new KsDef(ksDefToUpdate);
-
-        // removing all column definitions - thrift system_update_keyspace method requires that
-        ksDef.setCf_defs(new LinkedList<CfDef>());
-
-        for(int i = 1; i < statement.getChildCount(); i += 2)
-        {
-            String currentStatement = statement.getChild(i).getText().toUpperCase();
-            AddKeyspaceArgument mArgument = AddKeyspaceArgument.valueOf(currentStatement);
-            String mValue = statement.getChild(i + 1).getText();
-
-            switch(mArgument)
-            {
-            case PLACEMENT_STRATEGY:
-                ksDef.setStrategy_class(CliUtils.unescapeSQLString(mValue));
-                break;
-            case STRATEGY_OPTIONS:
-                ksDef.setStrategy_options(getStrategyOptionsFromTree(statement.getChild(i + 1)));
-                break;
-            case DURABLE_WRITES:
-                ksDef.setDurable_writes(Boolean.parseBoolean(mValue));
-                break;
-            default:
-                //must match one of the above or we'd throw an exception at the valueOf statement above.
-                assert(false);
-            }
-        }
-
-        // using default snitch options if strategy is NetworkTopologyStrategy and no options were set.
-        if (ksDef.getStrategy_class().contains(".NetworkTopologyStrategy"))
-        {
-            Map<String, String> currentStrategyOptions = ksDef.getStrategy_options();
-
-            // adding default data center from SimpleSnitch
-            if (currentStrategyOptions == null || currentStrategyOptions.isEmpty())
-            {
-                SimpleSnitch snitch = new SimpleSnitch();
-                Map<String, String> options = new HashMap<String, String>();
-                try
-                {
-                    options.put(snitch.getDatacenter(InetAddress.getLocalHost()), "1");
-                }
-                catch (UnknownHostException e)
-                {
-                    throw new RuntimeException(e);
-                }
-                ksDef.setStrategy_options(options);
-            }
-        }
-
-        return ksDef;
-    }
-
-    /**
-     * Update column family definition attributes
-     * @param statement - ANTLR tree representing current statement
-     * @param cfDefToUpdate - column family definition to apply updates on
-     * @return cfDef - updated column family definition
-     */
-    private CfDef updateCfDefAttributes(Tree statement, CfDef cfDefToUpdate)
-    {
-        CfDef cfDef = new CfDef(cfDefToUpdate);
-
-        for (int i = 1; i < statement.getChildCount(); i += 2)
-        {
-            String currentArgument = statement.getChild(i).getText().toUpperCase();
-            ColumnFamilyArgument mArgument = ColumnFamilyArgument.valueOf(currentArgument);
-            String mValue = statement.getChild(i + 1).getText();
-
-            switch(mArgument)
-            {
-            case COLUMN_TYPE:
-                cfDef.setColumn_type(CliUtils.unescapeSQLString(mValue));
-                break;
-            case COMPARATOR:
-                cfDef.setComparator_type(CliUtils.unescapeSQLString(mValue));
-                break;
-            case SUBCOMPARATOR:
-                cfDef.setSubcomparator_type(CliUtils.unescapeSQLString(mValue));
-                break;
-            case COMMENT:
-                cfDef.setComment(CliUtils.unescapeSQLString(mValue));
-                break;
-            case READ_REPAIR_CHANCE:
-                double chance = Double.parseDouble(mValue);
-
-                if (chance < 0 || chance > 1)
-                    throw new RuntimeException("Error: read_repair_chance must be between 0 and 1.");
-
-                cfDef.setRead_repair_chance(chance);
-                break;
-            case DCLOCAL_READ_REPAIR_CHANCE:
-                double localChance = Double.parseDouble(mValue);
-
-                if (localChance < 0 || localChance > 1)
-                    throw new RuntimeException("Error: dclocal_read_repair_chance must be between 0 and 1.");
-
-                cfDef.setDclocal_read_repair_chance(localChance);
-                break;
-            case GC_GRACE:
-                cfDef.setGc_grace_seconds(Integer.parseInt(mValue));
-                break;
-            case COLUMN_METADATA:
-                Tree arrayOfMetaAttributes = statement.getChild(i + 1);
-                if (!arrayOfMetaAttributes.getText().equals("ARRAY"))
-                    throw new RuntimeException("'column_metadata' format - [{ k:v, k:v, ..}, { ... }, ...]");
-                cfDef.setColumn_metadata(getCFColumnMetaFromTree(cfDef, arrayOfMetaAttributes));
-                break;
-            case MEMTABLE_OPERATIONS:
-                break;
-            case MEMTABLE_THROUGHPUT:
-                break;
-            case DEFAULT_VALIDATION_CLASS:
-                cfDef.setDefault_validation_class(CliUtils.unescapeSQLString(mValue));
-                break;
-            case MIN_COMPACTION_THRESHOLD:
-                int threshold = Integer.parseInt(mValue);
-                if (threshold <= 0)
-                    throw new RuntimeException("Disabling compaction by setting min/max compaction thresholds to 0 has been deprecated, set compaction_strategy_options={'enabled':false} instead");
-                cfDef.setMin_compaction_threshold(threshold);
-                break;
-            case MAX_COMPACTION_THRESHOLD:
-                threshold = Integer.parseInt(mValue);
-                if (threshold <= 0)
-                    throw new RuntimeException("Disabling compaction by setting min/max compaction thresholds to 0 has been deprecated, set compaction_strategy_options={'enabled':false} instead");
-                cfDef.setMax_compaction_threshold(Integer.parseInt(mValue));
-                break;
-            case REPLICATE_ON_WRITE:
-                cfDef.setReplicate_on_write(Boolean.parseBoolean(mValue));
-                break;
-            case KEY_VALIDATION_CLASS:
-                cfDef.setKey_validation_class(CliUtils.unescapeSQLString(mValue));
-                break;
-            case COMPACTION_STRATEGY:
-                cfDef.setCompaction_strategy(CliUtils.unescapeSQLString(mValue));
-                break;
-            case COMPACTION_STRATEGY_OPTIONS:
-                cfDef.setCompaction_strategy_options(getStrategyOptionsFromTree(statement.getChild(i+1)));
-                break;
-            case COMPRESSION_OPTIONS:
-                cfDef.setCompression_options(getStrategyOptionsFromTree(statement.getChild(i+1)));
-                break;
-            case BLOOM_FILTER_FP_CHANCE:
-                cfDef.setBloom_filter_fp_chance(Double.parseDouble(mValue));
-                break;
-            case MEMTABLE_FLUSH_PERIOD_IN_MS:
-                cfDef.setMemtable_flush_period_in_ms(Integer.parseInt(mValue));
-                break;
-            case CACHING:
-                cfDef.setCaching(CliUtils.unescapeSQLString(mValue));
-                break;
-            case CELLS_PER_ROW_TO_CACHE:
-                cfDef.setCells_per_row_to_cache(CliUtils.unescapeSQLString(mValue));
-                break;
-            case DEFAULT_TIME_TO_LIVE:
-                cfDef.setDefault_time_to_live(Integer.parseInt(mValue));
-                break;
-            case INDEX_INTERVAL:
-                cfDef.setIndex_interval(Integer.parseInt(mValue));
-                break;
-            case SPECULATIVE_RETRY:
-                cfDef.setSpeculative_retry(CliUtils.unescapeSQLString(mValue));
-                break;
-            case POPULATE_IO_CACHE_ON_FLUSH:
-                cfDef.setPopulate_io_cache_on_flush(Boolean.parseBoolean(mValue));
-                break;
-            default:
-                //must match one of the above or we'd throw an exception at the valueOf statement above.
-                assert(false);
-
-            }
-        }
-
-        return cfDef;
-    }
-
-    /**
-     * Delete a keyspace
-     * @param statement - a token tree representing current statement
-     * @throws TException - exception
-     * @throws InvalidRequestException - exception
-     * @throws NotFoundException - exception
-     * @throws SchemaDisagreementException
-     */
-    private void executeDelKeySpace(Tree statement)
-            throws TException, InvalidRequestException, NotFoundException, SchemaDisagreementException
-    {
-        if (!CliMain.isConnected())
-            return;
-
-        String keyspaceName = CliCompiler.getKeySpace(statement, thriftClient.describe_keyspaces());
-        String version = thriftClient.system_drop_keyspace(keyspaceName);
-        sessionState.out.println(version);
-
-        if (keyspaceName.equals(keySpace)) //we just deleted the keyspace we were authenticated too
-            keySpace = null;
-    }
-
-    /**
-     * Delete a column family
-     * @param statement - a token tree representing current statement
-     * @throws TException - exception
-     * @throws InvalidRequestException - exception
-     * @throws NotFoundException - exception
-     * @throws SchemaDisagreementException
-     */
-    private void executeDelColumnFamily(Tree statement)
-            throws TException, InvalidRequestException, NotFoundException, SchemaDisagreementException
-    {
-        if (!CliMain.isConnected() || !hasKeySpace())
-            return;
-
-        String cfName = CliCompiler.getColumnFamily(statement, currentCfDefs());
-        String mySchemaVersion = thriftClient.system_drop_column_family(cfName);
-        sessionState.out.println(mySchemaVersion);
-    }
-
-    private void executeList(Tree statement)
-            throws TException, InvalidRequestException, NotFoundException, IllegalAccessException, InstantiationException, NoSuchFieldException, UnavailableException, TimedOutException, CharacterCodingException
-    {
-        if (!CliMain.isConnected() || !hasKeySpace())
-            return;
-
-        long startTime = System.nanoTime();
-
-        // extract column family
-        String columnFamily = CliCompiler.getColumnFamily(statement, currentCfDefs());
-
-        String rawStartKey = "";
-        String rawEndKey = "";
-        int limitCount = Integer.MAX_VALUE; // will reset to default later if it's not specified
-        int columnCount = Integer.MAX_VALUE; // will reset to default later if it's not specified
-        boolean reversed = false;
-
-        // optional arguments: key range and limit
-        for (int i = 1; i < statement.getChildCount(); i++)
-        {
-            Tree child = statement.getChild(i);
-            if (child.getType() == CliParser.NODE_KEY_RANGE)
-            {
-                if (child.getChildCount() > 0)
-                {
-                    rawStartKey = CliUtils.unescapeSQLString(child.getChild(0).getText());
-                    if (child.getChildCount() > 1)
-                        rawEndKey = CliUtils.unescapeSQLString(child.getChild(1).getText());
-                }
-            }
-            else if (child.getType() == CliParser.NODE_LIMIT)
-            {
-                if (child.getChildCount() != 1)
-                {
-                    sessionState.out.println("Invalid limit clause");
-                    return;
-                }
-
-                limitCount = Integer.parseInt(child.getChild(0).getText());
-                if (limitCount <= 0)
-                {
-                    sessionState.out.println("Invalid limit " + limitCount);
-                    return;
-                }
-            }
-            else if (child.getType() == CliParser.NODE_COLUMNS)
-            {
-                if ((child.getChildCount() < 1) || (child.getChildCount() > 2))
-                {
-                    sessionState.err.println("Invalid cells clause.");
-                    return;
-                }
-
-                String columns = child.getChild(0).getText();
-
-                try
-                {
-                    columnCount = Integer.parseInt(columns);
-                    if (columnCount < 0)
-                    {
-                        sessionState.err.println("Invalid cell limit: " + columnCount);
-                        return;
-                    }
-
-                    if (child.getChildCount() == 2)
-                        reversed = child.getChild(1).getType() == CliParser.NODE_REVERSED;
-                }
-                catch (NumberFormatException nfe)
-                {
-                    sessionState.err.println("Invalid cell number format: " + columns);
-                    return;
-                }
-            }
-        }
-
-        if (limitCount == Integer.MAX_VALUE)
-        {
-            limitCount = 100;
-            sessionState.out.println("Using default limit of 100");
-        }
-        if (columnCount == Integer.MAX_VALUE)
-        {
-            columnCount = 100;
-            sessionState.out.println("Using default cell limit of 100");
-        }
-
-
-        CfDef columnFamilyDef = getCfDef(columnFamily);
-
-        // read all columns and superColumns
-        SlicePredicate predicate = new SlicePredicate();
-        SliceRange sliceRange = new SliceRange();
-        sliceRange.setStart(new byte[0]).setFinish(new byte[0]);
-        sliceRange.setCount(columnCount);
-        sliceRange.setReversed(reversed);
-        predicate.setSlice_range(sliceRange);
-
-        // set the key range
-        KeyRange range = new KeyRange(limitCount);
-        AbstractType<?> keyComparator = this.cfKeysComparators.get(columnFamily);
-        ByteBuffer startKey = rawStartKey.isEmpty() ? ByteBufferUtil.EMPTY_BYTE_BUFFER : getBytesAccordingToType(rawStartKey, keyComparator);
-        ByteBuffer endKey = rawEndKey.isEmpty() ? ByteBufferUtil.EMPTY_BYTE_BUFFER : getBytesAccordingToType(rawEndKey, keyComparator);
-        range.setStart_key(startKey).setEnd_key(endKey);
-
-        ColumnParent columnParent = new ColumnParent(columnFamily);
-        List<KeySlice> keySlices = thriftClient.get_range_slices(columnParent, predicate, range, consistencyLevel);
-        printSliceList(columnFamilyDef, keySlices);
-        elapsedTime(startTime);
-    }
-
-    // DROP INDEX ON <CF>.<COLUMN>
-    private void executeDropIndex(Tree statement) throws TException, SchemaDisagreementException, InvalidRequestException, NotFoundException
-    {
-        if (!CliMain.isConnected() || !hasKeySpace())
-            return;
-
-        // getColumnFamily will check if CF exists for us
-        String columnFamily = CliCompiler.getColumnFamily(statement, currentCfDefs());
-        String rawColumName = CliUtils.unescapeSQLString(statement.getChild(1).getText());
-
-        CfDef cfDef = getCfDef(columnFamily);
-
-        ByteBuffer columnName = columnNameAsBytes(rawColumName, cfDef);
-
-        boolean foundColumn = false;
-
-        for (ColumnDef column : cfDef.getColumn_metadata())
-        {
-            if (column.name.equals(columnName))
-            {
-                foundColumn = true;
-
-                if (column.getIndex_type() == null)
-                    throw new RuntimeException(String.format("Column '%s' does not have an index.", rawColumName));
-
-                column.setIndex_name(null);
-                column.setIndex_type(null);
-            }
-        }
-
-        if (!foundColumn)
-            throw new RuntimeException(String.format("Column '%s' definition was not found in ColumnFamily '%s'.",
-                                                     rawColumName,
-                                                     columnFamily));
-
-        String mySchemaVersion = thriftClient.system_update_column_family(cfDef);
-        sessionState.out.println(mySchemaVersion);
-        keyspacesMap.put(keySpace, thriftClient.describe_keyspace(keySpace));
-    }
-
-    // TRUNCATE <columnFamily>
-    private void executeTruncate(String columnFamily) throws TException, InvalidRequestException, UnavailableException, TimedOutException
-    {
-        if (!CliMain.isConnected() || !hasKeySpace())
-            return;
-
-        // getting CfDef, it will fail if there is no such column family in current keySpace.
-        CfDef cfDef = getCfDef(CliCompiler.getColumnFamily(columnFamily, currentCfDefs()));
-
-        thriftClient.truncate(cfDef.getName());
-        sessionState.out.println(columnFamily + " truncated.");
-    }
-
-    /**
-     * Command: CONSISTENCYLEVEL AS (ONE | QUORUM ...)
-     * Tree: ^(NODE_CONSISTENCY_LEVEL AS (ONE | QUORUM ...))
-     * @param statement - tree representing current statement
-     */
-    private void executeConsistencyLevelStatement(Tree statement)
-    {
-        if (!CliMain.isConnected())
-            return;
-
-        String userSuppliedLevel = statement.getChild(0).getText().toUpperCase();
-
-        try
-        {
-            consistencyLevel = ConsistencyLevel.valueOf(userSuppliedLevel);
-        }
-        catch (IllegalArgumentException e)
-        {
-            String elements = "ONE, TWO, THREE, QUORUM, ALL, LOCAL_QUORUM, EACH_QUORUM, ANY";
-            sessionState.out.println(String.format("'%s' is invalid. Available: %s", userSuppliedLevel, elements));
-            return;
-        }
-
-        sessionState.out.println(String.format("Consistency level is set to '%s'.", consistencyLevel));
-    }
-
-    /**
-     * Command: ASSUME <columnFamily> (VALIDATOR | COMPARATOR | KEYS | SUB_COMPARATOR) AS <type>
-     * Tree: ^(NODE_ASSUME <columnFamily> (VALIDATOR | COMPARATOR | KEYS | SUB_COMPARATOR) <type>))
-     * @param statement - tree representing current statement
-     */
-    private void executeAssumeStatement(Tree statement)
-    {
-        if (!CliMain.isConnected() || !hasKeySpace())
-            return;
-
-        String cfName = CliCompiler.getColumnFamily(statement, currentCfDefs());
-
-        // VALIDATOR | COMPARATOR | KEYS | SUB_COMPARATOR
-        String assumptionElement = statement.getChild(1).getText().toUpperCase();
-
-
-        // Could be UTF8Type, IntegerType, LexicalUUIDType etc.
-        String defaultType = CliUtils.unescapeSQLString(statement.getChild(2).getText());
-
-        if (applyAssumption(cfName, assumptionElement, defaultType))
-        {
-            assumptions.addAssumption(keySpace, cfName, assumptionElement, defaultType);
-            sessionState.out.println(String.format("Assumption for column family '%s' added successfully.", cfName));
-        }
-    }
-
-    private boolean applyAssumption(String cfName, String assumptionElement, String defaultType)
-    {
-        CfDef columnFamily;
-
-        try
-        {
-            columnFamily = getCfDef(cfName);
-        }
-        catch (RuntimeException e)
-        {
-            return false; // just fail if CF does not exist
-        }
-
-        // used to store in this.cfKeysComparator
-        AbstractType<?> comparator;
-
-        try
-        {
-            comparator = TypeParser.parse(defaultType);
-        }
-        catch (RequestValidationException e)
-        {
-            try
-            {
-                comparator = Function.valueOf(defaultType.toUpperCase()).getValidator();
-            }
-            catch (Exception ne)
-            {
-                String functions = Function.getFunctionNames();
-                sessionState.out.println("Type '" + defaultType + "' was not found. Available: " + functions
-                                         + " Or any class which extends o.a.c.db.marshal.AbstractType.");
-                return false;
-            }
-        }
-
-        // making string representation look property e.g. o.a.c.db.marshal.UTF8Type
-        defaultType = comparator.getClass().getName();
-
-        if (assumptionElement.equals("COMPARATOR"))
-        {
-            columnFamily.setComparator_type(defaultType);
-        }
-        else if (assumptionElement.equals("SUB_COMPARATOR"))
-        {
-            columnFamily.setSubcomparator_type(defaultType);
-        }
-        else if (assumptionElement.equals("VALIDATOR"))
-        {
-            columnFamily.setDefault_validation_class(defaultType);
-        }
-        else if (assumptionElement.equals("KEYS"))
-        {
-            this.cfKeysComparators.put(columnFamily.getName(), comparator);
-        }
-        else
-        {
-            String elements = "VALIDATOR, COMPARATOR, KEYS, SUB_COMPARATOR.";
-            sessionState.out.println(String.format("'%s' is invalid. Available: %s", assumptionElement, elements));
-            return false;
-        }
-
-        return true;
-    }
-
-    // SHOW API VERSION
-    private void executeShowVersion() throws TException
-    {
-        if (!CliMain.isConnected())
-            return;
-
-        sessionState.out.println(thriftClient.describe_version());
-    }
-
-    // SHOW KEYSPACES
-    private void executeShowKeySpaces() throws TException, InvalidRequestException
-    {
-        if (!CliMain.isConnected())
-            return;
-
-        printCQL3TablesWarning("show keyspaces");
-
-        List<KsDef> keySpaces = thriftClient.describe_keyspaces();
-
-        Collections.sort(keySpaces, new KsDefNamesComparator());
-        for (KsDef keySpace : keySpaces)
-        {
-            describeKeySpace(keySpace.name, keySpace);
-        }
-    }
-
-    // SHOW SCHEMA
-    private void executeShowSchema(Tree statement) throws TException, InvalidRequestException
-    {
-        if (!CliMain.isConnected())
-            return;
-
-        printCQL3TablesWarning("show schema");
-
-        final List<KsDef> keyspaces = thriftClient.describe_keyspaces();
-        Collections.sort(keyspaces, new KsDefNamesComparator());
-        final String keyspaceName = (statement.getChildCount() == 0)
-                                ? keySpace
-                                : CliCompiler.getKeySpace(statement, keyspaces);
-
-        Iterator<KsDef> ksIter;
-        if (keyspaceName != null)
-            ksIter = Collections2.filter(keyspaces, new Predicate<KsDef>()
-            {
-                public boolean apply(KsDef ksDef)
-                {
-                    return keyspaceName.equals(ksDef.name);
-                }
-            }).iterator();
-        else
-            ksIter = keyspaces.iterator();
-
-        while (ksIter.hasNext())
-            showKeyspace(sessionState.out, ksIter.next());
-
-        sessionState.out.flush();
-    }
-
-    /**
-     * Creates a CLI script to create the Keyspace it's Column Families
-     *
-     * @param output StringBuilder to write to.
-     * @param ksDef KsDef to create the cli script for.
-     */
-    private void showKeyspace(PrintStream output, KsDef ksDef)
-    {
-        output.append("create keyspace ").append(CliUtils.maybeEscapeName(ksDef.name));
-
-        writeAttr(output, true, "placement_strategy", normaliseType(ksDef.strategy_class, "org.apache.cassandra.locator"));
-
-        if (ksDef.strategy_options != null && !ksDef.strategy_options.isEmpty())
-        {
-            final StringBuilder opts = new StringBuilder();
-            opts.append("{");
-            String prefix = "";
-            for (Map.Entry<String, String> opt : ksDef.strategy_options.entrySet())
-            {
-                opts.append(prefix).append(CliUtils.escapeSQLString(opt.getKey())).append(" : ").append(CliUtils.escapeSQLString(opt.getValue()));
-                prefix = ", ";
-            }
-            opts.append("}");
-            writeAttrRaw(output, false, "strategy_options", opts.toString());
-        }
-
-        writeAttr(output, false, "durable_writes", ksDef.durable_writes);
-
-        output.append(";").append(NEWLINE);
-        output.append(NEWLINE);
-
-        output.append("use ").append(CliUtils.maybeEscapeName(ksDef.name)).append(";");
-        output.append(NEWLINE);
-        output.append(NEWLINE);
-
-        Collections.sort(ksDef.cf_defs, new CfDefNamesComparator());
-        for (CfDef cfDef : ksDef.cf_defs)
-            showColumnFamily(output, cfDef);
-        output.append(NEWLINE);
-        output.append(NEWLINE);
-    }
-
-    /**
-     * Creates a CLI script for the CfDef including meta data to the supplied StringBuilder.
-     *
-     * @param output File to write to.
-     * @param cfDef  CfDef to export attributes from.
-     */
-    private void showColumnFamily(PrintStream output, CfDef cfDef)
-    {
-        output.append("create column family ").append(CliUtils.maybeEscapeName(cfDef.name));
-
-        writeAttr(output, true, "column_type", cfDef.column_type);
-        writeAttr(output, false, "comparator", normaliseType(cfDef.comparator_type, "org.apache.cassandra.db.marshal"));
-        if (cfDef.column_type.equals("Super"))
-            writeAttr(output, false, "subcomparator", normaliseType(cfDef.subcomparator_type, "org.apache.cassandra.db.marshal"));
-        if (!StringUtils.isEmpty(cfDef.default_validation_class))
-            writeAttr(output, false, "default_validation_class",
-                        normaliseType(cfDef.default_validation_class, "org.apache.cassandra.db.marshal"));
-        writeAttr(output, false, "key_validation_class",
-                    normaliseType(cfDef.key_validation_class, "org.apache.cassandra.db.marshal"));
-
-        writeAttr(output, false, "read_repair_chance", cfDef.read_repair_chance);
-        writeAttr(output, false, "dclocal_read_repair_chance", cfDef.dclocal_read_repair_chance);
-        writeAttr(output, false, "gc_grace", cfDef.gc_grace_seconds);
-        writeAttr(output, false, "min_compaction_threshold", cfDef.min_compaction_threshold);
-        writeAttr(output, false, "max_compaction_threshold", cfDef.max_compaction_threshold);
-        writeAttr(output, false, "compaction_strategy", cfDef.compaction_strategy);
-        writeAttr(output, false, "caching", cfDef.caching);
-        writeAttr(output, false, "cells_per_row_to_cache", cfDef.cells_per_row_to_cache);
-        writeAttr(output, false, "default_time_to_live", cfDef.default_time_to_live);
-        writeAttr(output, false, "speculative_retry", cfDef.speculative_retry);
-
-        if (cfDef.isSetBloom_filter_fp_chance())
-            writeAttr(output, false, "bloom_filter_fp_chance", cfDef.bloom_filter_fp_chance);
-        if (!cfDef.compaction_strategy_options.isEmpty())
-        {
-            StringBuilder cOptions = new StringBuilder();
-
-            cOptions.append("{");
-
-            Map<String, String> options = cfDef.compaction_strategy_options;
-
-            int i = 0, size = options.size();
-
-            for (Map.Entry<String, String> entry : options.entrySet())
-            {
-                cOptions.append(CliUtils.quote(entry.getKey())).append(" : ").append(CliUtils.quote(entry.getValue()));
-
-                if (i != size - 1)
-                    cOptions.append(", ");
-
-                i++;
-            }
-
-            cOptions.append("}");
-
-            writeAttrRaw(output, false, "compaction_strategy_options", cOptions.toString());
-        }
-        if (!StringUtils.isEmpty(cfDef.comment))
-            writeAttr(output, false, "comment", cfDef.comment);
-
-        if (!cfDef.column_metadata.isEmpty())
-        {
-            output.append(NEWLINE)
-                  .append(TAB)
-                  .append("and column_metadata = [");
-
-            boolean first = true;
-            for (ColumnDef colDef : cfDef.column_metadata)
-            {
-                if (!first)
-                    output.append(",");
-                first = false;
-                showColumnMeta(output, cfDef, colDef);
-            }
-
-            output.append("]");
-        }
-
-        if (cfDef.compression_options != null && !cfDef.compression_options.isEmpty())
-        {
-            StringBuilder compOptions = new StringBuilder();
-
-            compOptions.append("{");
-
-            int i = 0, size = cfDef.compression_options.size();
-
-            for (Map.Entry<String, String> entry : cfDef.compression_options.entrySet())
-            {
-                compOptions.append(CliUtils.quote(entry.getKey())).append(" : ").append(CliUtils.quote(entry.getValue()));
-
-                if (i != size - 1)
-                    compOptions.append(", ");
-
-                i++;
-            }
-
-            compOptions.append("}");
-
-            writeAttrRaw(output, false, "compression_options", compOptions.toString());
-        }
-        if (cfDef.isSetIndex_interval())
-            writeAttr(output, false, "index_interval", cfDef.index_interval);
-
-        output.append(";");
-        output.append(NEWLINE);
-        output.append(NEWLINE);
-    }
-
-    /**
-     * Writes the supplied ColumnDef to the StringBuilder as a cli script.
-     *
-     * @param output The File to write to.
-     * @param cfDef  The CfDef as a source for comparator/validator
-     * @param colDef The Column Definition to export
-     */
-    private void showColumnMeta(PrintStream output, CfDef cfDef, ColumnDef colDef)
-    {
-        output.append(NEWLINE + TAB + TAB + "{");
-
-        final AbstractType<?> comparator = getFormatType(cfDef.column_type.equals("Super")
-                                                      ? cfDef.subcomparator_type
-                                                      : cfDef.comparator_type);
-        output.append("column_name : '" + CliUtils.escapeSQLString(comparator.getString(colDef.name)) + "'," + NEWLINE);
-        String validationClass = normaliseType(colDef.validation_class, "org.apache.cassandra.db.marshal");
-        output.append(TAB + TAB + "validation_class : " + CliUtils.escapeSQLString(validationClass));
-        if (colDef.isSetIndex_name())
-        {
-            output.append(",").append(NEWLINE)
-                  .append(TAB + TAB + "index_name : '" + CliUtils.escapeSQLString(colDef.index_name) + "'," + NEWLINE)
-                  .append(TAB + TAB + "index_type : " + CliUtils.escapeSQLString(Integer.toString(colDef.index_type.getValue())));
-
-            if (colDef.index_options != null && !colDef.index_options.isEmpty())
-            {
-                output.append(",").append(NEWLINE);
-                output.append(TAB + TAB + "index_options : {" + NEWLINE);
-                int numOpts = colDef.index_options.size();
-                for (Map.Entry<String, String> entry : colDef.index_options.entrySet())
-                {
-                    String option = CliUtils.escapeSQLString(entry.getKey());
-                    String optionValue = CliUtils.escapeSQLString(entry.getValue());
-
-                    output.append(TAB + TAB + TAB)
-                          .append("'" + option + "' : '")
-                          .append(optionValue)
-                          .append("'");
-
-                    if (--numOpts > 0)
-                        output.append(",").append(NEWLINE);
-                }
-                output.append("}");
-            }
-        }
-        output.append("}");
-    }
-
-    private String normaliseType(String path, String expectedPackage)
-    {
-        if (path.startsWith(expectedPackage))
-            return path.substring(expectedPackage.length() + 1);
-
-        return path;
-    }
-
-    private void writeAttr(PrintStream output, boolean first, String name, Boolean value)
-    {
-        writeAttrRaw(output, first, name, value.toString());
-    }
-    private void writeAttr(PrintStream output, boolean first, String name, Number value)
-    {
-        writeAttrRaw(output, first, name, value.toString());
-    }
-
-    private void writeAttr(PrintStream output, boolean first, String name, String value)
-    {
-        writeAttrRaw(output, first, name, "'" + CliUtils.escapeSQLString(value) + "'");
-    }
-
-    private void writeAttrRaw(PrintStream output, boolean first, String name, String value)
-    {
-        output.append(NEWLINE).append(TAB);
-        output.append(first ? "with " : "and ");
-        output.append(name).append(" = ");
-        output.append(value);
-    }
-    /**
-     * Returns true if this.keySpace is set, false otherwise
-     * @return boolean
-     */
-    private boolean hasKeySpace(boolean printError)
-    {
-        boolean hasKeyspace = keySpace != null;
-
-        if (!hasKeyspace && printError)
-            sessionState.err.println("Not authorized to a working keyspace.");
-
-        return hasKeyspace;
-    }
-
-    private boolean hasKeySpace()
-    {
-        return hasKeySpace(true);
-    }
-
-    public String getKeySpace()
-    {
-        return keySpace == null ? "unknown" : keySpace;
-    }
-
-    public void setKeySpace(String keySpace) throws NotFoundException, InvalidRequestException, TException
-    {
-        this.keySpace = keySpace;
-        // We do nothing with the return value, but it hits a cache and the tab-completer.
-        getKSMetaData(keySpace);
-    }
-
-    public String getUsername()
-    {
-        return username == null ? "default" : username;
-    }
-
-    public void setUsername(String username)
-    {
-        this.username = username;
-    }
-
-    // USE <keyspace_name>
-    private void executeUseKeySpace(Tree statement)
-    {
-        if (!CliMain.isConnected())
-            return;
-
-        int childCount = statement.getChildCount();
-        String keySpaceName, username = null, password = null;
-
-        // Get keyspace name
-        keySpaceName = CliUtils.unescapeSQLString(statement.getChild(0).getText());
-
-        if (childCount == 3)
-        {
-            username  = statement.getChild(1).getText();
-            password  = statement.getChild(2).getText();
-        }
-
-        if (keySpaceName == null)
-        {
-            sessionState.out.println("Keyspace argument required");
-            return;
-        }
-
-        try
-        {
-            AuthenticationRequest authRequest;
-
-            keySpaceName = CliCompiler.getKeySpace(keySpaceName, thriftClient.describe_keyspaces());
-
-            thriftClient.set_keyspace(keySpaceName);
-
-            if (username != null && password != null)
-            {
-                Map<String, String> credentials = new HashMap<String, String>();
-                /* remove quotes */
-                password = password.replace("\'", "");
-                credentials.put(IAuthenticator.USERNAME_KEY, username);
-                credentials.put(IAuthenticator.PASSWORD_KEY, password);
-                authRequest = new AuthenticationRequest(credentials);
-                thriftClient.login(authRequest);
-            }
-
-            keySpace = keySpaceName;
-            this.username = username != null ? username : "default";
-
-            keyspacesMap.remove(keySpace);
-            CliMain.updateCompletor(CliUtils.getCfNamesByKeySpace(getKSMetaData(keySpace)));
-            sessionState.out.println("Authenticated to keyspace: " + keySpace);
-        }
-        catch (AuthenticationException e)
-        {
-            sessionState.err.println("Exception during authentication to the cassandra node: " +
-                                     "verify keyspace exists, and you are using correct credentials.");
-        }
-        catch (AuthorizationException e)
-        {
-            sessionState.err.println("You are not authorized to use keyspace: " + keySpaceName);
-        }
-        catch (InvalidRequestException e)
-        {
-            sessionState.err.println(keySpaceName + " does not exist.");
-        }
-        catch (NotFoundException e)
-        {
-            sessionState.err.println(keySpaceName + " does not exist.");
-        }
-        catch (TException e)
-        {
-            if (sessionState.debug)
-                e.printStackTrace(sessionState.err);
-
-            sessionState.err.println("Login failure. Did you specify 'keyspace', 'username' and 'password'?");
-        }
-    }
-
-    private void executeTraceNextQuery() throws TException
-    {
-        if (!CliMain.isConnected())
-            return;
-
-        UUID sessionId = TimeUUIDType.instance.compose(thriftClient.trace_next_query());
-
-        sessionState.out.println("Will trace next query. Session ID: " + sessionId);
-    }
-
-    private void describeKeySpace(String keySpaceName, KsDef metadata) throws TException
-    {
-        NodeProbe probe = sessionState.getNodeProbe();
-
-        // getting compaction manager MBean to displaying index building information
-        CompactionManagerMBean compactionManagerMBean = (probe == null) ? null : probe.getCompactionManagerProxy();
-
-        // Describe and display
-        sessionState.out.println("Keyspace: " + keySpaceName + ":");
-        try
-        {
-            KsDef ks_def;
-            ks_def = metadata == null ? thriftClient.describe_keyspace(keySpaceName) : metadata;
-            sessionState.out.println("  Replication Strategy: " + ks_def.strategy_class);
-
-            sessionState.out.println("  Durable Writes: " + ks_def.durable_writes);
-
-            Map<String, String> options = ks_def.strategy_options;
-            sessionState.out.println("    Options: [" + ((options == null) ? "" : FBUtilities.toString(options)) + "]");
-
-            sessionState.out.println("  Column Families:");
-
-            Collections.sort(ks_def.cf_defs, new CfDefNamesComparator());
-
-            for (CfDef cf_def : ks_def.cf_defs)
-                describeColumnFamily(ks_def, cf_def, probe);
-
-            // compaction manager information
-            if (compactionManagerMBean != null)
-            {
-                for (Map<String, String> info : compactionManagerMBean.getCompactions())
-                {
-                    // if ongoing compaction type is index build
-                    if (info.get("taskType").equals(OperationType.INDEX_BUILD.toString()))
-                        continue;
-                    sessionState.out.printf("%nCurrently building index %s, completed %d of %d bytes.%n",
-                                            info.get("columnfamily"),
-                                            info.get("bytesComplete"),
-                                            info.get("totalBytes"));
-                }
-            }
-
-            // closing JMX connection
-            if (probe != null)
-                probe.close();
-        }
-        catch (InvalidRequestException e)
-        {
-            sessionState.out.println("Invalid request: " + e);
-        }
-        catch (NotFoundException e)
-        {
-            sessionState.out.println("Keyspace " + keySpaceName + " could not be found.");
-        }
-        catch (IOException e)
-        {
-            sessionState.out.println("Error while closing JMX connection: " + e.getMessage());
-        }
-    }
-
-    private void describeColumnFamily(KsDef ks_def, CfDef cf_def, NodeProbe probe)
-    {
-        // fetching bean for current column family store
-        ColumnFamilyStoreMBean cfMBean = (probe == null) ? null : probe.getCfsProxy(ks_def.getName(), cf_def.getName());
-
-        boolean isSuper = cf_def.column_type.equals("Super");
-        sessionState.out.printf("    ColumnFamily: %s%s%n", cf_def.name, isSuper ? " (Super)" : "");
-
-        if (cf_def.comment != null && !cf_def.comment.isEmpty())
-            sessionState.out.printf("    \"%s\"%n", cf_def.comment);
-
-        if (cf_def.key_validation_class != null)
-            sessionState.out.printf("      Key Validation Class: %s%n", cf_def.key_validation_class);
-
-        if (cf_def.default_validation_class != null)
-            sessionState.out.printf("      Default column value validator: %s%n", cf_def.default_validation_class);
-
-        sessionState.out.printf("      Cells sorted by: %s%s%n", cf_def.comparator_type, cf_def.column_type.equals("Super") ? "/" + cf_def.subcomparator_type : "");
-        sessionState.out.printf("      GC grace seconds: %s%n", cf_def.gc_grace_seconds);
-        sessionState.out.printf("      Compaction min/max thresholds: %s/%s%n", cf_def.min_compaction_threshold, cf_def.max_compaction_threshold);
-        sessionState.out.printf("      Read repair chance: %s%n", cf_def.read_repair_chance);
-        sessionState.out.printf("      DC Local Read repair chance: %s%n", cf_def.dclocal_read_repair_chance);
-        sessionState.out.printf("      Caching: %s%n", cf_def.caching);
-        sessionState.out.printf("      Default time to live: %s%n", cf_def.default_time_to_live);
-        sessionState.out.printf("      Bloom Filter FP chance: %s%n", cf_def.isSetBloom_filter_fp_chance() ? cf_def.bloom_filter_fp_chance : "default");
-        sessionState.out.printf("      Index interval: %s%n", cf_def.isSetIndex_interval() ? cf_def.index_interval : "default");
-        sessionState.out.printf("      Speculative Retry: %s%n", cf_def.speculative_retry);
-
-        // if we have connection to the cfMBean established
-        if (cfMBean != null)
-            sessionState.out.printf("      Built indexes: %s%n", cfMBean.getBuiltIndexes());
-
-        if (cf_def.getColumn_metadataSize() != 0)
-        {
-            String leftSpace = "      ";
-            String columnLeftSpace = leftSpace + "    ";
-
-            String compareWith = isSuper ? cf_def.subcomparator_type
-                    : cf_def.comparator_type;
-            AbstractType<?> columnNameValidator = getFormatType(compareWith);
-
-            sessionState.out.println(leftSpace + "Column Metadata:");
-            for (ColumnDef columnDef : cf_def.getColumn_metadata())
-            {
-                String columnName = columnNameValidator.getString(columnDef.name);
-                if (columnNameValidator instanceof BytesType)
-                {
-                    try
-                    {
-                        String columnString = UTF8Type.instance.getString(columnDef.name);
-                        columnName = columnString + " (" + columnName + ")";
-                    }
-                    catch (MarshalException e)
-                    {
-                        // guess it wasn't a utf8 column name after all
-                    }
-                }
-
-                sessionState.out.println(leftSpace + "  Column Name: " + columnName);
-                sessionState.out.println(columnLeftSpace + "Validation Class: " + columnDef.getValidation_class());
-
-                if 

<TRUNCATED>

[5/5] git commit: Remove cassandra-cli

Posted by al...@apache.org.
Remove cassandra-cli

patch by Aleksey Yeschenko; reviewed by Brandon Williams and Jason Brown
for CASSANDRA-7920


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

Branch: refs/heads/trunk
Commit: 380273826d05913d4b80b73e0db2ea49711b17c7
Parents: 24b18a9
Author: Aleksey Yeschenko <al...@apache.org>
Authored: Wed Sep 24 18:00:43 2014 -0700
Committer: Aleksey Yeschenko <al...@apache.org>
Committed: Wed Sep 24 18:33:29 2014 -0700

----------------------------------------------------------------------
 CHANGES.txt                                     |    1 +
 NEWS.txt                                        |    3 +-
 bin/cassandra-cli                               |   51 -
 bin/cassandra-cli.bat                           |   36 -
 build.xml                                       |   28 +-
 debian/cassandra.install                        |    1 -
 examples/hadoop_cql3_word_count/README.txt      |    2 +-
 examples/hadoop_word_count/README.txt           |    7 +-
 lib/jline-1.0.jar                               |  Bin 91183 -> 0 bytes
 lib/licenses/jline-1.0.txt                      |   33 -
 src/java/org/apache/cassandra/cli/Cli.g         |  728 ----
 .../org/apache/cassandra/cli/CliClient.java     | 3230 ------------------
 .../apache/cassandra/cli/CliCommandHelp.java    |   24 -
 .../org/apache/cassandra/cli/CliCompiler.java   |  172 -
 .../org/apache/cassandra/cli/CliCompleter.java  |   85 -
 src/java/org/apache/cassandra/cli/CliMain.java  |  405 ---
 .../org/apache/cassandra/cli/CliOptions.java    |  319 --
 .../apache/cassandra/cli/CliSessionState.java   |   94 -
 .../org/apache/cassandra/cli/CliUserHelp.java   |   29 -
 src/java/org/apache/cassandra/cli/CliUtils.java |  128 -
 .../cli/transport/FramedTransportFactory.java   |   32 -
 .../cli/transport/SSLTransportFactory.java      |   47 -
 .../org/apache/cassandra/db/DefsTables.java     |   62 -
 .../cassandra/thrift/SSLTransportFactory.java   |    7 +-
 .../org/apache/cassandra/cli/CliHelp.yaml       | 1262 -------
 .../cassandra/pig/CqlRecordReaderTest.java      |   20 +-
 .../cassandra/pig/CqlTableDataTypeTest.java     |   29 +-
 .../org/apache/cassandra/pig/CqlTableTest.java  |   44 +-
 .../org/apache/cassandra/pig/PigTestBase.java   |   74 +-
 .../pig/ThriftColumnFamilyDataTypeTest.java     |  259 +-
 .../cassandra/pig/ThriftColumnFamilyTest.java   |  467 ++-
 test/unit/org/apache/cassandra/cli/CliTest.java |  341 --
 32 files changed, 362 insertions(+), 7658 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/38027382/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index cf8f263..30a87a5 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 3.0
+ * Remove cassandra-cli (CASSANDRA-7920)
  * Optimize java source-based UDF invocation (CASSANDRA-7924)
  * Accept dollar quoted strings in CQL (CASSANDRA-7769)
  * Make assassinate a first class command (CASSANDRA-7935)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/38027382/NEWS.txt
----------------------------------------------------------------------
diff --git a/NEWS.txt b/NEWS.txt
index 0f47a69..96f2c05 100644
--- a/NEWS.txt
+++ b/NEWS.txt
@@ -21,10 +21,11 @@ New features
    - SSTable file name is changed. Now you don't have Keyspace/CF name
      in file name. Also, secondary index has its own directory under parent's
      directory.
-     
+
 
 Upgrading
 ---------
+   - cassandra-cli has been removed. Please use cqlsh instead.
    - YamlFileNetworkTopologySnitch has been removed; switch to
      GossipingPropertyFileSnitch instead.
    - CQL2 has been removed entirely in this release (previously deprecated

http://git-wip-us.apache.org/repos/asf/cassandra/blob/38027382/bin/cassandra-cli
----------------------------------------------------------------------
diff --git a/bin/cassandra-cli b/bin/cassandra-cli
deleted file mode 100755
index a2696da..0000000
--- a/bin/cassandra-cli
+++ /dev/null
@@ -1,51 +0,0 @@
-#!/bin/sh
-
-# 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.
-
-if [ "x$CASSANDRA_INCLUDE" = "x" ]; then
-    for include in "`dirname "$0"`/cassandra.in.sh" \
-                   "$HOME/.cassandra.in.sh" \
-                   /usr/share/cassandra/cassandra.in.sh \
-                   /usr/local/share/cassandra/cassandra.in.sh \
-                   /opt/cassandra/cassandra.in.sh; do
-        if [ -r "$include" ]; then
-            . "$include"
-            break
-        fi
-    done
-elif [ -r "$CASSANDRA_INCLUDE" ]; then
-    . "$CASSANDRA_INCLUDE"
-fi
-
-# Use JAVA_HOME if set, otherwise look for java in PATH
-if [ -x "$JAVA_HOME/bin/java" ]; then
-    JAVA="$JAVA_HOME/bin/java"
-else
-    JAVA="`which java`"
-fi
-
-if [ -z "$CLASSPATH" ]; then
-    echo "You must set the CLASSPATH var" >&2
-    exit 1
-fi
-
-"$JAVA" -ea -cp "$CLASSPATH" -Xmx256M \
-        -Dcassandra.storagedir="$cassandra_storagedir" \
-        -Dlogback.configurationFile=logback-tools.xml \
-        org.apache.cassandra.cli.CliMain "$@"
-
-# vi:ai sw=4 ts=4 tw=0 et

http://git-wip-us.apache.org/repos/asf/cassandra/blob/38027382/bin/cassandra-cli.bat
----------------------------------------------------------------------
diff --git a/bin/cassandra-cli.bat b/bin/cassandra-cli.bat
deleted file mode 100644
index 6211371..0000000
--- a/bin/cassandra-cli.bat
+++ /dev/null
@@ -1,36 +0,0 @@
-@REM
-@REM  Licensed to the Apache Software Foundation (ASF) under one or more
-@REM  contributor license agreements.  See the NOTICE file distributed with
-@REM  this work for additional information regarding copyright ownership.
-@REM  The ASF licenses this file to You under the Apache License, Version 2.0
-@REM  (the "License"); you may not use this file except in compliance with
-@REM  the License.  You may obtain a copy of the License at
-@REM
-@REM      http://www.apache.org/licenses/LICENSE-2.0
-@REM
-@REM  Unless required by applicable law or agreed to in writing, software
-@REM  distributed under the License is distributed on an "AS IS" BASIS,
-@REM  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-@REM  See the License for the specific language governing permissions and
-@REM  limitations under the License.
-
-@echo off
-if "%OS%" == "Windows_NT" setlocal
-
-pushd "%~dp0"
-call cassandra.in.bat
-
-if NOT DEFINED CASSANDRA_HOME set CASSANDRA_HOME=%~dp0..
-if NOT DEFINED JAVA_HOME goto :err
-
-echo Starting Cassandra Client
-"%JAVA_HOME%\bin\java" -cp %CASSANDRA_CLASSPATH% org.apache.cassandra.cli.CliMain %*
-goto finally
-
-:err
-echo The JAVA_HOME environment variable must be set to run this program!
-pause
-
-:finally
-
-ENDLOCAL

http://git-wip-us.apache.org/repos/asf/cassandra/blob/38027382/build.xml
----------------------------------------------------------------------
diff --git a/build.xml b/build.xml
index 8016a8b..e05b5c9 100644
--- a/build.xml
+++ b/build.xml
@@ -190,28 +190,6 @@
     </target>
 
     <!--
-       This generates the CLI grammar files from Cli.g
-    -->
-    <target name="check-gen-cli-grammar">
-        <uptodate property="cliUpToDate"
-                srcfile="${build.src.java}/org/apache/cassandra/cli/Cli.g"
-                targetfile="${build.src.gen-java}/org/apache/cassandra/cli/Cli.tokens"/>
-    </target>
-
-    <target name="gen-cli-grammar" depends="check-gen-cli-grammar" unless="cliUpToDate">
-      <echo>Building Grammar ${build.src.java}/org/apache/cassandra/cli/Cli.g  ....</echo>
-      <java classname="org.antlr.Tool"
-        classpath="${build.dir.lib}/jars/antlr-3.5.2.jar;${build.lib}/antlr-runtime-3.5.2.jar;${build.lib}/stringtemplate-4.0.2.jar"
-            fork="true"
-            failonerror="true">
-         <jvmarg value="-Xmx512M" />
-         <arg value="${build.src.java}/org/apache/cassandra/cli/Cli.g" />
-         <arg value="-fo" />
-         <arg value="${build.src.gen-java}/org/apache/cassandra/cli/" />
-      </java>
-    </target>
-
-    <!--
        This generates the CQL grammar files from Cql.g
     -->
     <target name="check-gen-cql3-grammar">
@@ -351,9 +329,6 @@
           <dependency groupId="ch.qos.logback" artifactId="logback-classic" version="1.1.2"/>
           <dependency groupId="org.codehaus.jackson" artifactId="jackson-core-asl" version="1.9.2"/>
           <dependency groupId="org.codehaus.jackson" artifactId="jackson-mapper-asl" version="1.9.2"/>
-          <dependency groupId="jline" artifactId="jline" version="1.0">
-            <exclusion groupId="junit" artifactId="junit"/>
-          </dependency>
           <dependency groupId="com.googlecode.json-simple" artifactId="json-simple" version="1.1"/>
           <dependency groupId="com.boundary" artifactId="high-scale-lib" version="1.0.6"/>
           <dependency groupId="com.github.jbellis" artifactId="jamm" version="0.2.6"/>
@@ -486,7 +461,6 @@
         <dependency groupId="org.slf4j" artifactId="slf4j-api"/>
         <dependency groupId="org.codehaus.jackson" artifactId="jackson-core-asl"/>
         <dependency groupId="org.codehaus.jackson" artifactId="jackson-mapper-asl"/>
-        <dependency groupId="jline" artifactId="jline"/>
         <dependency groupId="com.googlecode.json-simple" artifactId="json-simple"/>
         <dependency groupId="com.boundary" artifactId="high-scale-lib"/>
         <dependency groupId="org.yaml" artifactId="snakeyaml"/>
@@ -706,7 +680,7 @@
         depends="maven-ant-tasks-retrieve-build,build-project" description="Compile Cassandra classes"/>
     <target name="codecoverage" depends="jacoco-run,jacoco-report" description="Create code coverage report"/>
 
-    <target depends="init,gen-cli-grammar,gen-cql3-grammar"
+    <target depends="init,gen-cql3-grammar"
             name="build-project">
         <echo message="${ant.project.name}: ${ant.file}"/>
         <!-- Order matters! -->

http://git-wip-us.apache.org/repos/asf/cassandra/blob/38027382/debian/cassandra.install
----------------------------------------------------------------------
diff --git a/debian/cassandra.install b/debian/cassandra.install
index a4654d1..2fbbac1 100644
--- a/debian/cassandra.install
+++ b/debian/cassandra.install
@@ -11,7 +11,6 @@ debian/cassandra.in.sh usr/share/cassandra
 debian/cassandra.conf etc/security/limits.d
 debian/cassandra-sysctl.conf etc/sysctl.d
 bin/cassandra usr/sbin
-bin/cassandra-cli usr/bin
 bin/nodetool usr/bin
 bin/sstablekeys usr/bin
 bin/sstableloader usr/bin

http://git-wip-us.apache.org/repos/asf/cassandra/blob/38027382/examples/hadoop_cql3_word_count/README.txt
----------------------------------------------------------------------
diff --git a/examples/hadoop_cql3_word_count/README.txt b/examples/hadoop_cql3_word_count/README.txt
index e01632b..b69bdd5 100644
--- a/examples/hadoop_cql3_word_count/README.txt
+++ b/examples/hadoop_cql3_word_count/README.txt
@@ -19,7 +19,7 @@ contrib/word_count$ bin/word_count_setup
 contrib/word_count$ bin/word_count
 contrib/word_count$ bin/word_count_counters
 
-In order to view the results in Cassandra, one can use bin/cassandra-cli and
+In order to view the results in Cassandra, one can use bin/cqlsh and
 perform the following operations:
 $ bin/cqlsh localhost
 > use cql3_worldcount;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/38027382/examples/hadoop_word_count/README.txt
----------------------------------------------------------------------
diff --git a/examples/hadoop_word_count/README.txt b/examples/hadoop_word_count/README.txt
index 9a93908..ec6f512 100644
--- a/examples/hadoop_word_count/README.txt
+++ b/examples/hadoop_word_count/README.txt
@@ -19,12 +19,11 @@ contrib/word_count$ bin/word_count_setup
 contrib/word_count$ bin/word_count
 contrib/word_count$ bin/word_count_counters
 
-In order to view the results in Cassandra, one can use bin/cassandra-cli and
+In order to view the results in Cassandra, one can use bin/cqlsh and
 perform the following operations:
-$ bin/cassandra-cli
-> connect localhost/9160;
+$ bin/cqlsh localhost
 > use wordcount;
-> list output_words;
+> select * from output_words;
 
 The output of the word count can now be configured. In the bin/word_count
 file, you can specify the OUTPUT_REDUCER. The two options are 'filesystem'

http://git-wip-us.apache.org/repos/asf/cassandra/blob/38027382/lib/jline-1.0.jar
----------------------------------------------------------------------
diff --git a/lib/jline-1.0.jar b/lib/jline-1.0.jar
deleted file mode 100644
index 6c949b2..0000000
Binary files a/lib/jline-1.0.jar and /dev/null differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/38027382/lib/licenses/jline-1.0.txt
----------------------------------------------------------------------
diff --git a/lib/licenses/jline-1.0.txt b/lib/licenses/jline-1.0.txt
deleted file mode 100644
index 1cdc44c..0000000
--- a/lib/licenses/jline-1.0.txt
+++ /dev/null
@@ -1,33 +0,0 @@
-Copyright (c) 2002-2006, Marc Prud'hommeaux <mw...@cornell.edu>
-All rights reserved.
-
-Redistribution and use in source and binary forms, with or
-without modification, are permitted provided that the following
-conditions are met:
-
-Redistributions of source code must retain the above copyright
-notice, this list of conditions and the following disclaimer.
-
-Redistributions in binary form must reproduce the above copyright
-notice, this list of conditions and the following disclaimer
-in the documentation and/or other materials provided with
-the distribution.
-
-Neither the name of JLine nor the names of its contributors
-may be used to endorse or promote products derived from this
-software without specific prior written permission.
-
-THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
-"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING,
-BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY
-AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO
-EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE
-FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY,
-OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO,
-PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
-DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED
-AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT
-LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING
-IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED
-OF THE POSSIBILITY OF SUCH DAMAGE.
-

http://git-wip-us.apache.org/repos/asf/cassandra/blob/38027382/src/java/org/apache/cassandra/cli/Cli.g
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cli/Cli.g b/src/java/org/apache/cassandra/cli/Cli.g
deleted file mode 100644
index 07cdc1f..0000000
--- a/src/java/org/apache/cassandra/cli/Cli.g
+++ /dev/null
@@ -1,728 +0,0 @@
-/**
- * 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.
- */
-
-// ANTLR Grammar for the Cassandra Command Line Interface (CLI).
-
-grammar Cli;
-
-options {
-    output=AST;
-    ASTLabelType=CommonTree;
-    backtrack=true;
-}
-
-//
-// Nodes in the AST
-//
-tokens {
-    //
-    // Top-level nodes. These typically correspond to
-    // various top-level CLI statements.
-    //
-    NODE_CONNECT;
-    NODE_DESCRIBE;
-    NODE_DESCRIBE_CLUSTER;
-    NODE_USE_TABLE;
-    NODE_TRACE_NEXT_QUERY;
-    NODE_SET_TRACE_PROBABILITY;
-    NODE_EXIT;
-    NODE_HELP;
-    NODE_NO_OP;
-    NODE_SHOW_CLUSTER_NAME;
-    NODE_SHOW_VERSION;
-    NODE_SHOW_KEYSPACES;
-    NODE_SHOW_SCHEMA;
-    NODE_THRIFT_GET;
-    NODE_THRIFT_GET_WITH_CONDITIONS;
-    NODE_THRIFT_SET;
-    NODE_THRIFT_COUNT;
-    NODE_THRIFT_DEL;
-    NODE_THRIFT_INCR;
-    NODE_THRIFT_DECR;
-    NODE_ADD_COLUMN_FAMILY;
-    NODE_ADD_KEYSPACE;
-    NODE_DEL_KEYSPACE;
-    NODE_DEL_COLUMN_FAMILY;
-    NODE_UPDATE_KEYSPACE;
-    NODE_UPDATE_COLUMN_FAMILY;
-    NODE_LIST;
-    NODE_TRUNCATE;
-    NODE_ASSUME;
-    NODE_CONSISTENCY_LEVEL;
-    NODE_DROP_INDEX;
-
-    // Internal Nodes.
-    NODE_COLUMN_ACCESS;
-    NODE_ID_LIST;
-    NODE_NEW_CF_ACCESS;
-    NODE_NEW_KEYSPACE_ACCESS;
-    
-    CONVERT_TO_TYPE;
-    FUNCTION_CALL;
-    CONDITION;
-    CONDITIONS;
-    ARRAY;
-    HASH;
-    PAIR;
-
-    NODE_LIMIT;
-    NODE_COLUMNS;
-    NODE_REVERSED;
-    NODE_KEY_RANGE;
-}
-
-@parser::header {
-package org.apache.cassandra.cli;
-}
-
-@lexer::header {
-package org.apache.cassandra.cli;
-}
-
-@lexer::members
-{
-    public void reportError(RecognitionException e) 
-    {
-        StringBuilder errorMessage = new StringBuilder("Syntax error at position ").append(e.charPositionInLine).append(": ");
-
-        if (e instanceof NoViableAltException)
-        {
-            int index = e.charPositionInLine;
-            String error = this.input.substring(index, index);
-            String statement = this.input.substring(0, this.input.size() - 1);
-
-            errorMessage.append("unexpected \"").append(error).append("\" for `").append(statement).append("`.");
-        }
-        else
-        {
-            errorMessage.append(this.getErrorMessage(e, this.getTokenNames()));
-        }
-
-        throw new RuntimeException(errorMessage.toString());
-    }
-}
-
-@parser::members
-{
-    public void reportError(RecognitionException e) 
-    {
-        String errorMessage = "Syntax error at position " + e.charPositionInLine + ": " + this.getErrorMessage(e, this.getTokenNames());
-
-        throw new RuntimeException(errorMessage);
-    }
-}
-
-//
-// Parser Section
-//
-
-// the root node
-root: statement SEMICOLON? EOF -> statement;
-
-statement
-    : connectStatement
-    | exitStatement
-    | countStatement
-    | describeTable
-    | describeCluster
-    | addKeyspace
-    | addColumnFamily
-    | updateKeyspace
-    | updateColumnFamily
-    | delColumnFamily
-    | delKeyspace
-    | useKeyspace
-    | traceNextQuery
-    | setTraceProbability
-    | delStatement
-    | getStatement
-    | helpStatement
-    | setStatement
-    | incrStatement
-    | showStatement
-    | listStatement
-    | truncateStatement
-    | assumeStatement
-    | consistencyLevelStatement
-    | dropIndex
-    | -> ^(NODE_NO_OP)
-    ;
-
-connectStatement
-    : CONNECT host '/' port (username password)?
-        -> ^(NODE_CONNECT host port (username password)?)
-    | CONNECT ip_address '/' port (username password)?
-        -> ^(NODE_CONNECT ip_address port (username password)?)
-    ;
-
-helpStatement
-    : HELP HELP 
-        -> ^(NODE_HELP NODE_HELP)
-    | HELP CONNECT 
-        -> ^(NODE_HELP NODE_CONNECT)
-    | HELP USE 
-        -> ^(NODE_HELP NODE_USE_TABLE)
-    | HELP TRACE NEXT QUERY
-        -> ^(NODE_HELP NODE_TRACE_NEXT_QUERY)
-    | HELP SET TRACE PROBABILITY
-        -> ^(NODE_HELP NODE_SET_TRACE_PROBABILITY)
-    | HELP DESCRIBE
-        -> ^(NODE_HELP NODE_DESCRIBE)
-    | HELP DESCRIBE 'CLUSTER'
-        -> ^(NODE_HELP NODE_DESCRIBE_CLUSTER)
-    | HELP EXIT 
-        -> ^(NODE_HELP NODE_EXIT)
-    | HELP QUIT 
-        -> ^(NODE_HELP NODE_EXIT)
-    | HELP SHOW 'CLUSTER NAME'
-        -> ^(NODE_HELP NODE_SHOW_CLUSTER_NAME)
-    | HELP SHOW KEYSPACES 
-        -> ^(NODE_HELP NODE_SHOW_KEYSPACES)
-    | HELP SHOW SCHEMA
-            -> ^(NODE_HELP NODE_SHOW_SCHEMA)
-    | HELP SHOW API_VERSION
-        -> ^(NODE_HELP NODE_SHOW_VERSION)
-    | HELP CREATE KEYSPACE 
-        -> ^(NODE_HELP NODE_ADD_KEYSPACE)
-    | HELP UPDATE KEYSPACE
-        -> ^(NODE_HELP NODE_UPDATE_KEYSPACE)
-    | HELP CREATE COLUMN FAMILY 
-        -> ^(NODE_HELP NODE_ADD_COLUMN_FAMILY)
-    | HELP UPDATE COLUMN FAMILY
-        -> ^(NODE_HELP NODE_UPDATE_COLUMN_FAMILY)
-    | HELP DROP KEYSPACE 
-        -> ^(NODE_HELP NODE_DEL_KEYSPACE)
-    | HELP DROP COLUMN FAMILY 
-        -> ^(NODE_HELP NODE_DEL_COLUMN_FAMILY)
-    | HELP DROP INDEX
-        -> ^(NODE_HELP NODE_DROP_INDEX)
-    | HELP GET 
-        -> ^(NODE_HELP NODE_THRIFT_GET)
-    | HELP SET 
-        -> ^(NODE_HELP NODE_THRIFT_SET)
-    | HELP INCR
-        -> ^(NODE_HELP NODE_THRIFT_INCR)
-    | HELP DECR
-        -> ^(NODE_HELP NODE_THRIFT_DECR)
-    | HELP DEL 
-        -> ^(NODE_HELP NODE_THRIFT_DEL)
-    | HELP COUNT 
-        -> ^(NODE_HELP NODE_THRIFT_COUNT)
-    | HELP LIST 
-        -> ^(NODE_HELP NODE_LIST)
-    | HELP TRUNCATE
-        -> ^(NODE_HELP NODE_TRUNCATE)
-    | HELP ASSUME
-        -> ^(NODE_HELP NODE_ASSUME)
-    | HELP CONSISTENCYLEVEL
-        -> ^(NODE_HELP NODE_CONSISTENCY_LEVEL)
-    | HELP 
-        -> ^(NODE_HELP)
-    | '?'    
-        -> ^(NODE_HELP)
-    ;
-
-exitStatement
-    : QUIT -> ^(NODE_EXIT)
-    | EXIT -> ^(NODE_EXIT)
-    ;
-
-getStatement
-    : GET columnFamilyExpr ('AS' typeIdentifier)? ('LIMIT' limit=IntegerPositiveLiteral)?
-        -> ^(NODE_THRIFT_GET columnFamilyExpr ( ^(CONVERT_TO_TYPE typeIdentifier) )? ^(NODE_LIMIT $limit)?)
-    | GET columnFamily 'WHERE' getCondition ('AND' getCondition)* ('LIMIT' limit=IntegerPositiveLiteral)?
-        -> ^(NODE_THRIFT_GET_WITH_CONDITIONS columnFamily ^(CONDITIONS getCondition+) ^(NODE_LIMIT $limit)?) 
-    ;
-
-getCondition
-    : columnOrSuperColumn operator value
-        -> ^(CONDITION operator columnOrSuperColumn value)
-    ;
-
-operator
-    : '=' | '>' | '<' | '>=' | '<='
-    ;
-
-typeIdentifier
-    : Identifier | StringLiteral | IntegerPositiveLiteral 
-    ;
-
-setStatement
-    : SET columnFamilyExpr '=' objectValue=value (WITH TTL '=' ttlValue=IntegerPositiveLiteral)?
-        -> ^(NODE_THRIFT_SET columnFamilyExpr $objectValue ( $ttlValue )?)
-    ;
-
-incrStatement
-    : INCR columnFamilyExpr (BY byValue=incrementValue)?
-        -> ^(NODE_THRIFT_INCR columnFamilyExpr ( $byValue )?)
-    | DECR columnFamilyExpr (BY byValue=incrementValue)?
-        -> ^(NODE_THRIFT_DECR columnFamilyExpr ( $byValue )?)
-    ;
-
-countStatement
-    : COUNT columnFamilyExpr 
-        -> ^(NODE_THRIFT_COUNT columnFamilyExpr)
-    ;
-
-delStatement
-    : DEL columnFamilyExpr 
-        -> ^(NODE_THRIFT_DEL columnFamilyExpr)
-    ;
-
-showStatement
-    : showClusterName
-    | showVersion
-    | showKeyspaces
-    | showSchema
-    ;
-
-listStatement
-    : LIST columnFamily keyRangeExpr? rowLimitExpr? columnLimitExpr?
-        -> ^(NODE_LIST columnFamily keyRangeExpr? rowLimitExpr? columnLimitExpr?)
-    ;
-
-truncateStatement
-    : TRUNCATE columnFamily
-        -> ^(NODE_TRUNCATE columnFamily)
-    ;
-
-assumeStatement
-    : ASSUME columnFamily assumptionElement=Identifier 'AS' entityName
-        -> ^(NODE_ASSUME columnFamily $assumptionElement entityName)
-    ;
-
-consistencyLevelStatement
-    : CONSISTENCYLEVEL 'AS' defaultType=Identifier
-        -> ^(NODE_CONSISTENCY_LEVEL $defaultType)
-    ;
-
-showClusterName
-    : SHOW 'CLUSTER NAME'
-        -> ^(NODE_SHOW_CLUSTER_NAME)
-    ;
-
-addKeyspace
-    : CREATE KEYSPACE keyValuePairExpr 
-        -> ^(NODE_ADD_KEYSPACE keyValuePairExpr)
-    ;
-
-addColumnFamily
-    : CREATE COLUMN FAMILY keyValuePairExpr 
-        -> ^(NODE_ADD_COLUMN_FAMILY keyValuePairExpr)
-    ;
-
-updateKeyspace
-    : UPDATE KEYSPACE keyValuePairExpr
-        -> ^(NODE_UPDATE_KEYSPACE keyValuePairExpr)
-    ;
-
-updateColumnFamily
-    : UPDATE COLUMN FAMILY keyValuePairExpr
-        -> ^(NODE_UPDATE_COLUMN_FAMILY keyValuePairExpr)
-    ;
-
-delKeyspace
-    : DROP KEYSPACE keyspace 
-        -> ^(NODE_DEL_KEYSPACE keyspace)
-    ;
-
-delColumnFamily
-    : DROP COLUMN FAMILY columnFamily 
-        -> ^(NODE_DEL_COLUMN_FAMILY columnFamily)
-    ;
-
-dropIndex
-    : DROP INDEX ON columnFamily '.' columnName
-        -> ^(NODE_DROP_INDEX columnFamily columnName)
-    ;
-
-showVersion
-    : SHOW API_VERSION
-        -> ^(NODE_SHOW_VERSION)
-    ;
-
-showKeyspaces
-    : SHOW KEYSPACES 
-        -> ^(NODE_SHOW_KEYSPACES)
-    ;
-
-showSchema
-    : SHOW SCHEMA (keyspace)?
-        -> ^(NODE_SHOW_SCHEMA (keyspace)?)
-    ;
-
-describeTable
-    : DESCRIBE (keyspace)?
-        -> ^(NODE_DESCRIBE (keyspace)?)
-    ;
-    
-describeCluster
-    : DESCRIBE 'CLUSTER'
-        -> ^(NODE_DESCRIBE_CLUSTER)
-    ;
-
-useKeyspace
-    : USE keyspace ( username )? ( password )? 
-        -> ^(NODE_USE_TABLE keyspace ( username )? ( password )?)
-    ;
-    
-traceNextQuery
-    : TRACE NEXT QUERY
-        -> ^(NODE_TRACE_NEXT_QUERY)
-    ;
-
-setTraceProbability
-    : SET TRACE PROBABILITY tracingProbability
-        -> ^(NODE_SET_TRACE_PROBABILITY tracingProbability)
-    ;
-
-keyValuePairExpr
-    : entityName ( (AND | WITH) keyValuePair )*
-        -> ^(NODE_NEW_KEYSPACE_ACCESS entityName ( keyValuePair )* )
-    ;
-            
-keyValuePair 
-    : attr_name '=' attrValue 
-        -> attr_name attrValue
-    ;
-
-attrValue
-    : arrayConstruct
-    | hashConstruct
-    | attrValueString
-    | attrValueInt
-    | attrValueDouble
-    ;
-
-
-arrayConstruct 
-    : '[' (hashConstruct ','?)* ']'
-        -> ^(ARRAY (hashConstruct)*)
-    ; 
-
-hashConstruct 
-    : '{' hashElementPair (',' hashElementPair)* '}'
-        -> ^(HASH (hashElementPair)+)
-    ;
-
-hashElementPair
-    : rowKey ':' rowValue
-        -> ^(PAIR rowKey rowValue)
-    ;
-
-columnFamilyExpr
-    : columnFamily '[' rowKey ']' 
-        ( '[' column=columnOrSuperColumn ']' 
-            ('[' super_column=columnOrSuperColumn ']')? 
-        )?
-      -> ^(NODE_COLUMN_ACCESS columnFamily rowKey ($column ($super_column)? )?)
-    ;
-
-keyRangeExpr
-    :    '[' ( startKey=entityName? ':' endKey=entityName? )? ']'
-      -> ^(NODE_KEY_RANGE $startKey? $endKey?)
-    ;
-
-rowLimitExpr
-    : 'LIMIT' limit=IntegerPositiveLiteral
-        -> ^(NODE_LIMIT $limit)
-    ;
-
-columnLimitExpr
-    : 'COLUMNS' columns=IntegerPositiveLiteral reversedExpr?
-        -> ^(NODE_COLUMNS $columns reversedExpr?)
-    ;
-
-reversedExpr
-    : 'REVERSED'
-        -> ^(NODE_REVERSED)
-    ;
-
-columnName
-	: entityName
-	;
-
-attr_name
-    : Identifier
-    ;
-
-attrValueString
-    : (Identifier | StringLiteral)
-    ;
-      
-attrValueInt
-    : IntegerPositiveLiteral
-  | IntegerNegativeLiteral
-    ;
-
-attrValueDouble
-    : DoubleLiteral
-    ;
-  
-keyspace
-	: entityName
-	;
-
-replica_placement_strategy
-    : StringLiteral
-    ;
-
-keyspaceNewName
-	: entityName
-	;
-
-comparator
-    : StringLiteral
-    ;
-      
-command : Identifier
-    ;
-
-newColumnFamily
-	: entityName
-	;
-
-username: Identifier
-    ;
-
-password: StringLiteral
-    ;
-
-columnFamily
-  : entityName
-  ;
-
-entityName
-  : (Identifier | StringLiteral | IntegerPositiveLiteral | IntegerNegativeLiteral)
-  ;
-
-rowKey	
-    :  (Identifier | StringLiteral | IntegerPositiveLiteral | IntegerNegativeLiteral | functionCall)
-    ;
-
-rowValue  
-    :  (Identifier | StringLiteral | IntegerPositiveLiteral | IntegerNegativeLiteral | functionCall | hashConstruct)
-    ;
-
-value   
-    : (Identifier | IntegerPositiveLiteral | IntegerNegativeLiteral | StringLiteral | functionCall)
-    ;
-
-functionCall 
-    : functionName=Identifier '(' functionArgument? ')'
-        -> ^(FUNCTION_CALL $functionName functionArgument?)
-    ;
-
-functionArgument 
-    : Identifier | StringLiteral | IntegerPositiveLiteral | IntegerNegativeLiteral
-    ;
-
-columnOrSuperColumn
-    : (Identifier | IntegerPositiveLiteral | IntegerNegativeLiteral | StringLiteral | functionCall)
-    ;
-
-host    
-    : host_name
-        -> ^(NODE_ID_LIST host_name)
-    ;
-
-host_name
-    : Identifier ('.' Identifier)*
-    ;
-    
-ip_address
-    : IP_ADDRESS 
-        -> ^(NODE_ID_LIST IP_ADDRESS)
-    ;
-
-port    
-    : IntegerPositiveLiteral
-    ;
-
-incrementValue
-    : IntegerPositiveLiteral
-    | IntegerNegativeLiteral
-    ;
-
-traceSessionId
-    : Identifier
-    ;
-
-tracingProbability
-    : DoubleLiteral
-    ;
-
-//
-// Lexer Section
-//
-
-//
-// Keywords (in alphabetical order for convenience)
-//
-// CLI is case-insensitive with respect to these keywords.
-// However, they MUST be listed in upper case here.
-//
-CONFIG:      'CONFIG';
-CONNECT:     'CONNECT';
-COUNT:       'COUNT';
-DEL:         'DEL';
-DESCRIBE:    'DESCRIBE';
-USE:         'USE';
-TRACE:       'TRACE';
-PROBABILITY: 'PROBABILITY';
-NEXT:        'NEXT';
-QUERY:       'QUERY';
-GET:         'GET';
-HELP:        'HELP';
-EXIT:        'EXIT';
-FILE:        'FILE';
-QUIT:        'QUIT';
-SET:         'SET';
-INCR:        'INCR';
-DECR:        'DECR';
-SHOW:        'SHOW';
-KEYSPACE:    'KEYSPACE';
-KEYSPACES:   'KEYSPACES';
-API_VERSION: 'API VERSION';
-CREATE:      'CREATE';
-DROP:        'DROP';
-COLUMN:      'COLUMN';
-FAMILY:      'FAMILY';
-WITH:        'WITH';
-BY:          'BY';
-AND:         'AND';
-UPDATE:      'UPDATE';
-LIST:        'LIST';
-LIMIT:       'LIMIT';
-TRUNCATE:    'TRUNCATE';
-ASSUME:      'ASSUME';
-TTL:         'TTL';
-CONSISTENCYLEVEL:   'CONSISTENCYLEVEL';
-INDEX:       'INDEX';
-ON:          'ON';
-SCHEMA:      'SCHEMA';
-
-IP_ADDRESS 
-    : IntegerPositiveLiteral '.' IntegerPositiveLiteral '.' IntegerPositiveLiteral '.' IntegerPositiveLiteral
-    ;
-
-// private syntactic rules
-fragment
-Letter
-    : 'a'..'z' 
-    | 'A'..'Z'
-    ;
-
-fragment
-Digit
-    : '0'..'9'
-    ;
-
-fragment
-Alnum
-    : Letter
-    | Digit
-    ;
-
-// syntactic Elements
-IntegerPositiveLiteral
-   : Digit+
-   ;
-
-IntegerNegativeLiteral
-   : '-' Digit+
-   ;
-   
-DoubleLiteral
-   : Digit+ ('.' Digit+)?
-   | ' ' '.' Digit+
-   ;
-
-Identifier
-    : (Letter | Alnum) (Alnum | '_' | '-' )*
-    ;
-
-// literals
-StringLiteral
-    : '\'' SingleStringCharacter* '\''
-    ;
-
-fragment SingleStringCharacter
-    : ~('\'' | '\\')
-    | '\\' EscapeSequence
-    ;
-
-fragment EscapeSequence
-    : CharacterEscapeSequence
-    | '0'
-    | HexEscapeSequence
-    | UnicodeEscapeSequence
-    ;
-
-fragment CharacterEscapeSequence
-    : SingleEscapeCharacter
-    | NonEscapeCharacter
-    ;
-
-fragment NonEscapeCharacter
-    : ~(EscapeCharacter)
-    ;
-
-fragment SingleEscapeCharacter
-    : '\'' | '"' | '\\' | 'b' | 'f' | 'n' | 'r' | 't' | 'v'
-    ;
-
-fragment EscapeCharacter
-    : SingleEscapeCharacter
-    | DecimalDigit
-    | 'x'
-    | 'u'
-    ;
-
-fragment HexEscapeSequence
-    : 'x' HexDigit HexDigit
-    ;
-
-fragment UnicodeEscapeSequence
-    : 'u' HexDigit HexDigit HexDigit HexDigit
-    ;
-
-fragment HexDigit
-    : DecimalDigit | ('a'..'f') | ('A'..'F')
-    ;
-
-fragment DecimalDigit
-    : ('0'..'9')
-    ;
-
-//
-// syntactic elements
-//
-
-SEMICOLON
-    : ';'
-    ;
-
-WS
-    :  (' '|'\r'|'\t'|'\n') {$channel=HIDDEN;}  // whitepace
-    ;
-
-COMMENT 
-    : '--' (~('\n'|'\r'))*                     { $channel=HIDDEN; }
-    | '/*' (options {greedy=false;} : .)* '*/' { $channel=HIDDEN; }
-    ;


[2/5] Remove cassandra-cli

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/38027382/src/resources/org/apache/cassandra/cli/CliHelp.yaml
----------------------------------------------------------------------
diff --git a/src/resources/org/apache/cassandra/cli/CliHelp.yaml b/src/resources/org/apache/cassandra/cli/CliHelp.yaml
deleted file mode 100644
index cc838f8..0000000
--- a/src/resources/org/apache/cassandra/cli/CliHelp.yaml
+++ /dev/null
@@ -1,1262 +0,0 @@
-
-# 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.
-
-# Help file for online commands in Yaml.
-
-banner: |
-    Type 'help;' or '?' for help.
-    Type 'quit;' or 'exit;' to quit.
-
-help: |
-    Getting around:
-    ?                       Display this help.
-    help;                   Display this help.
-    help <command>;         Display command-specific help.
-    exit;                   Exit this utility.
-    quit;                   Exit this utility.
-
-    Commands:
-    assume                  Apply client side validation.
-    connect                 Connect to a Cassandra node.
-    consistencylevel        Sets consisteny level for the client to use.
-    count                   Count columns or super columns.
-    create column family    Add a column family to an existing keyspace.
-    create keyspace         Add a keyspace to the cluster.
-    del                     Delete a column, super column or row.
-    decr                    Decrements a counter column.
-    describe cluster        Describe the cluster configuration.
-    describe                Describe a keyspace and its column families or column family in current keyspace.
-    drop column family      Remove a column family and its data.
-    drop keyspace           Remove a keyspace and its data.
-    drop index              Remove an existing index from specific column.
-    get                     Get rows and columns.
-    incr                    Increments a counter column.
-    list                    List rows in a column family.
-    set                     Set columns.
-    show api version        Show the server API version.
-    show cluster name       Show the cluster name.
-    show keyspaces          Show all keyspaces and their column families.
-    show schema             Show a cli script to create keyspaces and column families.
-    truncate                Drop the data in a column family.
-    update column family    Update the settings for a column family.
-    update keyspace         Update the settings for a keyspace.
-    use                     Switch to a keyspace.
-
-commands:
-    - name: NODE_HELP
-      help: |
-        help <command>;
-
-        Display the general help page with a list of available commands.;
-    - name: NODE_CONNECT
-      help: |
-        connect <hostname>/<port> (<username> '<password>')?;
-
-        Connect to the a Cassandra node on the specified port.
-
-        If a username and password are supplied the login will occur when the
-        'use' statement is executed. If the server does not support authentication
-        it will silently ignore credentials.
-
-        For information on configuring authentication and authorisation see the
-        conf/cassandra.yaml file or the project documentation.
-
-        Required Parameters:
-        - hostname: Machine name or IP address of the node to connect to.
-
-        - port: rpc_port to connect to the node on, as defined in
-        conf/Cassandra.yaml for the node. The default port is 9160.
-
-        Optional Parameters:
-        - password: Password for the supplied username.
-
-        - username: Username to authenticate to the node as.
-
-        Examples:
-        connect localhost/9160;
-        connect localhost/9160 user 'badpasswd';
-        connect 127.0.0.1/9160 user 'badpasswd';
-    - name: NODE_USE_TABLE
-      help: |
-        use <keyspace>;
-        use <keyspace> <username> '<password>';
-
-        Use the specified keyspace.
-
-        If a username and password are supplied they will be used to authorize
-        against the keyspace. Otherwise the credentials supplied to the 'connect'
-        statement will be used to authorize the user . If the server does not
-        support authentication it will silently ignore credentials.
-
-        Required Parameters:
-        - keyspace: Name of the keyspace to use. The keyspace must exist.
-
-        Optional Parameters:
-        - password: Password for the supplied username.
-
-        - username: Username to login to the node as.
-
-        Examples:
-        use Keyspace1;
-        use Keyspace1 user 'badpasswd';
-    - name: NODE_DESCRIBE
-      help: |
-        describe;
-        describe <keyspace>;
-        describe <column_family>;
-
-        Describes the settings for the current or named keyspace, or the settings
-        of the column family in the current authenticated keyspace.
-
-        Optional Parameters:
-        - keyspace: Name of the keyspace to describe.
-        - column_family: Name of the column family to describe.
-
-        Examples:
-        describe; - Describes current authenticated keyspace
-        describe <keyspace>; - Describe this keyspace
-        describe <column_family>; - Describe the colum family in the current authenticated keyspace
-    - name: NODE_DESCRIBE_CLUSTER
-      help: |
-        describe cluster;
-
-        Describes the snitch, partitioner and schema versions for the currently
-        connected cluster.
-
-        NOTE: The cluster should only report one schema version. Multiple versions
-        may indicate a failed schema modification, consult the project documentation.
-
-        Examples:
-        describe cluster;
-    - name: NODE_EXIT
-      help: |
-        exit;
-        quit;
-
-        Exit this utility.
-
-        Examples:
-        exit;
-        quit;
-    - name: NODE_SHOW_CLUSTER_NAME
-      help: |
-        show cluster name;
-
-        Displays the name of the currently connected cluster.
-
-        Examples:
-        show cluster name;
-    - name: NODE_SHOW_VERSION
-      help: |
-        show api version;
-
-        Displays the API version number.
-
-        This version number is used by high level clients and is not the same as
-        the server release version.
-
-        Examples:
-        show api version;
-    - name: NODE_SHOW_KEYSPACES
-      help: |
-        show keyspaces;
-
-        Describes the settings and the column families for all keyspaces on the
-        currently connected cluster.
-
-        Examples:
-        show keyspaces;
-    - name: NODE_SHOW_SCHEMA
-      help: |
-        show schema;
-        show schema <keyspace>;
-
-        Creates a CLI script to create the current, specified or all keyspaces
-        and their column families.
-
-        Optional Parameters:
-        - keyspace: Name of the keyspace to create the script for. If omitted
-        the current keyspace is used, if there is no current keyspace all
-        keyspaces are considered.
-
-        Examples:
-        show schema;
-        show schema Keyspace1;
-    - name: NODE_ADD_KEYSPACE
-      help: |
-        create keyspace <keyspace>;
-        create keyspace <keyspace> with <att1>=<value1>;
-        create keyspace <keyspace> with <att1>=<value1> and <att2>=<value2> ...;
-
-        Create a keyspace with the specified attributes.
-
-        Required Parameters:
-        - keyspace: Name of the new keyspace, "system" is reserved for
-          Cassandra internals. Names may only contain letters, numbers and
-          underscores.
-
-        Keyspace Attributes (all are optional):
-        - placement_strategy: Class used to determine how replicas
-          are distributed among nodes. Defaults to NetworkTopologyStrategy with
-          one datacenter defined with a replication factor of 1 ("[datacenter1:1]").
-
-          Supported values are:
-            - org.apache.cassandra.locator.SimpleStrategy
-            - org.apache.cassandra.locator.NetworkTopologyStrategy
-            - org.apache.cassandra.locator.OldNetworkTopologyStrategy
-
-          SimpleStrategy merely places the first replica at the node whose
-          token is closest to the key (as determined by the Partitioner), and
-          additional replicas on subsequent nodes along the ring in increasing
-          Token order.
-
-          Supports a single strategy option 'replication_factor' that
-          specifies the replication factor for the cluster.
-
-          With NetworkTopologyStrategy, for each datacenter, you can specify
-          how many replicas you want on a per-keyspace basis. Replicas are
-          placed on different racks within each DC, if possible.
-
-          Supports strategy options which specify the replication factor for
-          each datacenter. The replication factor for the entire cluster is the
-          sum of all per datacenter values. Note that the datacenter names
-          must match those used in conf/cassandra-topology.properties.
-
-          OldNetworkToplogyStrategy [formerly RackAwareStrategy]
-          places one replica in each of two datacenters, and the third on a
-          different rack in in the first.  Additional datacenters are not
-          guaranteed to get a replica.  Additional replicas after three are
-          placed in ring order after the third without regard to rack or
-          datacenter.
-
-          Supports a single strategy option 'replication_factor' that
-          specifies the replication factor for the cluster.
-
-        - strategy_options: Optional additional options for placement_strategy.
-          Options have the form {key:value}, see the information on each
-          strategy and the examples.
-
-        - durable_writes: When set to false all Mutations on keyspace will by-pass CommitLog.
-          Set to true by default.
-
-        Examples:
-        create keyspace Keyspace2
-            with placement_strategy = 'org.apache.cassandra.locator.SimpleStrategy'
-            and strategy_options = {replication_factor:4};
-        create keyspace Keyspace3
-            with placement_strategy = 'org.apache.cassandra.locator.NetworkTopologyStrategy'
-            and strategy_options={DC1:2, DC2:2};
-        create keyspace Keyspace4
-            with placement_strategy = 'org.apache.cassandra.locator.OldNetworkTopologyStrategy'
-            and strategy_options = {replication_factor:1};
-    - name: NODE_UPDATE_KEYSPACE
-      help: |
-        update keyspace <keyspace>;
-        update keyspace <keyspace> with <att1>=<value1>;
-        update keyspace <keyspace> with <att1>=<value1> and <att2>=<value2> ...;
-
-        Update a keyspace with the specified attributes.
-
-        Note: updating some keyspace properties may require additional maintenance
-        actions. Consult project documentation for more details.
-
-        Required Parameters:
-        - keyspace: Name of the keyspace to update.
-
-        Keyspace Attributes (all are optional):
-        - placement_strategy: Class used to determine how replicas
-          are distributed among nodes. Defaults to NetworkTopologyStrategy with
-          one datacenter defined with a replication factor of 1 ("[datacenter1:1]").
-
-          Supported values are:
-            - org.apache.cassandra.locator.SimpleStrategy
-            - org.apache.cassandra.locator.NetworkTopologyStrategy
-            - org.apache.cassandra.locator.OldNetworkTopologyStrategy
-
-          SimpleStrategy merely places the first replica at the node whose
-          token is closest to the key (as determined by the Partitioner), and
-          additional replicas on subsequent nodes along the ring in increasing
-          Token order.
-
-          Supports a single strategy option 'replication_factor' that
-          specifies the replication factor for the cluster.
-
-          With NetworkTopologyStrategy, for each datacenter, you can specify
-          how many replicas you want on a per-keyspace basis. Replicas are
-          placed on different racks within each DC, if possible.
-
-          Supports strategy options which specify the replication factor for
-          each datacenter. The replication factor for the entire cluster is the
-          sum of all per datacenter values. Note that the datacenter names
-          must match those used in conf/cassandra-topology.properties.
-
-          OldNetworkToplogyStrategy [formerly RackAwareStrategy]
-          places one replica in each of two datacenters, and the third on a
-          different rack in in the first.  Additional datacenters are not
-          guaranteed to get a replica.  Additional replicas after three are
-          placed in ring order after the third without regard to rack or
-          datacenter.
-
-          Supports a single strategy option 'replication_factor' that
-          specifies the replication factor for the cluster.
-
-        - strategy_options: Optional additional options for placement_strategy.
-          Options have the form {key:value}, see the information on each
-          strategy and the examples.
-
-        - durable_writes: When set to false all Mutations on keyspace will by-pass CommitLog.
-          Set to true by default.
-
-        Examples:
-        update keyspace Keyspace2
-            with placement_strategy = 'org.apache.cassandra.locator.SimpleStrategy'
-            and strategy_options = {replication_factor:4};
-        update keyspace Keyspace3
-            with placement_strategy = 'org.apache.cassandra.locator.NetworkTopologyStrategy'
-            and strategy_options={DC1:2, DC2:2};
-        update keyspace Keyspace4
-            with placement_strategy = 'org.apache.cassandra.locator.OldNetworkTopologyStrategy'
-            and strategy_options = {replication_factor:1};
-    - name: NODE_ADD_COLUMN_FAMILY
-      help: |
-        create column family <name>;
-        create column family <name> with <att1>=<value1>;
-        create column family <name> with <att1>=<value1> and <att2>=<value2>...;
-
-        Create a column family in the current keyspace with the specified
-        attributes.
-
-        Required Parameters:
-        - name: Name of the new column family. Names may only contain letters,
-          numbers and underscores.
-
-        column family Attributes (all are optional):
-        - column_metadata: Defines the validation and indexes for known columns in
-          this column family.
-
-          Columns not listed in the column_metadata section will use the
-          default_validator to validate their values.
-
-          Column Required parameters:
-            - name: Binds a validator (and optionally an indexer) to columns
-              with this name in any row of the enclosing column family.
-
-            - validator: Validator to use for values for this column.
-
-              Supported values are:
-                - AsciiType
-                - BooleanType
-                - BytesType
-                - CounterColumnType (distributed counter column)
-                - DateType
-                - DoubleType
-                - FloatType
-                - Int32Type
-                - IntegerType (a generic variable-length integer type)
-                - LexicalUUIDType
-                - LongType
-                - UTF8Type
-                - CompositeType (should be used with sub-types specified e.g. 'CompositeType(UTF8Type, Int32Type)'
-                  quotes are important (!) in this case)
-
-              It is also valid to specify the fully-qualified class name to a class
-              that extends org.apache.cassandra.db.marshal.AbstractType.
-
-          Column Optional parameters:
-            - index_name: Name for the index. Both an index name and
-              type must be specified.
-
-            - index_type: The type of index to be created.
-
-              Suported values are:
-                - KEYS: a ColumnFamily backed index
-                - CUSTOM: a user supplied index implementaion. You must supply a
-                  'class_name' field in the index_options with the full classname 
-                  of the implementation.
-            
-            - index_options: Optional additional options for index_type.
-              Options have the form {key:value}.
-                   
-        - bloom_filter_fp_chance: Desired false positive probability for
-          sstable row bloom filters.  Default is 0.000744.
-
-        - index_interval: controls the sampling of entries from the primrary
-          row index in terms of space versus time.  The larger the interval,
-          the smaller and less effective the sampling will be. All the sampled
-          entries must fit in memory.  Default value is 128.
-
-        - column_type: Type of columns this column family holds, valid values are
-          Standard and Super. Default is Standard.
-
-        - comment: Human readable column family description.
-
-        - comparator: Validator to use to validate and compare column names in
-          this column family. For Standard column families it applies to columns, for
-          Super column families applied to  super columns. Also see the subcomparator
-          attribute. Default is BytesType, which is a straight forward lexical
-          comparison of the bytes in each column.
-
-          Supported values are:
-            - AsciiType
-            - BooleanType
-            - BytesType
-            - CounterColumnType (distributed counter column)
-            - DateType
-            - DoubleType
-            - FloatType
-            - Int32Type
-            - IntegerType (a generic variable-length integer type)
-            - LexicalUUIDType
-            - LongType
-            - UTF8Type
-            - CompositeType (should be used with sub-types specified e.g. 'CompositeType(UTF8Type, Int32Type)'
-              quotes are important (!) in this case)
-
-          It is also valid to specify the fully-qualified class name to a class that
-          extends org.apache.cassandra.db.marshal.AbstractType.
-
-        - default_validation_class: Validator to use for values in columns which are
-          not listed in the column_metadata. Default is BytesType which applies
-          no validation.
-
-          Supported values are:
-            - AsciiType
-            - BooleanType
-            - BytesType
-            - CounterColumnType (distributed counter column)
-            - DateType
-            - DoubleType
-            - FloatType
-            - Int32Type
-            - IntegerType (a generic variable-length integer type)
-            - LexicalUUIDType
-            - LongType
-            - UTF8Type
-            - CompositeType (should be used with sub-types specified e.g. 'CompositeType(UTF8Type, Int32Type)'
-              quotes are important (!) in this case)
-
-          It is also valid to specify the fully-qualified class name to a class that
-          extends org.apache.cassandra.db.marshal.AbstractType.
-
-        - key_validation_class: Validator to use for keys.
-          Default is BytesType which applies no validation.
-
-          Supported values are:
-            - AsciiType
-            - BooleanType
-            - BytesType
-            - DateType
-            - DoubleType
-            - FloatType
-            - Int32Type
-            - IntegerType (a generic variable-length integer type)
-            - LexicalUUIDType
-            - LongType
-            - UTF8Type
-
-          It is also valid to specify the fully-qualified class name to a class that
-          extends org.apache.cassandra.db.marshal.AbstractType.
-
-        - gc_grace: Time to wait in seconds before garbage collecting tombstone
-          deletion markers. Default value is 864000 or 10 days.
-
-          Set this to a large enough value that you are confident that the deletion
-          markers will be propagated to all replicas by the time this many seconds
-          has elapsed, even in the face of hardware failures.
-
-          See http://wiki.apache.org/Cassandra/DistributedDeletes
-
-        - read_repair_chance: Probability (0.0-1.0) with which to perform read
-          repairs for any read operation. Default is 0.1.
-
-          Note that disabling read repair entirely means that the dynamic snitch
-          will not have any latency information from all the replicas to recognize
-          when one is performing worse than usual.
-
-        - dclocal_read_repair_chance: Probability (0.0-1.0) with which to
-          perform read repairs against the node from the local data-center. If
-          this is lower than read_repair_chance, this will be ignored.
-
-          Example:
-            update column family Standard2
-                 with read_repair_chance=0.1
-                 and dclocal_read_repair_chance=0.5;
-
-            For 10 read queries, 1 will do read repair on all replicas (and
-            thus in particular on all replica of the local DC), 4 will only do
-            read repair on replica of the local DC and 5 will not do any read
-            repair.
-
-        - subcomparator:  Validator to use to validate and compare sub column names
-          in this column family. Only applied to Super column families. Default is
-          BytesType, which is a straight forward lexical comparison of the bytes in
-          each column.
-
-          Supported values are:
-            - AsciiType
-            - BooleanType
-            - BytesType
-            - CounterColumnType (distributed counter column)
-            - DateType
-            - DoubleType
-            - FloatType
-            - Int32Type
-            - IntegerType (a generic variable-length integer type)
-            - LexicalUUIDType
-            - LongType
-            - UTF8Type
-
-          It is also valid to specify the fully-qualified class name to a class that
-          extends org.apache.cassandra.db.marshal.AbstractType.
-
-        - max_compaction_threshold: The maximum number of SSTables allowed before a
-        minor compaction is forced. Default is 32, setting to 0 disables minor
-        compactions.
-
-        Decreasing this will cause minor compactions to start more frequently and
-        be less intensive. The min_compaction_threshold and max_compaction_threshold
-        boundaries are the number of tables Cassandra attempts to merge together at
-        once.
-
-        - min_compaction_threshold: The minimum number of SSTables needed
-        to start a minor compaction. Default is 4, setting to 0 disables minor
-        compactions.
-
-        Increasing this will cause minor compactions to start less frequently and
-        be more intensive. The min_compaction_threshold and max_compaction_threshold
-        boundaries are the number of tables Cassandra attempts to merge together at
-        once.
-
-        - compression_options: Options related to compression.
-          Options have the form {key:value}.
-          The main recognized options are:
-            - sstable_compression: the algorithm to use to compress sstables for
-              this column family. If none is provided, compression will not be
-              enabled. Supported values are SnappyCompressor, DeflateCompressor or
-              any custom compressor. It is also valid to specify the fully-qualified
-              class name to a class that implements org.apache.cassandra.io.ICompressor.
-
-            - chunk_length_kb: specify the size of the chunk used by sstable
-              compression (default to 64, must be a power of 2).
-
-          To disable compression just set compression_options to null like this
-          `compression_options = null`.
-
-        Examples:
-        create column family Super4
-            with column_type = 'Super'
-            and comparator = 'AsciiType'
-            and caching='ALL';
-        create column family Standard3
-            with comparator = 'LongType'
-            and caching='ALL';
-        create column family Standard4
-            with comparator = AsciiType
-            and column_metadata =
-            [{
-                column_name : Test,
-                validation_class : IntegerType,
-                index_type : 0,
-                index_name : IdxName},
-            {
-                column_name : 'other name',
-                validation_class : LongType
-            }];
-        create column family UseComposites
-           with comparator = 'CompositeType(UTF8Type, Int32Type)'
-           and caching='ALL';
-    - name: NODE_UPDATE_COLUMN_FAMILY
-      help: |
-        update column family <name>;
-        update column family <name> with <att1>=<value1>;
-        update column family <name> with <att1>=<value1> and <att2>=<value2>...;
-
-        Updates the settings for a column family in the current keyspace.
-
-        Required Parameters:
-        - name: Name of the column family to update.
-
-        column family Attributes (all are optional):
-        - column_metadata: Defines the validation and indexes for known columns in
-          this column family.
-
-          Columns not listed in the column_metadata section will use the
-          default_validator to validate their values.
-
-          Column Required parameters:
-            - name: Binds a validator (and optionally an indexer) to columns
-              with this name in any row of the enclosing column family.
-
-            - validator: Validator to use for values for this column.
-
-              Supported values are:
-                - AsciiType
-                - BooleanType
-                - BytesType
-                - CounterColumnType (distributed counter column)
-                - DateType
-                - DoubleType
-                - FloatType
-                - Int32Type
-                - IntegerType (a generic variable-length integer type)
-                - LexicalUUIDType
-                - LongType
-                - UTF8Type
-                - CompositeType (should be used with sub-types specified e.g. 'CompositeType(UTF8Type, Int32Type)'
-                  quotes are important (!) in this case)
-
-              It is also valid to specify the fully-qualified class name to a class
-              that extends org.apache.cassandra.db.marshal.AbstractType.
-
-          Column Optional parameters:
-            - index_name: Name for the index. Both an index name and
-              type must be specified.
-
-              NOTE: After the update has completed the column family will only
-              contain the secondary indexes listed in the update statement. Existing
-              indexes will be dropped if they are not present in the update.
-
-            - index_type: The type of index to be created.
-
-              Suported values are:
-                - KEYS: a ColumnFamily backed index
-                - CUSTOM: a user supplied index implementaion. You must supply a
-                  'class_name' field in the index_options with the full classname 
-                  of the implementation.
-            
-            - index_options: Optional additional options for index_type.
-              Options have the form {key:value}.
-
-        - bloom_filter_fp_chance: Desired false positive probability for
-          sstable row bloom filters.  Default is 0.000744.
-
-        - column_type: Type of columns this column family holds, valid values are
-          Standard and Super. Default is Standard.
-
-        - comment: Column family description.
-
-        - comparator: Validator to use to validate and compare column names in
-          this column family. For Standard column families it applies to columns, for
-          Super column families applied to  super columns. Also see the subcomparator
-          attribute. Default is BytesType, which is a straight forward lexical
-          comparison of the bytes in each column.
-
-          Supported values are:
-            - AsciiType
-            - BooleanType
-            - BytesType
-            - CounterColumnType (distributed counter column)
-            - DateType
-            - DoubleType
-            - FloatType
-            - Int32Type
-            - IntegerType (a generic variable-length integer type)
-            - LexicalUUIDType
-            - LongType
-            - UTF8Type
-            - CompositeType (should be used with sub-types specified e.g. 'CompositeType(UTF8Type, Int32Type)'
-              quotes are important (!) in this case)
-
-          It is also valid to specify the fully-qualified class name to a class that
-          extends org.apache.cassandra.db.marshal.AbstractType.
-
-        - default_validation_class: Validator to use for values in columns which are
-          not listed in the column_metadata. Default is BytesType which applies
-          no validation.
-
-          Supported values are:
-            - AsciiType
-            - BooleanType
-            - BytesType
-            - CounterColumnType (distributed counter column)
-            - DateType
-            - DoubleType
-            - FloatType
-            - Int32Type
-            - IntegerType (a generic variable-length integer type)
-            - LexicalUUIDType
-            - LongType
-            - UTF8Type
-            - CompositeType (should be used with sub-types specified e.g. 'CompositeType(UTF8Type, Int32Type)'
-              quotes are important (!) in this case)
-
-          It is also valid to specify the fully-qualified class name to a class that
-          extends org.apache.cassandra.db.marshal.AbstractType.
-
-        - key_validation_class: Validator to use for keys.
-          Default is BytesType which applies no validation.
-
-          Supported values are:
-            - AsciiType
-            - BooleanType
-            - BytesType
-            - DateType
-            - DoubleType
-            - FloatType
-            - Int32Type
-            - IntegerType (a generic variable-length integer type)
-            - LexicalUUIDType
-            - LongType
-            - UTF8Type
-
-          It is also valid to specify the fully-qualified class name to a class that
-          extends org.apache.cassandra.db.marshal.AbstractType.
-
-        - gc_grace: Time to wait in seconds before garbage collecting tombstone
-          deletion markers. Default value is 864000 or 10 days.
-
-          Set this to a large enough value that you are confident that the deletion
-          markers will be propagated to all replicas by the time this many seconds
-          has elapsed, even in the face of hardware failures.
-
-          See http://wiki.apache.org/Cassandra/DistributedDeletes
-
-        - read_repair_chance: Probability (0.0-1.0) with which to perform read
-          repairs for any read operation. Default is 0.1.
-
-          Note that disabling read repair entirely means that the dynamic snitch
-          will not have any latency information from all the replicas to recognize
-          when one is performing worse than usual.
-
-        - dclocal_read_repair_chance: Probability (0.0-1.0) with which to
-          perform read repairs against the node from the local data-center. If
-          this is lower than read_repair_chance, this will be ignored.
-
-          Example:
-            update column family Standard2
-                 with read_repair_chance=0.1
-                 and dclocal_read_repair_chance=0.5;
-
-            For 10 read queries, 1 will do read repair on all replicas (and
-            thus in particular on all replica of the local DC), 4 will only do
-            read repair on replica of the local DC and 5 will not do any read
-            repair.
-
-        - subcomparator:  Validator to use to validate and compare sub column names
-          in this column family. Only applied to Super column families. Default is
-          BytesType, which is a straight forward lexical comparison of the bytes in
-          each column.
-
-          Supported values are:
-            - AsciiType
-            - BooleanType
-            - BytesType
-            - CounterColumnType (distributed counter column)
-            - DateType
-            - DoubleType
-            - FloatType
-            - Int32Type
-            - IntegerType (a generic variable-length integer type)
-            - LexicalUUIDType
-            - LongType
-            - UTF8Type
-
-          It is also valid to specify the fully-qualified class name to a class that
-          extends org.apache.cassandra.db.marshal.AbstractType.
-
-        - caching: Enable/Disable caching for the column family.
-
-          There is 2 kind of caches: key cache and row cache. A column family
-          can be set to use no cache, only one of them or both.
-
-          A key cache hit saves 1 seek while a row cache hit avoids disk
-          completely (saving at least 2 seeks). However the row cache holds
-          entire rows and is thus much more space-intensive. Also, since full
-          rows are stored, row cache should be used only for column families
-          whose rows are small.
-
-          Note: the global memory size of both cache can be set in the
-          configuration file (yaml) or through JMX (though it isn't persisted
-          across restarts in that latter case).
-
-          Supported values are:
-            - ALL (Enable row cache and key Cache)
-            - KEYS_ONLY
-            - ROWS_ONLY
-            - NONE;
-        - cells_per_row_to_cache: State the number of cells per row to cache.
-
-          Defaults to 100. Set to "ALL" if you want the old cache behaviour.
-
-          Will not be used if row caching is not enabled.
-        - speculative_retry: Speculative retry is used to speculate a read failure.
-
-          Speculative retry will execute additional read on a different nodes when
-          the read request doesn't complete within the x milliseconds.
-
-          Xpercentile will execute additional read requests to a different replicas
-          when read request was not completed within X percentile of the
-          normal/earlier recorded latencies.
-
-          Xms will execute additional read request to a diffrent replica when read
-          request was not completed in X milliseconds.
-
-          ALWAYS will execute data read request to 2 (If available) of the replicas
-          expecting a node to fail read.
-
-          Supported values are:
-            - ALWAYS
-            - Xpercentile
-            - Xms
-            - NONE;
-
-        - max_compaction_threshold: The maximum number of SSTables allowed before a
-        minor compaction is forced. Default is 32, setting to 0 disables minor
-        compactions.
-
-        Decreasing this will cause minor compactions to start more frequently and
-        be less intensive. The min_compaction_threshold and max_compaction_threshold
-        boundaries are the number of tables Cassandra attempts to merge together at
-        once.
-
-        - min_compaction_threshold: The minimum number of SSTables needed
-        to start a minor compaction. Default is 4, setting to 0 disables minor
-        compactions.
-
-        Increasing this will cause minor compactions to start less frequently and
-        be more intensive. The min_compaction_threshold and max_compaction_threshold
-        boundaries are the number of tables Cassandra attempts to merge together at
-        once.
-
-        - compression_options: Options related to compression.
-          Options have the form {key:value}.
-          The main recognized options are:
-            - sstable_compression: the algorithm to use to compress sstables for
-              this column family. If none is provided, compression will not be
-              enabled. Supported values are SnappyCompressor, DeflateCompressor or
-              any custom compressor. It is also valid to specify the fully-qualified
-              class name to a class that implements org.apache.cassandra.io.ICompressor.
-
-            - chunk_length_kb: specify the size of the chunk used by sstable
-              compression (default to 64, must be a power of 2).
-
-          To disable compression just set compression_options to null like this
-          `compression_options = null`.
-
-        Examples:
-        update column family Super4
-            with column_type = 'Super'
-            and caching='ALL';
-        update column family Standard3
-            and caching='ALL';
-        update column family Standard4
-            with column_metadata =
-            [{
-                column_name : Test,
-                validation_class : IntegerType,
-                index_type : 0,
-                index_name : IdxName},
-            {
-                column_name : 'other name',
-                validation_class : LongType
-            }];
-    - name: NODE_DEL_KEYSPACE
-      help: |
-        drop keyspace <keyspace>;
-
-        Drops the specified keyspace.
-
-        A snapshot of the data is created in a sub directory of the Keyspace data directory. The files
-        must be manually deleted using either "nodetool clearsnapshot" or the command line.
-
-        Required Parameters:
-        - keyspace: Name of the keyspace to delete.
-
-        Example:
-        drop keyspace Keyspace1;
-    - name: NODE_DEL_COLUMN_FAMILY
-      help: |
-        drop column family <cf>;
-
-        Drops the specified column family.
-
-        A snapshot of the data is created in a sub directory of the Keyspace data directory. The files
-        must be manually deleted using either "nodetool clearsnapshot" or the command line.
-
-        Required Parameters:
-        - cf: Name of the column family to delete.
-
-        Example:
-        drop column family Standard2;
-    - name: NODE_DROP_INDEX
-      help: |
-        drop index on <cf>.<column>;
-
-        Drops index on specified column of the column family.
-
-        Required Parameters:
-          - cf: Name of the column family.
-          - column: Name of the column to delete index on.
-
-        Example:
-        drop index on Users.name;
-    - name: NODE_THRIFT_GET
-      help: |
-        get <cf>['<key>'];
-        get <cf>['<key>']['<col>'] (as <type>)*;
-        get <cf>['<key>']['<super>']['<col>'] (as <type>)*;
-        get <cf>['<key>']['<super>'];
-        get <cf>['<key>'][<function>];
-        get <cf>[function(<key>)][<function>(<super>)][<function>(<col>)];
-        get <cf> where <col> <operator> <value> [
-            and <col> <operator> <value> and ...] [limit <limit>];
-        get <cf> where <col> <operator> <function>(<value>) [
-            and <col> <operator> <function> and ...] [limit <limit>];
-
-        Gets columns or super columns for the specified column family and key. Or
-        returns all columns from rows which meet the specified criteria when using
-        the 'where' form.
-
-        Note: The implementation of secondary indexes in Cassandra 0.7 has some
-        restrictions, see
-        http://www.datastax.com/dev/blog/whats-new-Cassandra-07-secondary-indexes
-
-        Required Parameters:
-        - cf: Name of the column family to read from.
-
-        Optional Parameters:
-        - col: Name of the column to read. Or in the 'where' form name of the column
-        to test the value of.
-
-        - function: Name of a function to call to parse the supplied argument to the
-          specified type. Some functions will generate values without needing an
-          argument.
-
-          Valid options are:
-            - ascii
-            - bytes: if used without arguments generates a zero length byte array
-            - int
-            - integer
-            - lexicaluuid: if used without arguments generates a new random uuid
-            - long
-            - timeuuid: if used without arguments generates a new time uuid
-            - utf8
-
-        - key: Key for the row to read columns from. This parameter is
-          required in all cases except when the 'where' clause is used.
-
-        - limit: Number of rows to return. Default is 100.
-
-        - operator: Operator to test the column value with. Supported operators are
-          =, >, >=, <, <= .
-
-          In Cassandra 0.7 at least one = operator must be present.
-
-        - super: Name of the super column to read from. If super is supplied without
-          col then all columns from the super column are returned.
-
-        - type: Data type to interpret the the columns value as for display.
-
-          Valid options are:
-            - AsciiType
-            - BooleanType
-            - BytesType
-            - CounterColumnType (distributed counter column)
-            - DateType
-            - DoubleType
-            - FloatType
-            - Int32Type
-            - IntegerType (a generic variable-length integer type)
-            - LexicalUUIDType
-            - LongType
-            - UTF8Type
-
-        - value: The value to test the column for, if a function is provided the
-        value is parsed by the function. Otherwise the meta data for the target
-        column is used to determine the correct type.
-
-        Examples:
-        get Standard1[ascii('testkey')];
-        #tell cli to convert keys from ascii to bytes
-        assume Standard1 keys as ascii;
-        get Standard1[testkey][test_column] as IntegerType;
-        get Standard1[testkey][utf8(hello)];
-        get Indexed1 where birthdate=19750403;
-    - name: NODE_THRIFT_SET
-      help: |
-        set <cf>['<key>']['<col>'] = <value>;
-        set <cf>['<key>']['<super>']['<col>'] = <value>;
-        set <cf>['<key>']['<col>'] = <function>(<argument>);
-        set <cf>['<key>']['<super>']['<col>'] = <function>(<argument>);
-        set <cf>[<key>][<function>(<col>)] = <value> || <function>;
-        set <cf>[<function>(<key>)][<function>(<col>) || <col>] =
-            <value> || <function> with ttl = <secs>;
-
-        Sets the column value for the specified column family and key.
-
-        Required Parameters:
-        - cf: Name of the column family to set columns in.
-
-        - col: Name of the column to set.
-
-        - key: Key for the row to set columns in.
-
-        Optional Parameters:
-        - function: Name of a function to call to parse the supplied argument to the
-          specified type. Some functions will generate values without needing an
-          argument.
-
-          Valid options are:
-            - ascii
-            - bytes: if used without arguments generates a zero length byte array
-            - int
-            - integer
-            - lexicaluuid: if used without arguments generates a new random uuid
-            - long
-            - timeuuid: if used without arguments generates a new time uuid
-            - utf8
-
-        - secs: Time To Live for the column in seconds. Defaults to no ttl.
-
-        - super: Name of the super column to contain the column.
-
-        - value: The value to set the column to.
-
-        Examples:
-        set Super1[ascii('testkey')][ascii('my super')][ascii('test col')]='this is a test';
-        set Standard1['testkey']['test col']='this is also a test';
-        set Standard1[testkey][testcol] = utf8('this is utf8 string.');
-        set Standard1[testkey][timeuuid()] = utf8('hello world');
-        set Standard1[testkey][timeuuid()] = utf8('hello world') with ttl = 30;
-        set UseComposites[utf8('testkey')]['CompositeType(utf8(first),int(4))'] = utf8('inserts this string into a column with name first:4');
-    - name: NODE_THRIFT_DEL
-      help: |
-        del <cf>['<key>'];
-        del <cf>['<key>']['<col>'];
-        del <cf>['<key>']['<super>'];
-        del <cf>['<key>']['<super>']['<col>'];
-        del <cf>[<function>(<key>)][<function>(<super>)][<function>(<col>)];
-
-        Deletes a row, a column, or a subcolumn.
-
-        Required Parameters:
-        - cf: Name of the column family to delete from.
-
-        - key: Key for the row delete from.
-
-        Optional Parameters:
-        - col: Name of the column to delete.
-
-        - function: Name of a function to call to parse the supplied argument to the
-          specified type. Some functions will generate values without needing an
-          argument.
-
-           Supported values are:
-            - ascii
-            - bytes: if used without arguments generates a zero length byte array
-            - int
-            - integer
-            - lexicaluuid: if used without arguments generates a new random uuid
-            - long
-            - double
-            - timeuuid: if used without arguments generates a new time uuid
-            - utf8
-
-        - super: Name of the super column to delete from. If col is not specified
-        the super column and all sub columns will be deleted.
-
-        Examples:
-        del Super1[ascii('testkey')][ascii('my_super')][ascii('test_col')];
-        del Standard1['testkey'][ascii('test col')];
-        del Standard1['testkey'];
-        del Standard1[utf8('testkey')];
-    - name: NODE_THRIFT_COUNT
-      help: |
-        count <cf>['<key>'];
-        count <cf>['<key>']['<super>'];
-
-        Count the number of columns in the row with the specified key, or
-        subcolumns in the specified super column.
-
-        Required Parameters:
-        - cf: Name of the column family to read from..
-
-        - key: Key for the row to count.
-
-        Optional Parameters:
-        - super: Name of the super column to count subcolumns in.
-
-        Examples:
-        count Super1['testkey']['my super'];
-        count Standard1['testkey'];
-    - name: NODE_LIST
-      help: |
-        list <cf>;
-        list <cf>[<startKey>:];
-        list <cf>[<startKey>:<endKey>];
-        list <cf>[<startKey>:<endKey>] limit <limit>;
-        list <cf>[<startKey>:<endKey>] ?(limit <limit>) ?(columns <col_limit> ?(reversed));
-
-        List a range of rows, and all of their columns, in the specified column
-        family.
-
-        The order of rows returned is dependant on the Partitioner in use.
-
-        Required Parameters:
-        - cf: Name of the column family to list rows from.
-
-        Optional Parameters:
-        - endKey: Key to end the range at. The end key will be included
-        in the result. Defaults to an empty byte array.
-
-        - limit: Number of rows to return. Default is 100.
-
-        - columns: Number of columns to return per row, Default to MAX_INTEGER. reversed -
-        columns in reverse order.
-
-        - startKey: Key start the range from. The start key will be
-        included in the result. Defaults to an empty byte array.
-
-        Examples:
-        list Standard1;
-        list Super1[j:];
-        list Standard1[j:k] limit 40;
-        list Standard1 columns 2;
-        list Standard1 limit 40 columns 0;
-        list Standard1 columns 3 reversed;
-    - name: NODE_TRUNCATE
-      help: |
-        truncate <cf>;
-
-        Truncate specified column family.
-
-        Note: All nodes in the cluster must be up to truncate command to execute.
-
-        A snapshot of the data is created, which is deleted asyncronously during a
-        'graveyard' compaction.
-
-        Required Parameters:
-        - cf: Name of the column family to truncate.
-
-        Examples:
-        truncate Standard1;
-    - name: NODE_ASSUME
-      help: |
-        assume <cf> comparator as <type>;
-        assume <cf> sub_comparator as <type>;
-        assume <cf> validator as <type>;
-        assume <cf> keys as <type>;
-
-        Assume one of the attributes (comparator, sub_comparator, validator or keys)
-        of the given column family match specified type. The specified type will
-        be used when displaying data returned from the column family.
-
-        This statement does not change the column family definition stored in
-        Cassandra. It only affects the cli and how it will transform values
-        to be sent to and interprets results from Cassandra.
-
-        If results from Cassandra do not validate according to the assumptions an
-        error is displayed in the cli.
-
-        Required Parameters:
-        - cf: Name of the column family to make the assumption about.
-
-        - type: Validator type to use when processing values.
-
-          Supported values are:
-            - ascii
-            - bytes
-            - counterColumn (distributed counter column)
-            - int
-            - integer (a generic variable-length integer type)
-            - lexicalUUID
-            - long
-            - utf8
-
-          It is also valid to specify the fully-qualified class name to a class that
-          extends org.apache.cassandra.db.marshal.AbstractType.
-
-        Examples:
-        assume Standard1 comparator as lexicaluuid;
-        assume Standard1 keys as ascii;
-    - name: NODE_THRIFT_INCR
-      help: |
-        incr <cf>['<key>']['<col>'] [by <value>];
-        incr <cf>['<key>']['<super>']['<col>'] [by <value>];
-
-        Increment the specified counter column by the supplied value.
-
-        Note: Counter columns must be defined using a 'create column family' or
-        'update column family' statement in the column_metadata as using the
-        ColumnCounterType validator.
-
-        Required Parameters:
-        - cf: Name of the column family to increment the column in.
-
-        - col: Name of the counter column to increment.
-
-        - key: Key for the row to increment the counter in.
-
-        Optional Parameters:
-        - super: Name of the super column that contains the counter column.
-
-        - value: Signed integer value to increment the column by. If not supplied
-        1 is used.
-
-        Examples:
-        incr Counter1[ascii('testkey')][ascii('test col')];
-        incr SuperCounter1[ascii('testkey')][ascii('my super')][ascii('test col')] by 42;
-        incr Counter1[ascii('testkey')][ascii('test col')] by -4;
-    - name: NODE_THRIFT_DECR
-      help: |
-        decr <cf>['<key>']['<col>'] [by <value>];
-        decr <cf>['<key>']['<super>']['<col>'] [by <value>];
-
-        Decrement the specified column by the supplied value.
-
-        Note: Counter columns must be defined using a 'create column family' or
-        'update column family' statement in the column_metadata as using the
-        ColumnCounterType validator.
-
-        Required Parameters:
-        - cf: Name of the column family to decrement the column in.
-
-        - col: Name of the counter column to increment.
-
-        - key: Key for the row to decrement the counter in.
-
-        Optional Parameters:
-        - super: Name of the super column that contains the counter column.
-
-        - value: Signed integer value to decrement the column by. If not supplied
-        1 is used.
-
-        Examples:
-        decr Counter1[ascii('testkey')][ascii('test col')];
-        decr SuperCounter1[ascii('testkey')][ascii('my super')][ascii('test col')] by 42;
-        decr Counter1[ascii('testkey')][ascii('test col')] by 10;
-    - name: NODE_CONSISTENCY_LEVEL
-      help: |
-        consistencylevel as <level>
-
-        Sets the consistency level for the client to use. Defaults to One.
-
-        Required Parameters:
-        - level: Consistency level the client should use. Value is case
-          insensitive.
-
-          Supported values are:
-            - ONE
-            - TWO
-            - THREE
-            - QUORUM
-            - ALL
-            - LOCAL_QUORUM
-            - EACH_QUORUM
-            - ANY
-
-          Note: Consistency level ANY can only be used for write operations.

http://git-wip-us.apache.org/repos/asf/cassandra/blob/38027382/test/pig/org/apache/cassandra/pig/CqlRecordReaderTest.java
----------------------------------------------------------------------
diff --git a/test/pig/org/apache/cassandra/pig/CqlRecordReaderTest.java b/test/pig/org/apache/cassandra/pig/CqlRecordReaderTest.java
index 877d30b..a51950b 100644
--- a/test/pig/org/apache/cassandra/pig/CqlRecordReaderTest.java
+++ b/test/pig/org/apache/cassandra/pig/CqlRecordReaderTest.java
@@ -18,32 +18,20 @@
 package org.apache.cassandra.pig;
 
 import java.io.IOException;
-import java.nio.charset.CharacterCodingException;
 import java.util.Iterator;
 
 import org.junit.Assert;
 import org.junit.BeforeClass;
-import org.junit.Ignore;
 import org.junit.Test;
 
-import org.apache.cassandra.db.marshal.TimeUUIDType;
-import org.apache.cassandra.db.marshal.UUIDType;
 import org.apache.cassandra.exceptions.ConfigurationException;
-import org.apache.cassandra.thrift.AuthenticationException;
-import org.apache.cassandra.thrift.AuthorizationException;
-import org.apache.cassandra.thrift.InvalidRequestException;
-import org.apache.cassandra.thrift.NotFoundException;
-import org.apache.cassandra.thrift.TimedOutException;
-import org.apache.cassandra.thrift.UnavailableException;
-import org.apache.cassandra.utils.Hex;
-import org.apache.pig.data.DataByteArray;
 import org.apache.pig.data.Tuple;
 import org.apache.thrift.TException;
-import org.apache.thrift.transport.TTransportException;
 
 public class CqlRecordReaderTest extends PigTestBase
 {
     private static String[] statements = {
+        "DROP KEYSPACE IF EXISTS cql3ks",
         "CREATE KEYSPACE cql3ks WITH replication = {'class': 'SimpleStrategy', 'replication_factor' : 1};",
         "USE cql3ks;",
 
@@ -69,11 +57,11 @@ public class CqlRecordReaderTest extends PigTestBase
     };
 
     @BeforeClass
-    public static void setup() throws TTransportException, IOException, InterruptedException, ConfigurationException,
-    AuthenticationException, AuthorizationException, InvalidRequestException, UnavailableException, TimedOutException, TException, NotFoundException, CharacterCodingException, ClassNotFoundException, NoSuchFieldException, IllegalAccessException, InstantiationException
+    public static void setup() throws IOException, InterruptedException, ConfigurationException,
+        TException, ClassNotFoundException, NoSuchFieldException, IllegalAccessException, InstantiationException
     {
         startCassandra();
-        setupDataByCql(statements);
+        executeCQLStatements(statements);
         startHadoopCluster();
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/38027382/test/pig/org/apache/cassandra/pig/CqlTableDataTypeTest.java
----------------------------------------------------------------------
diff --git a/test/pig/org/apache/cassandra/pig/CqlTableDataTypeTest.java b/test/pig/org/apache/cassandra/pig/CqlTableDataTypeTest.java
index bbd5a87..98e170b 100644
--- a/test/pig/org/apache/cassandra/pig/CqlTableDataTypeTest.java
+++ b/test/pig/org/apache/cassandra/pig/CqlTableDataTypeTest.java
@@ -19,24 +19,16 @@
 package org.apache.cassandra.pig;
 
 import java.io.IOException;
-import java.nio.charset.CharacterCodingException;
 import java.util.Iterator;
 
 import org.apache.cassandra.db.marshal.TimeUUIDType;
 import org.apache.cassandra.db.marshal.UUIDType;
 import org.apache.cassandra.exceptions.ConfigurationException;
-import org.apache.cassandra.thrift.AuthenticationException;
-import org.apache.cassandra.thrift.AuthorizationException;
-import org.apache.cassandra.thrift.InvalidRequestException;
-import org.apache.cassandra.thrift.NotFoundException;
-import org.apache.cassandra.thrift.SchemaDisagreementException;
-import org.apache.cassandra.thrift.TimedOutException;
-import org.apache.cassandra.thrift.UnavailableException;
 import org.apache.cassandra.utils.Hex;
 import org.apache.pig.data.DataByteArray;
 import org.apache.pig.data.Tuple;
 import org.apache.thrift.TException;
-import org.apache.thrift.transport.TTransportException;
+
 import org.junit.Assert;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -64,6 +56,7 @@ public class CqlTableDataTypeTest extends PigTestBase
     //MAP
     //Create table to test the above data types
     private static String[] statements = {
+            "DROP KEYSPACE IF EXISTS cql3ks",
             "CREATE KEYSPACE cql3ks WITH replication = {'class': 'SimpleStrategy', 'replication_factor' : 1}",
             "USE cql3ks;",
 
@@ -208,17 +201,16 @@ public class CqlTableDataTypeTest extends PigTestBase
     };
 
     @BeforeClass
-    public static void setup() throws TTransportException, IOException, InterruptedException, ConfigurationException,
-                                      AuthenticationException, AuthorizationException, InvalidRequestException, UnavailableException, TimedOutException, TException, NotFoundException, CharacterCodingException, ClassNotFoundException, NoSuchFieldException, IllegalAccessException, InstantiationException
+    public static void setup() throws IOException, InterruptedException, ConfigurationException, TException,
+        ClassNotFoundException, NoSuchFieldException, IllegalAccessException, InstantiationException
     {
         startCassandra();
-        setupDataByCql(statements);
+        executeCQLStatements(statements);
         startHadoopCluster();
     }
 
     @Test
-    public void testCqlNativeStorageRegularType()
-    throws AuthenticationException, AuthorizationException, InvalidRequestException, UnavailableException, TimedOutException, TException, NotFoundException, SchemaDisagreementException, IOException
+    public void testCqlNativeStorageRegularType() throws TException, IOException
     {
         //input_cql=select * from cqltable where token(key) > ? and token(key) <= ?
         cqlTableTest("rows = LOAD 'cql://cql3ks/cqltable?" + defaultParameters + nativeParameters + "&input_cql=select%20*%20from%20cqltable%20where%20token(key)%20%3E%20%3F%20and%20token(key)%20%3C%3D%20%3F' USING CqlNativeStorage();");
@@ -288,8 +280,7 @@ public class CqlTableDataTypeTest extends PigTestBase
     }
 
     @Test
-    public void testCqlNativeStorageSetType()
-    throws AuthenticationException, AuthorizationException, InvalidRequestException, UnavailableException, TimedOutException, TException, NotFoundException, SchemaDisagreementException, IOException
+    public void testCqlNativeStorageSetType() throws TException, IOException
     {
         //input_cql=select * from settable where token(key) > ? and token(key) <= ?
         settableTest("set_rows = LOAD 'cql://cql3ks/settable?" + defaultParameters + nativeParameters + "&input_cql=select%20*%20from%20settable%20where%20token(key)%20%3E%20%3F%20and%20token(key)%20%3C%3D%20%3F' USING CqlNativeStorage();");
@@ -355,8 +346,7 @@ public class CqlTableDataTypeTest extends PigTestBase
     }
 
     @Test
-    public void testCqlNativeStorageListType()
-    throws AuthenticationException, AuthorizationException, InvalidRequestException, UnavailableException, TimedOutException, TException, NotFoundException, SchemaDisagreementException, IOException
+    public void testCqlNativeStorageListType() throws TException, IOException
     {
         //input_cql=select * from listtable where token(key) > ? and token(key) <= ?
         listtableTest("list_rows = LOAD 'cql://cql3ks/listtable?" + defaultParameters + nativeParameters + "&input_cql=select%20*%20from%20listtable%20where%20token(key)%20%3E%20%3F%20and%20token(key)%20%3C%3D%20%3F' USING CqlNativeStorage();");
@@ -422,8 +412,7 @@ public class CqlTableDataTypeTest extends PigTestBase
     }
 
     @Test
-    public void testCqlNativeStorageMapType()
-    throws AuthenticationException, AuthorizationException, InvalidRequestException, UnavailableException, TimedOutException, TException, NotFoundException, SchemaDisagreementException, IOException
+    public void testCqlNativeStorageMapType() throws TException, IOException
     {
         //input_cql=select * from maptable where token(key) > ? and token(key) <= ?
         maptableTest("map_rows = LOAD 'cql://cql3ks/maptable?" + defaultParameters + nativeParameters + "&input_cql=select%20*%20from%20maptable%20where%20token(key)%20%3E%20%3F%20and%20token(key)%20%3C%3D%20%3F' USING CqlNativeStorage();");

http://git-wip-us.apache.org/repos/asf/cassandra/blob/38027382/test/pig/org/apache/cassandra/pig/CqlTableTest.java
----------------------------------------------------------------------
diff --git a/test/pig/org/apache/cassandra/pig/CqlTableTest.java b/test/pig/org/apache/cassandra/pig/CqlTableTest.java
index 4ca043d..ecfeb63 100644
--- a/test/pig/org/apache/cassandra/pig/CqlTableTest.java
+++ b/test/pig/org/apache/cassandra/pig/CqlTableTest.java
@@ -19,21 +19,12 @@
 package org.apache.cassandra.pig;
 
 import java.io.IOException;
-import java.nio.charset.CharacterCodingException;
 import java.util.Iterator;
 
 import org.apache.cassandra.exceptions.ConfigurationException;
-import org.apache.cassandra.thrift.AuthenticationException;
-import org.apache.cassandra.thrift.AuthorizationException;
-import org.apache.cassandra.thrift.InvalidRequestException;
-import org.apache.cassandra.thrift.NotFoundException;
-import org.apache.cassandra.thrift.SchemaDisagreementException;
-import org.apache.cassandra.thrift.TimedOutException;
-import org.apache.cassandra.thrift.UnavailableException;
 import org.apache.pig.data.DataBag;
 import org.apache.pig.data.Tuple;
 import org.apache.thrift.TException;
-import org.apache.thrift.transport.TTransportException;
 import org.junit.Assert;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -41,6 +32,7 @@ import org.junit.Test;
 public class CqlTableTest extends PigTestBase
 {    
     private static String[] statements = {
+            "DROP KEYSPACE IF EXISTS cql3ks",
             "CREATE KEYSPACE cql3ks WITH replication = {'class': 'SimpleStrategy', 'replication_factor' : 1}",
             "USE cql3ks;",
 
@@ -81,17 +73,16 @@ public class CqlTableTest extends PigTestBase
     };
 
     @BeforeClass
-    public static void setup() throws TTransportException, IOException, InterruptedException, ConfigurationException,
-                                      AuthenticationException, AuthorizationException, InvalidRequestException, UnavailableException, TimedOutException, TException, NotFoundException, CharacterCodingException, ClassNotFoundException, NoSuchFieldException, IllegalAccessException, InstantiationException
+    public static void setup() throws IOException, InterruptedException, ConfigurationException, TException, ClassNotFoundException,
+        NoSuchFieldException, IllegalAccessException, InstantiationException
     {
         startCassandra();
-        setupDataByCql(statements);
+        executeCQLStatements(statements);
         startHadoopCluster();
     }
 
     @Test
-    public void testCqlNativeStorageSchema()
-    throws AuthenticationException, AuthorizationException, InvalidRequestException, UnavailableException, TimedOutException, TException, NotFoundException, SchemaDisagreementException, IOException
+    public void testCqlNativeStorageSchema() throws TException, IOException
     {
         //input_cql=select * from cqltable where token(key1) > ? and token(key1) <= ?
         cqlTableSchemaTest("rows = LOAD 'cql://cql3ks/cqltable?" + defaultParameters + nativeParameters +  "&input_cql=select%20*%20from%20cqltable%20where%20token(key1)%20%3E%20%3F%20and%20token(key1)%20%3C%3D%20%3F' USING CqlNativeStorage();");
@@ -136,15 +127,13 @@ public class CqlTableTest extends PigTestBase
     }
 
     @Test
-    public void testCqlNativeStorageSingleKeyTable()
-    throws AuthenticationException, AuthorizationException, InvalidRequestException, UnavailableException, TimedOutException, TException, NotFoundException, SchemaDisagreementException, IOException
+    public void testCqlNativeStorageSingleKeyTable() throws TException, IOException
     {
         //input_cql=select * from moredata where token(x) > ? and token(x) <= ?
         SingleKeyTableTest("moretestvalues= LOAD 'cql://cql3ks/moredata?" + defaultParameters + nativeParameters + "&input_cql=select%20*%20from%20moredata%20where%20token(x)%20%3E%20%3F%20and%20token(x)%20%3C%3D%20%3F' USING CqlNativeStorage();");
     }
 
-    private void SingleKeyTableTest(String initialQuery)
-    throws AuthenticationException, AuthorizationException, InvalidRequestException, UnavailableException, TimedOutException, TException, NotFoundException, SchemaDisagreementException, IOException
+    private void SingleKeyTableTest(String initialQuery) throws TException, IOException
     {
         pig.setBatchOn();
         pig.registerQuery(initialQuery);
@@ -167,15 +156,13 @@ public class CqlTableTest extends PigTestBase
     }
 
     @Test
-    public void testCqlNativeStorageCompositeKeyTable()
-    throws AuthenticationException, AuthorizationException, InvalidRequestException, UnavailableException, TimedOutException, TException, NotFoundException, SchemaDisagreementException, IOException
+    public void testCqlNativeStorageCompositeKeyTable() throws TException, IOException
     {
         //input_cql=select * from compmore where token(id) > ? and token(id) <= ?
         CompositeKeyTableTest("moredata= LOAD 'cql://cql3ks/compmore?" + defaultParameters + nativeParameters + "&input_cql=select%20*%20from%20compmore%20where%20token(id)%20%3E%20%3F%20and%20token(id)%20%3C%3D%20%3F' USING CqlNativeStorage();");
     }
 
-    private void CompositeKeyTableTest(String initialQuery)
-    throws AuthenticationException, AuthorizationException, InvalidRequestException, UnavailableException, TimedOutException, TException, NotFoundException, SchemaDisagreementException, IOException
+    private void CompositeKeyTableTest(String initialQuery) throws TException, IOException
     {
         pig.setBatchOn();
         pig.registerQuery(initialQuery);
@@ -204,15 +191,13 @@ public class CqlTableTest extends PigTestBase
     }
 
     @Test
-    public void testCqlNativeStorageCollectionColumnTable()
-    throws AuthenticationException, AuthorizationException, InvalidRequestException, UnavailableException, TimedOutException, TException, NotFoundException, SchemaDisagreementException, IOException
+    public void testCqlNativeStorageCollectionColumnTable() throws TException, IOException
     {
         //input_cql=select * from collectiontable where token(m) > ? and token(m) <= ?
         CollectionColumnTableTest("collectiontable= LOAD 'cql://cql3ks/collectiontable?" + defaultParameters + nativeParameters + "&input_cql=select%20*%20from%20collectiontable%20where%20token(m)%20%3E%20%3F%20and%20token(m)%20%3C%3D%20%3F' USING CqlNativeStorage();");
     }
 
-    private void CollectionColumnTableTest(String initialQuery)
-    throws AuthenticationException, AuthorizationException, InvalidRequestException, UnavailableException, TimedOutException, TException, NotFoundException, SchemaDisagreementException, IOException
+    private void CollectionColumnTableTest(String initialQuery) throws TException, IOException
     {
         pig.setBatchOn();
         pig.registerQuery(initialQuery);
@@ -245,7 +230,8 @@ public class CqlTableTest extends PigTestBase
     }
 
     @Test
-    public void testCassandraStorageSchema() throws IOException, ClassNotFoundException, TException, TimedOutException, NotFoundException, InvalidRequestException, NoSuchFieldException, UnavailableException, IllegalAccessException, InstantiationException
+    public void testCassandraStorageSchema() throws IOException, ClassNotFoundException, TException, NoSuchFieldException,
+        IllegalAccessException, InstantiationException
     {
         //results: (key1,{((111,),),((111,column1),100),((111,column2),10.1)})
         pig.registerQuery("rows = LOAD 'cassandra://cql3ks/cqltable?" + defaultParameters + "' USING CassandraStorage();");
@@ -259,10 +245,10 @@ public class CqlTableTest extends PigTestBase
             DataBag columns = (DataBag) t.get(1);
             Iterator<Tuple> iter = columns.iterator();
             int i = 0;
-            while(iter.hasNext())
+            while (iter.hasNext())
             {
                 i++;
-                Tuple column = (Tuple) iter.next();
+                Tuple column = iter.next();
                 if (i==1)
                 {
                     Assert.assertEquals(((Tuple) column.get(0)).get(0), 111);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/38027382/test/pig/org/apache/cassandra/pig/PigTestBase.java
----------------------------------------------------------------------
diff --git a/test/pig/org/apache/cassandra/pig/PigTestBase.java b/test/pig/org/apache/cassandra/pig/PigTestBase.java
index 4b3e422..8c27f6c 100644
--- a/test/pig/org/apache/cassandra/pig/PigTestBase.java
+++ b/test/pig/org/apache/cassandra/pig/PigTestBase.java
@@ -18,14 +18,9 @@
  */
 package org.apache.cassandra.pig;
 
-import java.io.ByteArrayOutputStream;
 import java.io.IOException;
-import java.io.PrintStream;
-import java.nio.charset.CharacterCodingException;
 
 import org.apache.cassandra.SchemaLoader;
-import org.apache.cassandra.cli.CliMain;
-import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.db.marshal.TypeParser;
@@ -35,11 +30,6 @@ import org.apache.cassandra.service.EmbeddedCassandraService;
 import org.apache.cassandra.thrift.Cassandra;
 import org.apache.cassandra.thrift.Compression;
 import org.apache.cassandra.thrift.ConsistencyLevel;
-import org.apache.cassandra.thrift.InvalidRequestException;
-import org.apache.cassandra.thrift.NotFoundException;
-import org.apache.cassandra.thrift.SchemaDisagreementException;
-import org.apache.cassandra.thrift.TimedOutException;
-import org.apache.cassandra.thrift.UnavailableException;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.pig.ExecType;
@@ -56,7 +46,6 @@ import org.apache.thrift.transport.TTransport;
 import org.apache.thrift.transport.TTransportException;
 import org.junit.After;
 import org.junit.AfterClass;
-import org.junit.Assert;
 import org.junit.Before;
 
 public class PigTestBase extends SchemaLoader
@@ -119,75 +108,20 @@ public class PigTestBase extends SchemaLoader
         {
             return TypeParser.parse(type);
         }
-        catch (ConfigurationException e)
-        {
-            throw new IOException(e);
-        }
-        catch (SyntaxException e)
+        catch (ConfigurationException | SyntaxException e)
         {
             throw new IOException(e);
         }
     }
 
-    protected static void setupDataByCli(String[] statements) throws CharacterCodingException, ClassNotFoundException, TException, TimedOutException, NotFoundException, InvalidRequestException, NoSuchFieldException, UnavailableException, IllegalAccessException, InstantiationException
-    {
-        // new error/output streams for CliSessionState
-        ByteArrayOutputStream errStream = new ByteArrayOutputStream();
-        ByteArrayOutputStream outStream = new ByteArrayOutputStream();
-
-        // checking if we can connect to the running cassandra node on localhost
-        CliMain.connect("127.0.0.1", 9170);
-
-        // setting new output stream
-        CliMain.sessionState.setOut(new PrintStream(outStream));
-        CliMain.sessionState.setErr(new PrintStream(errStream));
-
-        // re-creating keyspace for tests
-        try
-        {
-            // dropping in case it exists e.g. could be left from previous run
-            CliMain.processStatement("drop keyspace thriftKs;");
-        }
-        catch (Exception e)
-        {
-        }
-
-        for (String statement : statements)
-        {
-            errStream.reset();
-            System.out.println("Executing statement: " + statement);
-            CliMain.processStatement(statement);
-            String result = outStream.toString();
-            System.out.println("result: " + result);
-            outStream.reset(); // reset stream so we have only output from next statement all the time
-            errStream.reset(); // no errors to the end user.
-        }
-    }
-    
-    protected static void setupDataByCql(String[] statements) throws InvalidRequestException, UnavailableException, TimedOutException, TException
+    protected static void executeCQLStatements(String[] statements) throws TException
     {
         Cassandra.Client client = getClient();
-        // re-creating keyspace for tests
-        try
-        {
-            // dropping in case it exists e.g. could be left from previous run
-            client.execute_cql3_query(ByteBufferUtil.bytes("DROP KEYSPACE cql3ks"), Compression.NONE, ConsistencyLevel.ONE);
-        }
-        catch (Exception e)
-        {
-        }
 
         for (String statement : statements)
         {
-            try
-            {
-                System.out.println("Executing statement: " + statement);
-                client.execute_cql3_query(ByteBufferUtil.bytes(statement), Compression.NONE, ConsistencyLevel.ONE);
-            }
-            catch (SchemaDisagreementException e)
-            {
-                Assert.fail();
-            }
+            System.out.println("Executing statement: " + statement);
+            client.execute_cql3_query(ByteBufferUtil.bytes(statement), Compression.NONE, ConsistencyLevel.ONE);
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/38027382/test/pig/org/apache/cassandra/pig/ThriftColumnFamilyDataTypeTest.java
----------------------------------------------------------------------
diff --git a/test/pig/org/apache/cassandra/pig/ThriftColumnFamilyDataTypeTest.java b/test/pig/org/apache/cassandra/pig/ThriftColumnFamilyDataTypeTest.java
index cc54620..2ffd524 100644
--- a/test/pig/org/apache/cassandra/pig/ThriftColumnFamilyDataTypeTest.java
+++ b/test/pig/org/apache/cassandra/pig/ThriftColumnFamilyDataTypeTest.java
@@ -19,181 +19,138 @@
 package org.apache.cassandra.pig;
 
 import java.io.IOException;
-import java.nio.charset.CharacterCodingException;
-import java.util.Iterator;
 
 import org.apache.cassandra.db.marshal.TimeUUIDType;
 import org.apache.cassandra.db.marshal.UUIDType;
 import org.apache.cassandra.exceptions.ConfigurationException;
-import org.apache.cassandra.thrift.AuthenticationException;
-import org.apache.cassandra.thrift.AuthorizationException;
-import org.apache.cassandra.thrift.InvalidRequestException;
-import org.apache.cassandra.thrift.NotFoundException;
-import org.apache.cassandra.thrift.TimedOutException;
-import org.apache.cassandra.thrift.UnavailableException;
 import org.apache.cassandra.utils.Hex;
 import org.apache.pig.data.DataBag;
 import org.apache.pig.data.DataByteArray;
 import org.apache.pig.data.Tuple;
 import org.apache.thrift.TException;
-import org.apache.thrift.transport.TTransportException;
-import org.junit.Assert;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
+import static junit.framework.Assert.assertEquals;
+
 public class ThriftColumnFamilyDataTypeTest extends PigTestBase
 {
-    //AsciiType
-    //LongType
-    //BytesType
-    //BooleanType
-    //CounterColumnType
-    //DecimalType
-    //DoubleType
-    //FloatType
-    //InetAddressType
-    //Int32Type
-    //UTF8Type
-    //DateType
-    //UUIDType
-    //IntegerType
-    //TimeUUIDType
-    //IntegerType
-    //LexicalUUIDType
     private static String[] statements = {
-            "create keyspace thriftKs with placement_strategy = 'org.apache.cassandra.locator.SimpleStrategy' and" +
-            " strategy_options={replication_factor:1};",
-            "use thriftKs;",
-
-            "create column family SomeApp " +
-                    " with comparator = UTF8Type " +
-                    " and default_validation_class = UTF8Type " +
-                    " and key_validation_class = UTF8Type " +
-                    " and column_metadata = [" +
-                    "{column_name: col_ascii, validation_class: AsciiType}, " +
-                    "{column_name: col_long, validation_class: LongType}, " +
-                    "{column_name: col_bytes, validation_class: BytesType}, " +
-                    "{column_name: col_boolean, validation_class: BooleanType}, " +
-                    "{column_name: col_decimal, validation_class: DecimalType}, " +
-                    "{column_name: col_double, validation_class: DoubleType}, " +
-                    "{column_name: col_float, validation_class: FloatType}," +
-                    "{column_name: col_inetaddress, validation_class: InetAddressType}, " +
-                    "{column_name: col_int32, validation_class: Int32Type}, " +
-                    "{column_name: col_uft8, validation_class: UTF8Type}, " +
-                    "{column_name: col_date, validation_class: DateType}, " +
-                    "{column_name: col_uuid, validation_class: UUIDType}, " +
-                    "{column_name: col_integer, validation_class: IntegerType}, " +
-                    "{column_name: col_timeuuid, validation_class: TimeUUIDType}, " +
-                    "{column_name: col_lexical_uuid, validation_class: LexicalUUIDType}, " +
-                    "]; ",
-
-             "set SomeApp['foo']['col_ascii'] = 'ascii';",
-             "set SomeApp['foo']['col_boolean'] = false;",
-             "set SomeApp['foo']['col_bytes'] = 'DEADBEEF';",
-             "set SomeApp['foo']['col_date'] = '2011-02-03T04:05:00+0000';",
-             "set SomeApp['foo']['col_decimal'] = '23.345';",
-             "set SomeApp['foo']['col_double'] = '2.7182818284590451';",
-             "set SomeApp['foo']['col_float'] = '23.45';",
-             "set SomeApp['foo']['col_inetaddress'] = '127.0.0.1';",          
-             "set SomeApp['foo']['col_int32'] = 23;",
-             "set SomeApp['foo']['col_integer'] = 12345;",
-             "set SomeApp['foo']['col_long'] = 12345678;",
-             "set SomeApp['foo']['col_lexical_uuid'] = 'e23f450f-53a6-11e2-7f7f-7f7f7f7f7f77';",
-             "set SomeApp['foo']['col_timeuuid'] = 'e23f450f-53a6-11e2-7f7f-7f7f7f7f7f7f';",
-             "set SomeApp['foo']['col_uft8'] = 'hello';",
-             "set SomeApp['foo']['col_uuid'] = '550e8400-e29b-41d4-a716-446655440000';",
-
-             "create column family CC with " +
-                       "key_validation_class = UTF8Type and " +
-                       "default_validation_class=CounterColumnType " +
-                       "and comparator=UTF8Type;",
-
-             "incr CC['chuck']['kick'];",
-             "incr CC['chuck']['kick'];",
-             "incr CC['chuck']['kick'];"
+            "DROP KEYSPACE IF EXISTS thrift_ks",
+            "CREATE KEYSPACE thrift_ks WITH replication = {'class': 'SimpleStrategy', 'replication_factor': 1};",
+            "USE thrift_ks;",
+
+            "CREATE TABLE some_app (" +
+            "key text PRIMARY KEY," +
+            "col_ascii ascii," +
+            "col_bigint bigint," +
+            "col_blob blob," +
+            "col_boolean boolean," +
+            "col_decimal decimal," +
+            "col_double double," +
+            "col_float float," +
+            "col_inet inet," +
+            "col_int int," +
+            "col_text text," +
+            "col_timestamp timestamp," +
+            "col_timeuuid timeuuid," +
+            "col_uuid uuid," +
+            "col_varint varint)" +
+            " WITH COMPACT STORAGE;",
+
+            "INSERT INTO some_app (key, col_ascii, col_bigint, col_blob, col_boolean, col_decimal, col_double, col_float," +
+                "col_inet, col_int, col_text, col_timestamp, col_uuid, col_varint, col_timeuuid) " +
+                    "VALUES ('foo', 'ascii', 12345678, 0xDEADBEEF, false, 23.345, 2.7182818284590451, 23.45, '127.0.0.1', 23, 'hello', " +
+                        "'2011-02-03T04:05:00+0000', 550e8400-e29b-41d4-a716-446655440000, 12345, e23f450f-53a6-11e2-7f7f-7f7f7f7f7f7f);",
+
+            "CREATE TABLE cc (key text, name text, value counter, PRIMARY KEY (key, name)) WITH COMPACT STORAGE",
+
+            "UPDATE cc SET value = value + 3 WHERE key = 'chuck' AND name = 'kick'",
     };
 
     @BeforeClass
-    public static void setup() throws TTransportException, IOException, InterruptedException, ConfigurationException,
-                                      AuthenticationException, AuthorizationException, InvalidRequestException, UnavailableException, TimedOutException, TException, NotFoundException, CharacterCodingException, ClassNotFoundException, NoSuchFieldException, IllegalAccessException, InstantiationException
+    public static void setup() throws IOException, InterruptedException, ConfigurationException, TException,
+        ClassNotFoundException, NoSuchFieldException, IllegalAccessException, InstantiationException
     {
         startCassandra();
-        setupDataByCli(statements);
+        executeCQLStatements(statements);
         startHadoopCluster();
     }
 
     @Test
-    public void testCassandraStorageDataType() throws IOException, ClassNotFoundException, TException, TimedOutException, NotFoundException, InvalidRequestException, NoSuchFieldException, UnavailableException, IllegalAccessException, InstantiationException
+    public void testCassandraStorageDataType() throws IOException, ClassNotFoundException, TException,
+        NoSuchFieldException, IllegalAccessException, InstantiationException
     {
-        pig.registerQuery("rows = LOAD 'cassandra://thriftKs/SomeApp?" + defaultParameters + "' USING CassandraStorage();");
-
-        //{key: chararray, col_ascii: (name: chararray,value: chararray),
-        //col_boolean: (name: chararray,value: bytearray),
-        //col_bytes: (name: chararray,value: bytearray),
-        //col_date: (name: chararray,value: long),
-        //col_decimal: (name: chararray,value: chararray),
-        //col_double: (name: chararray,value: double),
-        //col_float: (name: chararray,value: float),
-        //col_inetaddress: (name: chararray,value: chararray),
-        //col_int32: (name: chararray,value: int),
-        //col_integer: (name: chararray,value: int),
-        //col_lexical_uuid: (name: chararray,value: chararray),
-        //col_long: (name: chararray,value: long),
-        //col_timeuuid: (name: chararray,value: bytearray),
-        //col_uft8: (name: chararray,value: chararray),
-        //col_uuid: (name: chararray,value: chararray),
-        //columns: {(name: chararray,value: chararray)}}
-        Iterator<Tuple> it = pig.openIterator("rows");
-        if (it.hasNext()) {
-            Tuple t = it.next();
-            Assert.assertEquals(t.get(0), "foo");
-            Tuple column = (Tuple) t.get(1);
-            Assert.assertEquals(column.get(1), "ascii");
-            column = (Tuple) t.get(2);
-            Assert.assertEquals(column.get(1), false);
-            column = (Tuple) t.get(3);
-            Assert.assertEquals(column.get(1), new DataByteArray(Hex.hexToBytes("DEADBEEF")));
-            column = (Tuple) t.get(4);
-            Assert.assertEquals(column.get(1), 1296705900000L);
-            column = (Tuple) t.get(5);
-            Assert.assertEquals(column.get(1), "23.345");
-            column = (Tuple) t.get(6);
-            Assert.assertEquals(column.get(1), 2.7182818284590451d);
-            column = (Tuple) t.get(7);
-            Assert.assertEquals(column.get(1), 23.45f);
-            column = (Tuple) t.get(8);
-            Assert.assertEquals(column.get(1), "127.0.0.1");
-            column = (Tuple) t.get(9);
-            Assert.assertEquals(column.get(1), 23);
-            column = (Tuple) t.get(10);
-            Assert.assertEquals(column.get(1), 12345);
-            column = (Tuple) t.get(11);
-            Assert.assertEquals(column.get(1), new DataByteArray((TimeUUIDType.instance.fromString("e23f450f-53a6-11e2-7f7f-7f7f7f7f7f77").array())));
-            column = (Tuple) t.get(12);
-            Assert.assertEquals(column.get(1), 12345678L);
-            column = (Tuple) t.get(13);
-            Assert.assertEquals(column.get(1), new DataByteArray((TimeUUIDType.instance.fromString("e23f450f-53a6-11e2-7f7f-7f7f7f7f7f7f").array())));
-            column = (Tuple) t.get(14);
-            Assert.assertEquals(column.get(1), "hello");
-            column = (Tuple) t.get(15);
-            Assert.assertEquals(column.get(1), new DataByteArray((UUIDType.instance.fromString("550e8400-e29b-41d4-a716-446655440000").array())));
-        }
-
-        pig.registerQuery("cc_rows = LOAD 'cassandra://thriftKs/CC?" + defaultParameters + "' USING CassandraStorage();");
-
-        //(chuck,{(kick,3)})
-        it = pig.openIterator("cc_rows");
-        if (it.hasNext()) {
-            Tuple t = it.next();
-            Assert.assertEquals(t.get(0), "chuck");           
-            DataBag columns = (DataBag) t.get(1);
-            Iterator<Tuple> iter = columns.iterator();
-            if(iter.hasNext())
-            {
-                Tuple column = iter.next();
-                Assert.assertEquals(column.get(0), "kick");
-                Assert.assertEquals(column.get(1), 3L);
-            }
-         }
+        pig.registerQuery("rows = LOAD 'cassandra://thrift_ks/some_app?" + defaultParameters + "' USING CassandraStorage();");
+        Tuple t = pig.openIterator("rows").next();
+
+        // key
+        assertEquals("foo", t.get(0));
+
+        // col_ascii
+        Tuple column = (Tuple) t.get(1);
+        assertEquals("ascii", column.get(1));
+
+        // col_bigint
+        column = (Tuple) t.get(2);
+        assertEquals(12345678L, column.get(1));
+
+        // col_blob
+        column = (Tuple) t.get(3);
+        assertEquals(new DataByteArray(Hex.hexToBytes("DEADBEEF")), column.get(1));
+
+        // col_boolean
+        column = (Tuple) t.get(4);
+        assertEquals(false, column.get(1));
+
+        // col_decimal
+        column = (Tuple) t.get(5);
+        assertEquals("23.345", column.get(1));
+
+        // col_double
+        column = (Tuple) t.get(6);
+        assertEquals(2.7182818284590451d, column.get(1));
+
+        // col_float
+        column = (Tuple) t.get(7);
+        assertEquals(23.45f, column.get(1));
+
+        // col_inet
+        column = (Tuple) t.get(8);
+        assertEquals("127.0.0.1", column.get(1));
+
+        // col_int
+        column = (Tuple) t.get(9);
+        assertEquals(23, column.get(1));
+
+        // col_text
+        column = (Tuple) t.get(10);
+        assertEquals("hello", column.get(1));
+
+        // col_timestamp
+        column = (Tuple) t.get(11);
+        assertEquals(1296705900000L, column.get(1));
+
+        // col_timeuuid
+        column = (Tuple) t.get(12);
+        assertEquals(new DataByteArray((TimeUUIDType.instance.fromString("e23f450f-53a6-11e2-7f7f-7f7f7f7f7f7f").array())), column.get(1));
+
+        // col_uuid
+        column = (Tuple) t.get(13);
+        assertEquals(new DataByteArray((UUIDType.instance.fromString("550e8400-e29b-41d4-a716-446655440000").array())), column.get(1));
+
+        // col_varint
+        column = (Tuple) t.get(14);
+        assertEquals(12345, column.get(1));
+
+        pig.registerQuery("cc_rows = LOAD 'cassandra://thrift_ks/cc?" + defaultParameters + "' USING CassandraStorage();");
+        t = pig.openIterator("cc_rows").next();
+
+        assertEquals("chuck", t.get(0));
+
+        DataBag columns = (DataBag) t.get(1);
+        column = columns.iterator().next();
+        assertEquals("kick", column.get(0));
+        assertEquals(3L, column.get(1));
     }
 }


[3/5] Remove cassandra-cli

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/38027382/src/java/org/apache/cassandra/cli/CliCommandHelp.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cli/CliCommandHelp.java b/src/java/org/apache/cassandra/cli/CliCommandHelp.java
deleted file mode 100644
index dce8d60..0000000
--- a/src/java/org/apache/cassandra/cli/CliCommandHelp.java
+++ /dev/null
@@ -1,24 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.cassandra.cli;
-
-public class CliCommandHelp
-{
-    public String name;
-    public String help;
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/38027382/src/java/org/apache/cassandra/cli/CliCompiler.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cli/CliCompiler.java b/src/java/org/apache/cassandra/cli/CliCompiler.java
deleted file mode 100644
index 6b84be1..0000000
--- a/src/java/org/apache/cassandra/cli/CliCompiler.java
+++ /dev/null
@@ -1,172 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.cassandra.cli;
-
-import java.util.List;
-
-import org.antlr.runtime.ANTLRStringStream;
-import org.antlr.runtime.CharStream;
-import org.antlr.runtime.CommonTokenStream;
-import org.antlr.runtime.tree.Tree;
-import org.apache.cassandra.thrift.CfDef;
-import org.apache.cassandra.thrift.KsDef;
-
-
-public class CliCompiler
-{
-
-    // ANTLR does not provide case-insensitive tokenization support
-    // out of the box. So we override the LA (lookahead) function
-    // of the ANTLRStringStream class. Note: This doesn't change the
-    // token text-- but just relaxes the matching rules to match
-    // in upper case. [Logic borrowed from Hive code.]
-    //
-    // Also see discussion on this topic in:
-    // http://www.antlr.org/wiki/pages/viewpage.action?pageId=1782.
-    public static class ANTLRNoCaseStringStream  extends ANTLRStringStream
-    {
-        public ANTLRNoCaseStringStream(String input)
-        {
-            super(input);
-        }
-
-        public int LA(int i)
-        {
-            int returnChar = super.LA(i);
-            if (returnChar == CharStream.EOF)
-            {
-                return returnChar;
-            }
-            else if (returnChar == 0)
-            {
-                return returnChar;
-            }
-
-            return Character.toUpperCase((char)returnChar);
-        }
-    }
-
-    public static Tree compileQuery(String query)
-    {
-        Tree queryTree;
-
-        try
-        {
-            ANTLRStringStream input = new ANTLRNoCaseStringStream(query);
-
-            CliLexer lexer = new CliLexer(input);
-            CommonTokenStream tokens = new CommonTokenStream(lexer);
-
-            CliParser parser = new CliParser(tokens);
-
-            // start parsing...
-            queryTree = (Tree)(parser.root().getTree());
-
-            // semantic analysis if any...
-            //  [tbd]
-
-        }
-        catch(Exception e)
-        {
-            // if there was an exception we don't want to process request any further
-            throw new RuntimeException(e.getMessage(), e);
-        }
-
-        return queryTree;
-    }
-    /*
-     * NODE_COLUMN_ACCESS related functions.
-     */
-
-    public static String getColumnFamily(Tree astNode, Iterable<CfDef> cfDefs)
-    {
-        return getColumnFamily(CliUtils.unescapeSQLString(astNode.getChild(0).getText()), cfDefs);
-    }
-
-    public static String getColumnFamily(String cfName, Iterable<CfDef> cfDefs)
-    {
-        int matches = 0;
-        String lastMatchedName = "";
-
-        for (CfDef cfDef : cfDefs)
-        {
-            if (cfDef.name.equals(cfName))
-            {
-                return cfName;
-            }
-            else if (cfDef.name.toUpperCase().equals(cfName.toUpperCase()))
-            {
-                lastMatchedName = cfDef.name;
-                matches++;
-            }
-        }
-
-        if (matches > 1 || matches == 0)
-            throw new RuntimeException(cfName + " not found in current keyspace.");
-
-        return lastMatchedName;
-    }
-
-    public static String getKeySpace(Tree statement, List<KsDef> keyspaces)
-    {
-        return getKeySpace(CliUtils.unescapeSQLString(statement.getChild(0).getText()), keyspaces);
-    }
-
-    public static String getKeySpace(String ksName, List<KsDef> keyspaces)
-    {
-        int matches = 0;
-        String lastMatchedName = "";
-
-        for (KsDef ksDef : keyspaces)
-        {
-            if (ksDef.name.equals(ksName))
-            {
-                return ksName;
-            }
-            else if (ksDef.name.toUpperCase().equals(ksName.toUpperCase()))
-            {
-                lastMatchedName = ksDef.name;
-                matches++;
-            }
-        }
-
-        if (matches > 1 || matches == 0)
-            throw new RuntimeException("Keyspace '" + ksName + "' not found.");
-
-        return lastMatchedName;
-    }
-
-    public static String getKey(Tree astNode)
-    {
-        return CliUtils.unescapeSQLString(astNode.getChild(1).getText());
-    }
-
-    public static int numColumnSpecifiers(Tree astNode)
-    {
-        // Skip over keyspace, column family and rowKey
-        return astNode.getChildCount() - 2;
-    }
-
-    // Returns the pos'th (0-based index) column specifier in the astNode
-    public static String getColumn(Tree astNode, int pos)
-    {
-        // Skip over keyspace, column family and rowKey
-        return CliUtils.unescapeSQLString(astNode.getChild(pos + 2).getText());
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/38027382/src/java/org/apache/cassandra/cli/CliCompleter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cli/CliCompleter.java b/src/java/org/apache/cassandra/cli/CliCompleter.java
deleted file mode 100644
index a95de54..0000000
--- a/src/java/org/apache/cassandra/cli/CliCompleter.java
+++ /dev/null
@@ -1,85 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.cassandra.cli;
-
-import jline.SimpleCompletor;
-
-public class CliCompleter extends SimpleCompletor
-{
-    private static final String[] commands = {
-            "connect",
-            "describe keyspace",
-            "exit",
-            "help",
-            "quit",
-            "show cluster name",
-            "show keyspaces",
-            "show schema",
-            "show api version",
-            "create keyspace",
-            "create column family",
-            "drop keyspace",
-            "drop column family",
-            "rename keyspace",
-            "rename column family",
-            "consistencylevel",
-
-            "help connect",
-            "help describe keyspace",
-            "help exit",
-            "help help",
-            "help quit",
-            "help show cluster name",
-            "help show keyspaces",
-            "help show schema",
-            "help show api version",
-            "help create keyspace",
-            "help create column family",
-            "help drop keyspace",
-            "help drop column family",
-            "help rename keyspace",
-            "help rename column family",
-            "help get",
-            "help set",
-            "help del",
-            "help count",
-            "help list",
-            "help truncate",
-            "help consistencylevel"
-    };
-    private static final String[] keyspaceCommands = {
-            "get",
-            "set",
-            "count",
-            "del",
-            "list",
-            "truncate",
-            "incr",
-            "decr"
-    };
-
-    public CliCompleter()
-    {
-        super(commands);
-    }
-
-    String[] getKeyspaceCommands()
-    {
-        return keyspaceCommands;
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/38027382/src/java/org/apache/cassandra/cli/CliMain.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cli/CliMain.java b/src/java/org/apache/cassandra/cli/CliMain.java
deleted file mode 100644
index 7cfc77b..0000000
--- a/src/java/org/apache/cassandra/cli/CliMain.java
+++ /dev/null
@@ -1,405 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.cassandra.cli;
-
-import java.io.BufferedReader;
-import java.io.File;
-import java.io.FileReader;
-import java.io.IOException;
-import java.nio.charset.CharacterCodingException;
-import java.util.*;
-
-import org.apache.cassandra.auth.IAuthenticator;
-import org.apache.cassandra.io.util.FileUtils;
-import org.apache.cassandra.thrift.*;
-import org.apache.cassandra.utils.FBUtilities;
-import org.apache.thrift.TException;
-import org.apache.thrift.protocol.TBinaryProtocol;
-import org.apache.thrift.transport.TTransport;
-import jline.ConsoleReader;
-import jline.History;
-
-/**
- * Cassandra Command Line Interface (CLI) Main
- */
-public class CliMain
-{
-    public final static String OLD_HISTORYFILE = ".cassandra.history";
-    public final static String HISTORYFILE = "cli.history";
-
-    private static TTransport transport = null;
-    private static Cassandra.Client thriftClient = null;
-    public  static final CliSessionState sessionState = new CliSessionState();
-    private static CliClient cliClient;
-    private static final CliCompleter completer = new CliCompleter();
-    private static int lineNumber = 1;
-
-    /**
-     * Establish a thrift connection to cassandra instance
-     *
-     * @param server - hostname or IP of the server
-     * @param port   - Thrift port number
-     */
-    public static void connect(String server, int port)
-    {
-        if (transport != null)
-            transport.close();
-
-        try
-        {
-            transport = sessionState.transportFactory.openTransport(server, port);
-        }
-        catch (Exception e)
-        {
-            e.printStackTrace(sessionState.err);
-
-            String error = (e.getCause() == null) ? e.getMessage() : e.getCause().getMessage();
-            throw new RuntimeException("Exception connecting to " + server + "/" + port + ". Reason: " + error + ".");
-        }
-
-        TBinaryProtocol binaryProtocol = new TBinaryProtocol(transport, true, true);
-        thriftClient = new Cassandra.Client(binaryProtocol);
-        cliClient = new CliClient(sessionState, thriftClient);
-
-        if ((sessionState.username != null) && (sessionState.password != null))
-        {
-            // Authenticate
-            Map<String, String> credentials = new HashMap<String, String>();
-            credentials.put(IAuthenticator.USERNAME_KEY, sessionState.username);
-            credentials.put(IAuthenticator.PASSWORD_KEY, sessionState.password);
-            AuthenticationRequest authRequest = new AuthenticationRequest(credentials);
-            try
-            {
-                thriftClient.login(authRequest);
-                cliClient.setUsername(sessionState.username);
-            }
-            catch (AuthenticationException e)
-            {
-                thriftClient = null;
-                sessionState.err.println("Exception during authentication to the cassandra node, " +
-                                         "Verify the keyspace exists, and that you are using the correct credentials.");
-                return;
-            }
-            catch (AuthorizationException e)
-            {
-                thriftClient = null;
-                sessionState.err.println("You are not authorized to use keyspace: " + sessionState.keyspace);
-                return;
-            }
-            catch (TException e)
-            {
-                thriftClient = null;
-                sessionState.err.println("Login failure. Did you specify 'keyspace', 'username' and 'password'?");
-                return;
-            }
-        }
-
-        if (sessionState.keyspace != null)
-        {
-            try
-            {
-                sessionState.keyspace = CliCompiler.getKeySpace(sessionState.keyspace, thriftClient.describe_keyspaces());;
-                thriftClient.set_keyspace(sessionState.keyspace);
-                cliClient.setKeySpace(sessionState.keyspace);
-                updateCompletor(CliUtils.getCfNamesByKeySpace(cliClient.getKSMetaData(sessionState.keyspace)));
-            }
-            catch (InvalidRequestException | NotFoundException e)
-            {
-                sessionState.err.println("Keyspace " + sessionState.keyspace + " not found");
-                return;
-            }
-            catch (TException e)
-            {
-                sessionState.err.println("Did you specify 'keyspace'?");
-                return;
-            }
-        }
-
-        // Lookup the cluster name, this is to make it clear which cluster the user is connected to
-        String clusterName;
-
-        try
-        {
-            clusterName = thriftClient.describe_cluster_name();
-        }
-        catch (Exception e)
-        {
-            sessionState.err.println("Exception retrieving information about the cassandra node, check you have connected to the thrift port.");
-
-            e.printStackTrace(sessionState.err);
-
-            return;
-        }
-
-        sessionState.out.printf("Connected to: \"%s\" on %s/%d%n", clusterName, server, port);
-    }
-
-    /**
-     * Disconnect thrift connection to cassandra instance
-     */
-    public static void disconnect()
-    {
-        if (transport != null)
-        {
-            transport.close();
-            transport = null;
-        }
-    }
-
-    /**
-     * Checks whether the thrift client is connected.
-     * @return boolean - true when connected, false otherwise
-     */
-    public static boolean isConnected()
-    {
-        if (thriftClient == null)
-        {
-            sessionState.out.println("Not connected to a cassandra instance.");
-            return false;
-        }
-        return true;
-    }
-
-    public static void updateCompletor(Set<String> candidates)
-    {
-        Set<String> actions = new HashSet<String>();
-        for (String cf : candidates)
-        {
-            for (String cmd : completer.getKeyspaceCommands())
-                actions.add(String.format("%s %s", cmd, cf));
-        }
-
-        String[] strs = Arrays.copyOf(actions.toArray(), actions.toArray().length, String[].class);
-
-        completer.setCandidateStrings(strs);
-    }
-
-    public static void processStatement(String query) throws CharacterCodingException, TException, NotFoundException, InvalidRequestException, NoSuchFieldException, UnavailableException, IllegalAccessException, InstantiationException
-    {
-        cliClient.executeCLIStatement(query);
-    }
-
-    public static void processStatementInteractive(String query)
-    {
-        try
-        {
-            cliClient.executeCLIStatement(query);
-        }
-        catch (Exception e)
-        {
-            String errorTemplate = sessionState.inFileMode() ? "Line " + lineNumber + " => " : "";
-
-            Throwable exception = (e.getCause() == null) ? e : e.getCause();
-            String message = (exception instanceof InvalidRequestException) ? ((InvalidRequestException) exception).getWhy() : e.getMessage();
-
-            sessionState.err.println(errorTemplate + message);
-
-            if (sessionState.debug || !(e instanceof RuntimeException))
-                e.printStackTrace(sessionState.err);
-
-            if (sessionState.batch || sessionState.inFileMode())
-            {
-                System.exit(4);
-            }
-        }
-        finally
-        {
-            lineNumber++;
-        }
-    }
-
-    public static void main(String args[]) throws IOException
-    {
-        // process command line arguments
-        CliOptions cliOptions = new CliOptions();
-        cliOptions.processArgs(sessionState, args);
-
-        // connect to cassandra server if host argument specified.
-        if (sessionState.hostName != null)
-        {
-            try
-            {
-                connect(sessionState.hostName, sessionState.thriftPort);
-            }
-            catch (RuntimeException e)
-            {
-                sessionState.err.println(e.getMessage());
-            }
-        }
-
-        if ( cliClient == null )
-        {
-            // Connection parameter was either invalid or not present.
-            // User must connect explicitly using the "connect" CLI statement.
-            cliClient = new CliClient(sessionState, null);
-        }
-
-        // load statements from file and process them
-        if (sessionState.inFileMode())
-        {
-            BufferedReader reader = null;
-
-            try
-            {
-                reader = new BufferedReader(new FileReader(sessionState.filename));
-                evaluateFileStatements(reader);
-            }
-            catch (IOException e)
-            {
-                sessionState.err.println(e.getMessage());
-                System.exit(1);
-            }
-            finally
-            {
-                FileUtils.closeQuietly(reader);
-            }      
-
-            return;
-        }
-
-        ConsoleReader reader = new ConsoleReader();
-
-        if (!sessionState.batch)
-        {
-            reader.addCompletor(completer);
-            reader.setBellEnabled(false);
-            File historyFile = handleHistoryFiles();
-
-            try
-            {
-                History history = new History(historyFile);
-                reader.setHistory(history);
-            }
-            catch (IOException exp)
-            {
-                sessionState.err.printf("Unable to open %s for writing", historyFile.getAbsolutePath());
-            }
-        }
-        else if (!sessionState.verbose) // if in batch mode but no verbose flag
-        {
-            sessionState.out.close();
-        }
-
-        cliClient.printBanner();
-
-        String prompt;
-        String line = "";
-        String currentStatement = "";
-        boolean inCompoundStatement = false;
-
-        while (line != null)
-        {
-            prompt = (inCompoundStatement) ? "...\t" : getPrompt(cliClient);
-
-            try
-            {
-                line = reader.readLine(prompt);
-            }
-            catch (IOException e)
-            {
-                // retry on I/O Exception
-            }
-
-            if (line == null)
-                return;
-
-            line = line.trim();
-
-            // skipping empty and comment lines
-            if (line.isEmpty() || line.startsWith("--"))
-                continue;
-
-            currentStatement += line;
-
-            if (line.endsWith(";") || line.equals("?"))
-            {
-                processStatementInteractive(currentStatement);
-                currentStatement = "";
-                inCompoundStatement = false;
-            }
-            else
-            {
-                currentStatement += " "; // ready for new line
-                inCompoundStatement = true;
-            }
-        }
-    }
-
-    private static File handleHistoryFiles()
-    {
-        File outputDir = FBUtilities.getToolsOutputDirectory();
-        File historyFile = new File(outputDir, HISTORYFILE);
-        File oldHistoryFile = new File(System.getProperty("user.home"), OLD_HISTORYFILE);
-        if(oldHistoryFile.exists())
-            FileUtils.renameWithConfirm(oldHistoryFile, historyFile);
-
-        return historyFile;
-    }
-
-    private static void evaluateFileStatements(BufferedReader reader) throws IOException
-    {
-        String line;
-        String currentStatement = "";
-
-        boolean commentedBlock = false;
-
-        while ((line = reader.readLine()) != null)
-        {
-            line = line.trim();
-
-            // skipping empty and comment lines
-            if (line.isEmpty() || line.startsWith("--"))
-                continue;
-
-            if (line.startsWith("/*"))
-                commentedBlock = true;
-
-            if (line.startsWith("*/") || line.endsWith("*/"))
-            {
-                commentedBlock = false;
-                continue;
-            }
-
-            if (commentedBlock) // skip commented lines
-                continue;
-
-            currentStatement += line;
-
-            if (line.endsWith(";"))
-            {
-                processStatementInteractive(currentStatement);
-                currentStatement = "";
-            }
-            else
-            {
-                currentStatement += " "; // ready for new line
-            }
-        }
-    }
-
-    /**
-     * Returns prompt for current connection
-     * @param client - currently connected client
-     * @return String - prompt with username and keyspace (if any)
-     */
-    private static String getPrompt(CliClient client)
-    {
-        return "[" + client.getUsername() + "@" + client.getKeySpace() + "] ";
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/38027382/src/java/org/apache/cassandra/cli/CliOptions.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cli/CliOptions.java b/src/java/org/apache/cassandra/cli/CliOptions.java
deleted file mode 100644
index 7894bf9..0000000
--- a/src/java/org/apache/cassandra/cli/CliOptions.java
+++ /dev/null
@@ -1,319 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.cassandra.cli;
-
-import java.util.HashMap;
-import java.util.Map;
-
-import com.google.common.base.Joiner;
-import org.apache.commons.cli.*;
-
-import org.apache.cassandra.config.EncryptionOptions;
-import org.apache.cassandra.thrift.ITransportFactory;
-import org.apache.cassandra.thrift.SSLTransportFactory;
-
-/**
- *
- * Used to process, and act upon the arguments passed to the CLI.
- *
- */
-public class CliOptions
-{
-    private static final CLIOptions options; // Info about command line options
-
-    // Name of the command line tool (used for error messages)
-    private static final String TOOL_NAME = "cassandra-cli";
-
-    // Command line options
-    private static final String HOST_OPTION = "host";
-    private static final String PORT_OPTION = "port";
-    private static final String TRANSPORT_FACTORY = "transport-factory";
-    private static final String DEBUG_OPTION = "debug";
-    private static final String USERNAME_OPTION = "username";
-    private static final String PASSWORD_OPTION = "password";
-    private static final String KEYSPACE_OPTION = "keyspace";
-    private static final String BATCH_OPTION = "batch";
-    private static final String HELP_OPTION = "help";
-    private static final String FILE_OPTION = "file";
-    private static final String JMX_PORT_OPTION = "jmxport";
-    private static final String JMX_USERNAME_OPTION = "jmxusername";
-    private static final String JMX_PASSWORD_OPTION = "jmxpassword";
-    private static final String VERBOSE_OPTION  = "verbose";
-
-    private static final String SSL_TRUSTSTORE = "truststore";
-    private static final String SSL_TRUSTSTORE_PW = "truststore-password";
-    private static final String SSL_PROTOCOL = "ssl-protocol";
-    private static final String SSL_ALGORITHM = "ssl-alg";
-    private static final String SSL_STORE_TYPE = "store-type";
-    private static final String SSL_CIPHER_SUITES = "ssl-ciphers";
-
-    // Default values for optional command line arguments
-    private static final String DEFAULT_HOST        = "127.0.0.1";
-    private static final int    DEFAULT_THRIFT_PORT = 9160;
-
-    // Register the command line options and their properties (such as
-    // whether they take an extra argument, etc.
-    static
-    {
-        options = new CLIOptions();
-
-        options.addOption("h",  HOST_OPTION,     "HOSTNAME", "cassandra server's host name");
-        options.addOption("p",  PORT_OPTION,     "PORT",     "cassandra server's thrift port");
-        options.addOption("u",  USERNAME_OPTION, "USERNAME", "user name for cassandra authentication");
-        options.addOption("pw", PASSWORD_OPTION, "PASSWORD", "password for cassandra authentication");
-        options.addOption("k",  KEYSPACE_OPTION, "KEYSPACE", "cassandra keyspace user is authenticated against");
-        options.addOption("f",  FILE_OPTION,     "FILENAME", "load statements from the specific file");
-        options.addOption(null, JMX_PORT_OPTION, "JMX-PORT", "JMX service port");
-        options.addOption(null, JMX_USERNAME_OPTION, "JMX-USERNAME", "JMX service username");
-        options.addOption(null, JMX_PASSWORD_OPTION, "JMX-PASSWORD", "JMX service password");
-        options.addOption("tf", TRANSPORT_FACTORY, "TRANSPORT-FACTORY", "Fully-qualified ITransportFactory class name for creating a connection to cassandra");
-
-        // ssl connection-related options
-        options.addOption("ts", SSL_TRUSTSTORE, "TRUSTSTORE", "SSL: full path to truststore");
-        options.addOption("tspw", SSL_TRUSTSTORE_PW, "TRUSTSTORE-PASSWORD", "SSL: password of the truststore");
-        options.addOption("prtcl", SSL_PROTOCOL, "PROTOCOL", "SSL: connections protocol to use (default: TLS)");
-        options.addOption("alg", SSL_ALGORITHM, "ALGORITHM", "SSL: algorithm (default: SunX509)");
-        options.addOption("st", SSL_STORE_TYPE, "STORE-TYPE", "SSL: type of store");
-        options.addOption("ciphers", SSL_CIPHER_SUITES, "CIPHER-SUITES", "SSL: comma-separated list of encryption suites to use");
-
-        // options without argument
-        options.addOption("B",  BATCH_OPTION,   "enabled batch mode (suppress output; errors are fatal)");
-        options.addOption(null, DEBUG_OPTION,   "display stack-traces (NOTE: We print strack-traces in the places where it makes sense even without --debug)");
-        options.addOption("?",  HELP_OPTION,    "usage help");
-        options.addOption("v",  VERBOSE_OPTION, "verbose output when using batch mode");
-    }
-
-    private static void printUsage()
-    {
-        new HelpFormatter().printHelp(TOOL_NAME, options);
-    }
-
-    public void processArgs(CliSessionState css, String[] args)
-    {
-        CommandLineParser parser = new GnuParser();
-
-        try
-        {
-            CommandLine cmd = parser.parse(options, args, false);
-
-            if (cmd.hasOption(HOST_OPTION))
-            {
-                css.hostName = cmd.getOptionValue(HOST_OPTION);
-            }
-            else
-            {
-                css.hostName = DEFAULT_HOST;
-            }
-
-            if (cmd.hasOption(DEBUG_OPTION))
-            {
-                css.debug = true;
-            }
-
-            // Look for optional args.
-            if (cmd.hasOption(PORT_OPTION))
-            {
-                css.thriftPort = Integer.parseInt(cmd.getOptionValue(PORT_OPTION));
-            }
-            else
-            {
-                css.thriftPort = DEFAULT_THRIFT_PORT;
-            }
-
-            // Look for authentication credentials (username and password)
-            if (cmd.hasOption(USERNAME_OPTION))
-            {
-                css.username = cmd.getOptionValue(USERNAME_OPTION);
-            }
-
-            if (cmd.hasOption(PASSWORD_OPTION))
-            {
-                css.password = cmd.getOptionValue(PASSWORD_OPTION);
-            }
-
-            // Look for keyspace
-            if (cmd.hasOption(KEYSPACE_OPTION))
-            {
-                css.keyspace = cmd.getOptionValue(KEYSPACE_OPTION);
-            }
-
-            if (cmd.hasOption(BATCH_OPTION))
-            {
-                css.batch = true;
-            }
-
-            if (cmd.hasOption(FILE_OPTION))
-            {
-                css.filename = cmd.getOptionValue(FILE_OPTION);
-            }
-
-            if (cmd.hasOption(JMX_PORT_OPTION))
-            {
-                css.jmxPort = Integer.parseInt(cmd.getOptionValue(JMX_PORT_OPTION));
-            }
-
-            if (cmd.hasOption(JMX_USERNAME_OPTION))
-            {
-                css.jmxUsername = cmd.getOptionValue(JMX_USERNAME_OPTION);
-            }
-
-            if (cmd.hasOption(JMX_PASSWORD_OPTION))
-            {
-                css.jmxPassword = cmd.getOptionValue(JMX_PASSWORD_OPTION);
-            }
-
-            if (cmd.hasOption(HELP_OPTION))
-            {
-                printUsage();
-                System.exit(1);
-            }
-
-            if (cmd.hasOption(VERBOSE_OPTION))
-            {
-                css.verbose = true;
-            }
-
-            if(cmd.hasOption(SSL_TRUSTSTORE))
-            {
-                css.encOptions.truststore = cmd.getOptionValue(SSL_TRUSTSTORE);
-            }
-
-            if(cmd.hasOption(SSL_TRUSTSTORE_PW))
-            {
-                css.encOptions.truststore_password = cmd.getOptionValue(SSL_TRUSTSTORE_PW);
-            }
-
-            if(cmd.hasOption(SSL_PROTOCOL))
-            {
-                css.encOptions.protocol = cmd.getOptionValue(SSL_PROTOCOL);
-            }
-
-            if(cmd.hasOption(SSL_ALGORITHM))
-            {
-                css.encOptions.algorithm = cmd.getOptionValue(SSL_ALGORITHM);
-            }
-
-            if(cmd.hasOption(SSL_STORE_TYPE))
-            {
-                css.encOptions.store_type = cmd.getOptionValue(SSL_STORE_TYPE);
-            }
-
-            if(cmd.hasOption(SSL_CIPHER_SUITES))
-            {
-                css.encOptions.cipher_suites = cmd.getOptionValue(SSL_CIPHER_SUITES).split(",");
-            }
-
-            if (cmd.hasOption(TRANSPORT_FACTORY))
-            {
-                css.transportFactory = validateAndSetTransportFactory(cmd.getOptionValue(TRANSPORT_FACTORY));
-                configureTransportFactory(css.transportFactory, css.encOptions);
-            }
-
-            // Abort if there are any unrecognized arguments left
-            if (cmd.getArgs().length > 0)
-            {
-                System.err.printf("Unknown argument: %s%n", cmd.getArgs()[0]);
-                System.err.println();
-                printUsage();
-                System.exit(1);
-            }
-        }
-        catch (ParseException e)
-        {
-            System.err.println(e.getMessage());
-            System.err.println();
-            printUsage();
-            System.exit(1);
-        }
-    }
-
-    private static class CLIOptions extends Options
-    {
-        /**
-         * Add option with argument and argument name
-         * @param opt shortcut for option name
-         * @param longOpt complete option name
-         * @param argName argument name
-         * @param description description of the option
-         * @return updated Options object
-         */
-        public Options addOption(String opt, String longOpt, String argName, String description)
-        {
-            Option option = new Option(opt, longOpt, true, description);
-            option.setArgName(argName);
-
-            return addOption(option);
-        }
-
-        /**
-         * Add option without argument
-         * @param opt shortcut for option name
-         * @param longOpt complete option name
-         * @param description description of the option
-         * @return updated Options object
-         */
-        public Options addOption(String opt, String longOpt, String description)
-        {
-            return addOption(new Option(opt, longOpt, false, description));
-        }
-    }
-
-    private static ITransportFactory validateAndSetTransportFactory(String transportFactory)
-    {
-        try
-        {
-            Class<?> factory = Class.forName(transportFactory);
-            if (!ITransportFactory.class.isAssignableFrom(factory))
-                throw new IllegalArgumentException(String.format("transport factory '%s' " +
-                                                                 "not derived from ITransportFactory", transportFactory));
-            return (ITransportFactory) factory.newInstance();
-        }
-        catch (Exception e)
-        {
-            throw new IllegalArgumentException(String.format("Cannot create a transport factory '%s'.", transportFactory), e);
-        }
-    }
-
-    private static void configureTransportFactory(ITransportFactory transportFactory, EncryptionOptions encOptions)
-    {
-        Map<String, String> options = new HashMap<>();
-        // If the supplied factory supports the same set of options as our SSL impl, set those
-        if (transportFactory.supportedOptions().contains(SSLTransportFactory.TRUSTSTORE))
-            options.put(SSLTransportFactory.TRUSTSTORE, encOptions.truststore);
-        if (transportFactory.supportedOptions().contains(SSLTransportFactory.TRUSTSTORE_PASSWORD))
-            options.put(SSLTransportFactory.TRUSTSTORE_PASSWORD, encOptions.truststore_password);
-        if (transportFactory.supportedOptions().contains(SSLTransportFactory.PROTOCOL))
-            options.put(SSLTransportFactory.PROTOCOL, encOptions.protocol);
-        if (transportFactory.supportedOptions().contains(SSLTransportFactory.CIPHER_SUITES))
-            options.put(SSLTransportFactory.CIPHER_SUITES, Joiner.on(',').join(encOptions.cipher_suites));
-
-        if (transportFactory.supportedOptions().contains(SSLTransportFactory.KEYSTORE)
-                && encOptions.require_client_auth)
-            options.put(SSLTransportFactory.KEYSTORE, encOptions.keystore);
-        if (transportFactory.supportedOptions().contains(SSLTransportFactory.KEYSTORE_PASSWORD)
-                && encOptions.require_client_auth)
-            options.put(SSLTransportFactory.KEYSTORE_PASSWORD, encOptions.keystore_password);
-
-        // Now check if any of the factory's supported options are set as system properties
-        for (String optionKey : transportFactory.supportedOptions())
-            if (System.getProperty(optionKey) != null)
-                options.put(optionKey, System.getProperty(optionKey));
-
-        transportFactory.setOptions(options);
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/38027382/src/java/org/apache/cassandra/cli/CliSessionState.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cli/CliSessionState.java b/src/java/org/apache/cassandra/cli/CliSessionState.java
deleted file mode 100644
index f0de713..0000000
--- a/src/java/org/apache/cassandra/cli/CliSessionState.java
+++ /dev/null
@@ -1,94 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.cassandra.cli;
-
-import java.io.InputStream;
-import java.io.PrintStream;
-
-import org.apache.cassandra.config.EncryptionOptions;
-import org.apache.cassandra.config.EncryptionOptions.ClientEncryptionOptions;
-import org.apache.cassandra.thrift.ITransportFactory;
-import org.apache.cassandra.thrift.TFramedTransportFactory;
-import org.apache.cassandra.tools.NodeProbe;
-
-/**
- * Used to hold the state for the CLI.
- */
-public class CliSessionState
-{
-
-    public String  hostName;      // cassandra server name
-    public int     thriftPort;    // cassandra server's thrift port
-    public boolean debug = false; // print stack traces when errors occur in the CLI
-    public String  username;      // cassandra login name (if password-based authenticator is used)
-    public String  password;      // cassandra login password (if password-based authenticator is used)
-    public String  keyspace;      // cassandra keyspace user is authenticating
-    public boolean batch = false; // enable/disable batch processing mode
-    public String  filename = ""; // file to read commands from
-    public int     jmxPort = 7199;// JMX service port
-    public String  jmxUsername;   // JMX service username
-    public String  jmxPassword;   // JMX service password
-    public boolean verbose = false; // verbose output
-    public ITransportFactory transportFactory = new TFramedTransportFactory();
-    public EncryptionOptions encOptions = new ClientEncryptionOptions();
-
-    /*
-     * Streams to read/write from
-     */
-    public InputStream in;
-    public PrintStream out;
-    public PrintStream err;
-
-    public CliSessionState()
-    {
-        in = System.in;
-        out = System.out;
-        err = System.err;
-    }
-
-    public void setOut(PrintStream newOut)
-    {
-        this.out = newOut;
-    }
-
-    public void setErr(PrintStream newErr)
-    {
-        this.err = newErr;
-    }
-
-    public boolean inFileMode()
-    {
-        return !this.filename.isEmpty();
-    }
-
-    public NodeProbe getNodeProbe()
-    {
-        try
-        {
-            return jmxUsername != null && jmxPassword != null
-                   ? new NodeProbe(hostName, jmxPort, jmxUsername, jmxPassword)
-                   : new NodeProbe(hostName, jmxPort);
-        }
-        catch (Exception e)
-        {
-            err.printf("WARNING: Could not connect to the JMX on %s:%d - some information won't be shown.%n%n", hostName, jmxPort);
-        }
-
-        return null;
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/38027382/src/java/org/apache/cassandra/cli/CliUserHelp.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cli/CliUserHelp.java b/src/java/org/apache/cassandra/cli/CliUserHelp.java
deleted file mode 100644
index aa4192c..0000000
--- a/src/java/org/apache/cassandra/cli/CliUserHelp.java
+++ /dev/null
@@ -1,29 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.cassandra.cli;
-
-import java.util.List;
-
-public class CliUserHelp
-{
-    public String banner;
-
-    public String help;
-
-    public List<CliCommandHelp> commands;
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/38027382/src/java/org/apache/cassandra/cli/CliUtils.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cli/CliUtils.java b/src/java/org/apache/cassandra/cli/CliUtils.java
deleted file mode 100644
index d0c8e37..0000000
--- a/src/java/org/apache/cassandra/cli/CliUtils.java
+++ /dev/null
@@ -1,128 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.cassandra.cli;
-
-import java.util.LinkedHashSet;
-import java.util.List;
-import java.util.Set;
-
-import org.apache.cassandra.thrift.CfDef;
-import org.apache.cassandra.thrift.IndexOperator;
-import org.apache.cassandra.thrift.KsDef;
-import org.apache.commons.lang3.StringEscapeUtils;
-
-public class CliUtils
-{
-    /**
-     * Strips leading and trailing "'" characters, and handles
-     * and escaped characters such as \n, \r, etc.
-     * @param b - string to unescape
-     * @return String - unexspaced string
-     */
-    public static String unescapeSQLString(String b)
-    {
-        if (b.charAt(0) == '\'' && b.charAt(b.length()-1) == '\'')
-            b = b.substring(1, b.length()-1);
-        return StringEscapeUtils.unescapeJava(b);
-    }
-
-    public static String escapeSQLString(String b)
-    {
-        // single quotes are not escaped in java, need to be for cli
-        return StringEscapeUtils.escapeJava(b).replace("\'", "\\'");
-    }
-
-    public static String maybeEscapeName(String name)
-    {
-        return Character.isLetter(name.charAt(0)) ? name : "\'" + name + "\'";
-    }
-
-    /**
-     * Returns IndexOperator from string representation
-     * @param operator - string representing IndexOperator (=, >=, >, <, <=)
-     * @return IndexOperator - enum value of IndexOperator or null if not found
-     */
-    public static IndexOperator getIndexOperator(String operator)
-    {
-        if (operator.equals("="))
-        {
-            return IndexOperator.EQ;
-        }
-        else if (operator.equals(">="))
-        {
-            return IndexOperator.GTE;
-        }
-        else if (operator.equals(">"))
-        {
-            return IndexOperator.GT;
-        }
-        else if (operator.equals("<"))
-        {
-            return IndexOperator.LT;
-        }
-        else if (operator.equals("<="))
-        {
-            return IndexOperator.LTE;
-        }
-
-        return null;
-    }
-
-    /**
-     * Returns set of column family names in specified keySpace.
-     * @param keySpace - keyspace definition to get column family names from.
-     * @return Set - column family names
-     */
-    public static Set<String> getCfNamesByKeySpace(KsDef keySpace)
-    {
-        Set<String> names = new LinkedHashSet<String>();
-
-        for (CfDef cfDef : keySpace.getCf_defs())
-        {
-            names.add(cfDef.getName());
-        }
-
-        return names;
-    }
-
-    /**
-     * Parse the statement from cli and return KsDef
-     *
-     * @param keyspaceName - name of the keyspace to lookup
-     * @param keyspaces - List of known keyspaces
-     *
-     * @return metadata about keyspace or null
-     */
-    public static KsDef getKeySpaceDef(String keyspaceName, List<KsDef> keyspaces)
-    {
-        keyspaceName = keyspaceName.toUpperCase();
-
-        for (KsDef ksDef : keyspaces)
-        {
-            if (ksDef.name.toUpperCase().equals(keyspaceName))
-                return ksDef;
-        }
-
-        return null;
-    }
-
-    public static String quote(String str)
-    {
-        return String.format("'%s'", str);
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/38027382/src/java/org/apache/cassandra/cli/transport/FramedTransportFactory.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cli/transport/FramedTransportFactory.java b/src/java/org/apache/cassandra/cli/transport/FramedTransportFactory.java
deleted file mode 100644
index dc7ef81..0000000
--- a/src/java/org/apache/cassandra/cli/transport/FramedTransportFactory.java
+++ /dev/null
@@ -1,32 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.cassandra.cli.transport;
-
-import org.apache.thrift.transport.TFramedTransport;
-import org.apache.thrift.transport.TTransport;
-import org.apache.thrift.transport.TTransportFactory;
-
-public class FramedTransportFactory extends TTransportFactory
-{
-    public static final int DEFAULT_MAX_FRAME_SIZE = 15 * 1024 * 1024; // 15 MiB
-
-    public TTransport getTransport(TTransport base)
-    {
-        return new TFramedTransport(base, DEFAULT_MAX_FRAME_SIZE);
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/38027382/src/java/org/apache/cassandra/cli/transport/SSLTransportFactory.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cli/transport/SSLTransportFactory.java b/src/java/org/apache/cassandra/cli/transport/SSLTransportFactory.java
deleted file mode 100644
index 4aa9fc1..0000000
--- a/src/java/org/apache/cassandra/cli/transport/SSLTransportFactory.java
+++ /dev/null
@@ -1,47 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.cassandra.cli.transport;
-
-import org.apache.cassandra.cli.CliMain;
-import org.apache.cassandra.cli.CliSessionState;
-import org.apache.thrift.transport.TSSLTransportFactory;
-import org.apache.thrift.transport.TSSLTransportFactory.TSSLTransportParameters;
-import org.apache.thrift.transport.TTransport;
-import org.apache.thrift.transport.TTransportException;
-import org.apache.thrift.transport.TTransportFactory;
-
-public class SSLTransportFactory extends TTransportFactory
-{
-    private static final int SOCKET_TIMEOUT = 0;
-
-    public TTransport getTransport(TTransport trans)
-    {
-        final CliSessionState sessionState = CliMain.sessionState;
-        try
-        {
-            TSSLTransportParameters params = new TSSLTransportParameters(sessionState.encOptions.protocol, sessionState.encOptions.cipher_suites);
-            params.setTrustStore(sessionState.encOptions.truststore, sessionState.encOptions.truststore_password);
-            trans = TSSLTransportFactory.getClientSocket(sessionState.hostName, sessionState.thriftPort, SOCKET_TIMEOUT, params);
-            return new FramedTransportFactory().getTransport(trans);
-        }
-        catch (TTransportException e)
-        {
-            throw new RuntimeException("Failed to create a client SSL connection.", e);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/38027382/src/java/org/apache/cassandra/db/DefsTables.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/DefsTables.java b/src/java/org/apache/cassandra/db/DefsTables.java
index f1fe0bf..14c5241 100644
--- a/src/java/org/apache/cassandra/db/DefsTables.java
+++ b/src/java/org/apache/cassandra/db/DefsTables.java
@@ -48,68 +48,6 @@ import org.apache.cassandra.utils.ByteBufferUtil;
  * SCHEMA_{KEYSPACES, COLUMNFAMILIES, COLUMNS}_CF are used to store Keyspace/ColumnFamily attributes to make schema
  * load/distribution easy, it replaces old mechanism when local migrations where serialized, stored in system.Migrations
  * and used for schema distribution.
- *
- * SCHEMA_KEYSPACES_CF layout:
- *
- * <key (AsciiType)>
- *   ascii => json_serialized_value
- *   ...
- * </key>
- *
- * Where <key> is a name of keyspace e.g. "ks".
- *
- * SCHEMA_COLUMNFAMILIES_CF layout:
- *
- * <key (AsciiType)>
- *     composite(ascii, ascii) => json_serialized_value
- * </key>
- *
- * Where <key> is a name of keyspace e.g. "ks"., first component of the column name is name of the ColumnFamily, last
- * component is the name of the ColumnFamily attribute.
- *
- * SCHEMA_COLUMNS_CF layout:
- *
- * <key (AsciiType)>
- *     composite(ascii, ascii, ascii) => json_serialized value
- * </key>
- *
- * Where <key> is a name of keyspace e.g. "ks".
- *
- * Cell names where made composite to support 3-level nesting which represents following structure:
- * "ColumnFamily name":"column name":"column attribute" => "value"
- *
- * Example of schema (using CLI):
- *
- * schema_keyspaces
- * ----------------
- * RowKey: ks
- *  => (column=durable_writes, value=true, timestamp=1327061028312185000)
- *  => (column=name, value="ks", timestamp=1327061028312185000)
- *  => (column=replication_factor, value=0, timestamp=1327061028312185000)
- *  => (column=strategy_class, value="org.apache.cassandra.locator.NetworkTopologyStrategy", timestamp=1327061028312185000)
- *  => (column=strategy_options, value={"datacenter1":"1"}, timestamp=1327061028312185000)
- *
- * schema_columnfamilies
- * ---------------------
- * RowKey: ks
- *  => (column=cf:bloom_filter_fp_chance, value=0.0, timestamp=1327061105833119000)
- *  => (column=cf:caching, value="NONE", timestamp=1327061105833119000)
- *  => (column=cf:column_type, value="Standard", timestamp=1327061105833119000)
- *  => (column=cf:comment, value="ColumnFamily", timestamp=1327061105833119000)
- *  => (column=cf:default_validation_class, value="org.apache.cassandra.db.marshal.BytesType", timestamp=1327061105833119000)
- *  => (column=cf:gc_grace_seconds, value=864000, timestamp=1327061105833119000)
- *  => (column=cf:id, value=1000, timestamp=1327061105833119000)
- *  => (column=cf:key_alias, value="S0VZ", timestamp=1327061105833119000)
- *  ... part of the output omitted.
- *
- * schema_columns
- * --------------
- * RowKey: ks
- *  => (column=cf:c:index_name, value=null, timestamp=1327061105833119000)
- *  => (column=cf:c:index_options, value=null, timestamp=1327061105833119000)
- *  => (column=cf:c:index_type, value=null, timestamp=1327061105833119000)
- *  => (column=cf:c:name, value="aGVsbG8=", timestamp=1327061105833119000)
- *  => (column=cf:c:validation_class, value="org.apache.cassandra.db.marshal.AsciiType", timestamp=1327061105833119000)
  */
 public class DefsTables
 {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/38027382/src/java/org/apache/cassandra/thrift/SSLTransportFactory.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/thrift/SSLTransportFactory.java b/src/java/org/apache/cassandra/thrift/SSLTransportFactory.java
index b23d28c..497cd5c 100644
--- a/src/java/org/apache/cassandra/thrift/SSLTransportFactory.java
+++ b/src/java/org/apache/cassandra/thrift/SSLTransportFactory.java
@@ -18,7 +18,8 @@
 package org.apache.cassandra.thrift;
 
 import com.google.common.collect.Sets;
-import org.apache.cassandra.cli.transport.FramedTransportFactory;
+
+import org.apache.thrift.transport.TFramedTransport;
 import org.apache.thrift.transport.TSSLTransportFactory;
 import org.apache.thrift.transport.TTransport;
 
@@ -27,6 +28,8 @@ import java.util.Set;
 
 public class SSLTransportFactory implements ITransportFactory
 {
+    public static final int DEFAULT_MAX_FRAME_SIZE = 15 * 1024 * 1024; // 15 MiB
+
     public static final String TRUSTSTORE = "enc.truststore";
     public static final String TRUSTSTORE_PASSWORD = "enc.truststore.password";
     public static final String KEYSTORE = "enc.keystore";
@@ -57,7 +60,7 @@ public class SSLTransportFactory implements ITransportFactory
         if (null != keystore)
             params.setKeyStore(keystore, keystorePassword);
         TTransport trans = TSSLTransportFactory.getClientSocket(host, port, SOCKET_TIMEOUT, params);
-        return new FramedTransportFactory().getTransport(trans);
+        return new TFramedTransport(trans, DEFAULT_MAX_FRAME_SIZE);
     }
 
     @Override