You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by jb...@apache.org on 2011/04/14 20:27:35 UTC

svn commit: r1092430 - in /cassandra/branches/cassandra-0.8: ./ src/java/org/apache/cassandra/cli/ src/resources/ src/resources/org/ src/resources/org/apache/ src/resources/org/apache/cassandra/ src/resources/org/apache/cassandra/cli/

Author: jbellis
Date: Thu Apr 14 18:27:34 2011
New Revision: 1092430

URL: http://svn.apache.org/viewvc?rev=1092430&view=rev
Log:
move cli help to yaml file
patch by Aaron Morton; reviewed by jbellis for CASSANDRA-2008

Added:
    cassandra/branches/cassandra-0.8/src/java/org/apache/cassandra/cli/CliCommandHelp.java
    cassandra/branches/cassandra-0.8/src/resources/
    cassandra/branches/cassandra-0.8/src/resources/org/
    cassandra/branches/cassandra-0.8/src/resources/org/apache/
    cassandra/branches/cassandra-0.8/src/resources/org/apache/cassandra/
    cassandra/branches/cassandra-0.8/src/resources/org/apache/cassandra/cli/
    cassandra/branches/cassandra-0.8/src/resources/org/apache/cassandra/cli/CliHelp.yaml
Modified:
    cassandra/branches/cassandra-0.8/build.xml
    cassandra/branches/cassandra-0.8/src/java/org/apache/cassandra/cli/Cli.g
    cassandra/branches/cassandra-0.8/src/java/org/apache/cassandra/cli/CliClient.java
    cassandra/branches/cassandra-0.8/src/java/org/apache/cassandra/cli/CliMain.java
    cassandra/branches/cassandra-0.8/src/java/org/apache/cassandra/cli/CliUserHelp.java

Modified: cassandra/branches/cassandra-0.8/build.xml
URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.8/build.xml?rev=1092430&r1=1092429&r2=1092430&view=diff
==============================================================================
--- cassandra/branches/cassandra-0.8/build.xml (original)
+++ cassandra/branches/cassandra-0.8/build.xml Thu Apr 14 18:27:34 2011
@@ -27,6 +27,7 @@
     <property name="basedir" value="."/>
     <property name="build.src" value="${basedir}/src"/>
     <property name="build.src.java" value="${basedir}/src/java"/>
+    <property name="build.src.resources" value="${basedir}/src/resources"/>
     <property name="build.src.driver" value="${basedir}/drivers/java/src" />
     <property name="avro.src" value="${basedir}/src/avro"/>
     <property name="build.src.gen-java" value="${basedir}/src/gen-java"/>
@@ -383,7 +384,9 @@
             <src path="${build.src.driver}" />
             <classpath refid="cassandra.classpath"/>
         </javac>
-
+        <copy todir="${build.classes.main}">
+            <fileset dir="${build.src.resources}" />
+        </copy>
         <taskdef name="paranamer" classname="com.thoughtworks.paranamer.ant.ParanamerGeneratorTask">
           <classpath refid="cassandra.classpath" />
         </taskdef>

Modified: cassandra/branches/cassandra-0.8/src/java/org/apache/cassandra/cli/Cli.g
URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.8/src/java/org/apache/cassandra/cli/Cli.g?rev=1092430&r1=1092429&r2=1092430&view=diff
==============================================================================
--- cassandra/branches/cassandra-0.8/src/java/org/apache/cassandra/cli/Cli.g (original)
+++ cassandra/branches/cassandra-0.8/src/java/org/apache/cassandra/cli/Cli.g Thu Apr 14 18:27:34 2011
@@ -119,7 +119,7 @@ package org.apache.cassandra.cli;
 
         if (e instanceof NoViableAltException)
         {
-            errorMessage = "Command not found: `" + this.input + "`. Type 'help' or '?' for help.";
+            errorMessage = "Command not found: `" + this.input + "`. Type 'help;' or '?' for help.";
         }
         else
         {

Modified: cassandra/branches/cassandra-0.8/src/java/org/apache/cassandra/cli/CliClient.java
URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.8/src/java/org/apache/cassandra/cli/CliClient.java?rev=1092430&r1=1092429&r2=1092430&view=diff
==============================================================================
--- cassandra/branches/cassandra-0.8/src/java/org/apache/cassandra/cli/CliClient.java (original)
+++ cassandra/branches/cassandra-0.8/src/java/org/apache/cassandra/cli/CliClient.java Thu Apr 14 18:27:34 2011
@@ -17,7 +17,9 @@
  */
 package org.apache.cassandra.cli;
 
+import java.io.IOError;
 import java.io.IOException;
+import java.io.InputStream;
 import java.net.InetAddress;
 import java.net.UnknownHostException;
 import java.nio.ByteBuffer;
@@ -43,9 +45,13 @@ import org.apache.cassandra.utils.FBUtil
 import org.apache.cassandra.utils.UUIDGen;
 import org.apache.thrift.TBaseHelper;
 import org.apache.thrift.TException;
+import org.yaml.snakeyaml.constructor.Constructor;
+import org.yaml.snakeyaml.Loader;
+import org.yaml.snakeyaml.TypeDescription;
+import org.yaml.snakeyaml.Yaml;
 
 // Cli Client Side Library
-public class CliClient extends CliUserHelp
+public class CliClient
 {
 
     /**
@@ -98,6 +104,34 @@ public class CliClient extends CliUserHe
         STRATEGY_OPTIONS
     }
 
+    /*
+        * 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,
+        ROWS_CACHED,
+        ROW_CACHE_SAVE_PERIOD,
+        KEYS_CACHED,
+        KEY_CACHE_SAVE_PERIOD,
+        READ_REPAIR_CHANCE,
+        GC_GRACE,
+        COLUMN_METADATA,
+        MEMTABLE_OPERATIONS,
+        MEMTABLE_THROUGHPUT,
+        MEMTABLE_FLUSH_AFTER,
+        DEFAULT_VALIDATION_CLASS,
+        MIN_COMPACTION_THRESHOLD,
+        MAX_COMPACTION_THRESHOLD,
+        REPLICATE_ON_WRITE,
+        ROW_CACHE_PROVIDER,
+        KEY_VALIDATION_CLASS
+    }
+
     private static final String DEFAULT_PLACEMENT_STRATEGY = "org.apache.cassandra.locator.NetworkTopologyStrategy";
 
     private Cassandra.Client thriftClient = null;
@@ -106,13 +140,45 @@ public class CliClient extends CliUserHe
     private String username = null;
     private Map<String, KsDef> keyspacesMap = new HashMap<String, KsDef>();
     private Map<String, AbstractType> cfKeysComparators;
-    private ConsistencyLevel consistencyLevel = ConsistencyLevel.ONE;   
- 
+    private ConsistencyLevel consistencyLevel = ConsistencyLevel.ONE;
+    private CliUserHelp help;
     public CliClient(CliSessionState cliSessionState, Cassandra.Client thriftClient)
     {
         this.sessionState = cliSessionState;
         this.thriftClient = thriftClient;
         this.cfKeysComparators = new HashMap<String, AbstractType>();
+        help = getHelp();
+    }
+
+        private CliUserHelp getHelp()
+    {
+        final InputStream is =  CliClient.class.getClassLoader().getResourceAsStream("org/apache/cassandra/cli/CliHelp.yaml");
+        assert is != null;
+
+        try
+        {
+            final Constructor constructor = new Constructor(CliUserHelp.class);
+            TypeDescription desc = new TypeDescription(CliUserHelp.class);
+            desc.putListPropertyType("commands", CliCommandHelp.class);
+            final Yaml yaml = new Yaml(new Loader(constructor));
+            return (CliUserHelp)yaml.load(is);
+        }
+        finally
+        {
+            try
+            {
+                is.close();
+            }
+            catch (IOException e)
+            {
+                throw new IOError(e);
+            }
+        }
+    }
+
+    public void printBanner()
+    {
+        sessionState.out.println(help.banner);
     }
 
     // Execute a CLI Statement 
@@ -134,7 +200,7 @@ public class CliClient extends CliUserHe
                     executeGetWithConditions(tree);
                     break;
                 case CliParser.NODE_HELP:
-                    printCmdHelp(tree, sessionState);
+                    executeHelp(tree);
                     break;
                 case CliParser.NODE_THRIFT_SET:
                     executeSet(tree);
@@ -240,7 +306,27 @@ public class CliClient extends CliUserHe
         
         return keyspacesMap.get(keyspace);
     }
-    
+
+    private void executeHelp(Tree tree)
+    {
+        if (tree.getChildCount() > 0)
+        {
+            String token = tree.getChild(0).getText();
+            for (CliCommandHelp ch : help.commands)
+            {
+                if (token.equals(ch.name))
+                {
+                    sessionState.out.println(ch.help);
+                    break;
+                }
+            }
+        }
+        else
+        {
+            sessionState.out.println(help.help);
+        }
+    }
+
     private void executeCount(Tree statement)
             throws TException, InvalidRequestException, UnavailableException, TimedOutException
     {
@@ -677,7 +763,7 @@ public class CliClient extends CliUserHe
         // table.cf['key']
         if (columnSpecCnt == 0)
         {
-            sessionState.err.println("No column name specified, (type 'help' or '?' for help on syntax).");
+            sessionState.err.println("No column name specified, (type 'help;' or '?' for help on syntax).");
             return;
         }
         // table.cf['key']['column'] = 'value'

Added: cassandra/branches/cassandra-0.8/src/java/org/apache/cassandra/cli/CliCommandHelp.java
URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.8/src/java/org/apache/cassandra/cli/CliCommandHelp.java?rev=1092430&view=auto
==============================================================================
--- cassandra/branches/cassandra-0.8/src/java/org/apache/cassandra/cli/CliCommandHelp.java (added)
+++ cassandra/branches/cassandra-0.8/src/java/org/apache/cassandra/cli/CliCommandHelp.java Thu Apr 14 18:27:34 2011
@@ -0,0 +1,24 @@
+/**
+ * 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

Modified: cassandra/branches/cassandra-0.8/src/java/org/apache/cassandra/cli/CliMain.java
URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.8/src/java/org/apache/cassandra/cli/CliMain.java?rev=1092430&r1=1092429&r2=1092430&view=diff
==============================================================================
--- cassandra/branches/cassandra-0.8/src/java/org/apache/cassandra/cli/CliMain.java (original)
+++ cassandra/branches/cassandra-0.8/src/java/org/apache/cassandra/cli/CliMain.java Thu Apr 14 18:27:34 2011
@@ -181,12 +181,6 @@ public class CliMain
         }
     }
 
-    private static void printBanner()
-    {
-        sessionState.out.println("Welcome to cassandra CLI.\n");
-        sessionState.out.println("Type 'help;' or '?' for help. Type 'quit;' or 'exit;' to quit.");
-    }
-
     /**
      * Checks whether the thrift client is connected.
      * @return boolean - true when connected, false otherwise
@@ -315,7 +309,7 @@ public class CliMain
             sessionState.out.close();
         }
 
-        printBanner();
+        cliClient.printBanner();
 
         String prompt;
         String line = "";

Modified: cassandra/branches/cassandra-0.8/src/java/org/apache/cassandra/cli/CliUserHelp.java
URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.8/src/java/org/apache/cassandra/cli/CliUserHelp.java?rev=1092430&r1=1092429&r2=1092430&view=diff
==============================================================================
--- cassandra/branches/cassandra-0.8/src/java/org/apache/cassandra/cli/CliUserHelp.java (original)
+++ cassandra/branches/cassandra-0.8/src/java/org/apache/cassandra/cli/CliUserHelp.java Thu Apr 14 18:27:34 2011
@@ -17,401 +17,16 @@
  */
 package org.apache.cassandra.cli;
 
-import java.util.EnumMap;
-
-import org.antlr.runtime.tree.Tree;
+import java.util.List;
 
 /**
  * @author Pavel A. Yaskevich
  */
-public class CliUserHelp {
-
-    /*
-     * 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,
-        ROWS_CACHED,
-        ROW_CACHE_SAVE_PERIOD,
-        KEYS_CACHED,
-        KEY_CACHE_SAVE_PERIOD,
-        READ_REPAIR_CHANCE,
-        GC_GRACE,
-        COLUMN_METADATA,
-        MEMTABLE_OPERATIONS,
-        MEMTABLE_THROUGHPUT,
-        MEMTABLE_FLUSH_AFTER,
-        DEFAULT_VALIDATION_CLASS,
-        MIN_COMPACTION_THRESHOLD,
-        MAX_COMPACTION_THRESHOLD,
-        REPLICATE_ON_WRITE,
-        ROW_CACHE_PROVIDER,
-        KEY_VALIDATION_CLASS
-    }
-
-    protected EnumMap<ColumnFamilyArgument, String> argumentExplanations = new EnumMap<ColumnFamilyArgument, String>(ColumnFamilyArgument.class)
-    {{
-        put(ColumnFamilyArgument.COLUMN_TYPE, "Super or Standard");
-        put(ColumnFamilyArgument.COMMENT, "Human-readable column family description. Any string is acceptable");
-        put(ColumnFamilyArgument.COMPARATOR, "The class used as a comparator when sorting column names.\n                  Valid options include: AsciiType, BytesType, LexicalUUIDType,\n                  LongType, IntegerType, TimeUUIDType, and UTF8Type");
-        put(ColumnFamilyArgument.SUBCOMPARATOR, "Comparator for sorting subcolumn names, for Super columns only");
-        put(ColumnFamilyArgument.MEMTABLE_OPERATIONS, "Flush memtables after this many operations (in millions)");
-        put(ColumnFamilyArgument.MEMTABLE_THROUGHPUT, "... or after this many MB have been written");
-        put(ColumnFamilyArgument.MEMTABLE_FLUSH_AFTER, "... or after this many minutes");
-        put(ColumnFamilyArgument.ROWS_CACHED, "Number or percentage of rows to cache");
-        put(ColumnFamilyArgument.ROW_CACHE_SAVE_PERIOD, "Period with which to persist the row cache, in seconds");
-        put(ColumnFamilyArgument.KEYS_CACHED, "Number or percentage of keys to cache");
-        put(ColumnFamilyArgument.KEY_CACHE_SAVE_PERIOD, "Period with which to persist the key cache, in seconds");
-        put(ColumnFamilyArgument.READ_REPAIR_CHANCE, "Probability (0.0-1.0) with which to perform read repairs on CL.ONE reads");
-        put(ColumnFamilyArgument.GC_GRACE, "Discard tombstones after this many seconds");
-        put(ColumnFamilyArgument.MIN_COMPACTION_THRESHOLD, "Avoid minor compactions of less than this number of sstable files");
-        put(ColumnFamilyArgument.MAX_COMPACTION_THRESHOLD, "Compact no more than this number of sstable files at once");
-        put(ColumnFamilyArgument.REPLICATE_ON_WRITE, "Replicate every counter update from the leader to the follower replicas");
-        put(ColumnFamilyArgument.ROW_CACHE_PROVIDER, "Row cache provider, opions are SerializingProvider/ConcurrentLinkedHashCacheProvider");
-    }};
-    
-    protected void printCmdHelp(Tree statement, CliSessionState state)
-    {
-        if (statement.getChildCount() > 0)
-        {
-            int helpType = statement.getChild(0).getType();
-
-            switch(helpType)
-            {
-            case CliParser.NODE_HELP:
-                state.out.println("help <command>;\n");
-                state.out.println("Display the general help page with a list of available commands.");
-                break;
-            case CliParser.NODE_CONNECT:
-                state.out.println("connect <hostname>/<port> (<username> '<password>')?;\n");
-                state.out.println("Connect to the specified host on the specified port (using specified username and password).\n");
-                state.out.println("example:");
-                state.out.println("connect localhost/9160;");
-                state.out.println("connect localhost/9160 user 'badpasswd';");
-                state.out.println("connect 127.0.0.1/9160 user 'badpasswd';");
-                break;
-
-            case CliParser.NODE_USE_TABLE:
-                state.out.println("use <keyspace>;");
-                state.out.println("use <keyspace> <username> '<password>';\n");
-                state.out.println("Switch to the specified keyspace. The optional username and password fields");
-                state.out.println("are needed when performing authentication.\n");
-                break;
-
-            case CliParser.NODE_DESCRIBE_TABLE:
-                state.out.println("describe keyspace (<keyspace>)?;\n");
-                state.out.println("Show additional information about the specified keyspace.");
-                state.out.println("Command could be used without <keyspace> argument if you are already authenticated to keyspace.\n");
-                state.out.println("example:");
-                state.out.println("describe keyspace system;");
-                break;
-
-            case CliParser.NODE_DESCRIBE_CLUSTER:
-                state.out.println("describe cluster;\n");
-                state.out.println("Display information about cluster: snitch, partitioner, schema versions.");
-                break;
-
-            case CliParser.NODE_EXIT:
-                state.out.println("exit;");
-                state.out.println("quit;\n");
-                state.out.println("Exit this utility.");
-                break;
-
-            case CliParser.NODE_SHOW_CLUSTER_NAME:
-                state.out.println("show cluster name;\n");
-                state.out.println("Displays the name of the currently connected cluster.");
-                break;
-
-            case CliParser.NODE_SHOW_VERSION:
-                state.out.println("show api version;\n");
-                state.out.println("Displays the API version number.");
-                break;
-
-            case CliParser.NODE_SHOW_KEYSPACES:
-                state.out.println("show keyspaces;\n");
-                state.out.println("Displays a list of the keyspaces available on the currently connected cluster.");
-                break;
-
-            case CliParser.NODE_ADD_KEYSPACE:
-                state.out.println("create keyspace <keyspace>;");
-                state.out.println("create keyspace <keyspace> with <att1>=<value1>;");
-                state.out.println("create keyspace <keyspace> with <att1>=<value1> and <att2>=<value2> ...;\n");
-                state.out.println("Create a new keyspace with the specified values for the given set of attributes.\n");
-                state.out.println("valid attributes are:");
-                state.out.println("    placement_strategy: the fully qualified class used to place replicas in");
-                state.out.println("                        this keyspace. Valid values are");
-                state.out.println("                        org.apache.cassandra.locator.SimpleStrategy,");
-                state.out.println("                        org.apache.cassandra.locator.NetworkTopologyStrategy,");
-                state.out.println("                        and org.apache.cassandra.locator.OldNetworkTopologyStrategy");
-                state.out.println("      strategy_options: additional options for placement_strategy.");
-                state.out.println("                        Simple and OldNetworkTopology strategies require");
-                state.out.println("                        'replication_factor':N, and NetworkTopologyStrategy");
-                state.out.println("                        requires a map of 'DCName':N per-DC.");
-                state.out.println("\nexamples:");
-                state.out.println("create keyspace foo with");
-                state.out.println("        placement_strategy = 'org.apache.cassandra.locator.SimpleStrategy'");
-                state.out.println("        and strategy_options=[{replication_factor:3}];");
-                state.out.println("create keyspace foo with");
-                state.out.println("        placement_strategy = 'org.apache.cassandra.locator.NetworkTopologyStrategy';");
-                state.out.println("        and strategy_options=[{DC1:2, DC2:2}];");
-                break;
-
-            case CliParser.NODE_UPDATE_KEYSPACE:
-                state.out.println("update keyspace <keyspace>;");
-                state.out.println("update keyspace <keyspace> with <att1>=<value1>;");
-                state.out.println("update keyspace <keyspace> with <att1>=<value1> and <att2>=<value2> ...;\n");
-                state.out.println("Update a keyspace with the specified values for the given set of attributes.\n");
-                state.out.println("valid attributes are:");
-                state.out.println("    placement_strategy: the fully qualified class used to place replicas in");
-                state.out.println("                        this keyspace. Valid values are");
-                state.out.println("                        org.apache.cassandra.locator.SimpleStrategy,");
-                state.out.println("                        org.apache.cassandra.locator.NetworkTopologyStrategy,");
-                state.out.println("                        and org.apache.cassandra.locator.OldNetworkTopologyStrategy");
-                state.out.println("      strategy_options: additional options for placement_strategy.");
-                state.out.println("                        Simple and OldNetworkTopology strategies require");
-                state.out.println("                        'replication_factor':N, and NetworkTopologyStrategy");
-                state.out.println("                        requires a map of 'DCName':N per-DC.");
-                state.out.println("\nexamples:");
-                state.out.println("update keyspace foo with");
-                state.out.println("        placement_strategy = 'org.apache.cassandra.locator.SimpleStrategy'");
-                state.out.println("        and strategy_options=[{replication_factor:4}];");
-                state.out.println("update keyspace foo with");
-                state.out.println("        placement_strategy = 'org.apache.cassandra.locator.NetworkTopologyStrategy';");
-                state.out.println("        and strategy_options=[{DC1:3, DC2:2}];");
-                break;
-
-            case CliParser.NODE_ADD_COLUMN_FAMILY:
-                state.out.println("create column family Bar;");
-                state.out.println("create column family Bar with <att1>=<value1>;");
-                state.out.println("create column family Bar with <att1>=<value1> and <att2>=<value2>...;\n");
-                state.out.println("Create a new column family with the specified values for the given set of");
-                state.out.println("attributes. Note that you must be using a keyspace.\n");
-                state.out.println("valid attributes are:");
-                for (ColumnFamilyArgument argument : ColumnFamilyArgument.values())
-                    state.out.printf("    - %s: %s%n", argument.toString().toLowerCase(), argumentExplanations.get(argument));
-                state.out.println("    - column_metadata: Metadata which describes columns of column family.");
-                state.out.println("        Supported format is [{ k:v, k:v, ... }, { ... }, ...]");
-                state.out.println("        Valid attributes: column_name, validation_class (see comparator),");
-                state.out.println("                          index_type (integer), index_name.");
-                state.out.println("example:\n");
-                state.out.println("create column family Bar with column_type = 'Super' and comparator = 'AsciiType'");
-                state.out.println("      and rows_cached = 10000;");
-                state.out.println("create column family Baz with comparator = 'LongType' and rows_cached = 10000;");
-                state.out.print("create column family Foo with comparator=LongType and column_metadata=");
-                state.out.print("[{ column_name:Test, validation_class:IntegerType, index_type:0, index_name:IdxName");
-                state.out.println("}, { column_name:'other name', validation_class:LongType }];");
-                break;
-
-            case CliParser.NODE_UPDATE_COLUMN_FAMILY:
-                state.out.println("update column family Bar;");
-                state.out.println("update column family Bar with <att1>=<value1>;");
-                state.out.println("update column family Bar with <att1>=<value1> and <att2>=<value2>...;\n");
-                state.out.println("Update a column family with the specified values for the given set of");
-                state.out.println("attributes. Note that you must be using a keyspace.\n");
-                state.out.println("valid attributes are:");
-                for (ColumnFamilyArgument argument : ColumnFamilyArgument.values())
-                {
-                    if (argument == ColumnFamilyArgument.COMPARATOR || argument == ColumnFamilyArgument.SUBCOMPARATOR)
-                        continue;
-                    state.out.printf("    - %s: %s%n", argument.toString().toLowerCase(), argumentExplanations.get(argument));
-                }
-                state.out.println("    - column_metadata: Metadata which describes columns of column family.");
-                state.out.println("        Supported format is [{ k:v, k:v, ... }, { ... }, ...]");
-                state.out.println("        Valid attributes: column_name, validation_class (see comparator),");
-                state.out.println("                          index_type (integer), index_name.");
-                state.out.println("example:\n");
-                state.out.print("update column family Foo with column_metadata=");
-                state.out.print("[{ column_name:Test, validation_class:IntegerType, index_type:0, index_name:IdxName");
-                state.out.println("}] and rows_cached=100 and comment='this is helpful comment.';");
-                break;
-
-            case CliParser.NODE_DEL_KEYSPACE:
-                state.out.println("drop keyspace <keyspace>;\n");
-                state.out.println("Drops the specified keyspace.\n");
-                state.out.println("example:");
-                state.out.println("drop keyspace foo;");
-                break;
-
-            case CliParser.NODE_DEL_COLUMN_FAMILY:
-                state.out.println("drop column family <name>;\n");
-                state.out.println("Drops the specified column family.\n");
-                state.out.println("example:");
-                state.out.println("drop column family foo;");
-                break;
-
-            case CliParser.NODE_THRIFT_GET :
-                state.out.println("get <cf>['<key>'];");
-                state.out.println("get <cf>['<key>']['<col>'] (as <type>)*;");
-                state.out.println("get <cf>['<key>']['<super>'];");
-                state.out.println("get <cf>['<key>'][<function>];");
-                state.out.println("get <cf>['<key>'][<function>(<super>)][<function>(<col>)];");
-                state.out.println("get <cf> where <column> = <value> [and <column> > <value> and ...] [limit <integer>];");
-                state.out.println("Default LIMIT is 100. Available operations: =, >, >=, <, <=\n");
-                state.out.println("get <cf>['<key>']['<super>']['<col>'] (as <type>)*;");
-                state.out.print("Note: `as <type>` is optional, it dynamically converts column value to the specified type");
-                state.out.println(", column value validator will be set to <type>.");
-                state.out.println("Available functions: " + CliClient.Function.getFunctionNames());
-                state.out.println("Available types: IntegerType, LongType, UTF8Type, ASCIIType, TimeUUIDType, LexicalUUIDType.\n");
-                state.out.println("examples:");
-                state.out.println("get bar[testkey];");
-                state.out.println("get bar[testkey][test_column] as IntegerType;");
-                state.out.println("get bar[testkey][utf8(hello)];");
-                break;
-
-            case CliParser.NODE_THRIFT_SET:
-                state.out.println("set <cf>['<key>']['<col>'] = <value>;");
-                state.out.println("set <cf>['<key>']['<super>']['<col>'] = <value>;");
-                state.out.println("set <cf>['<key>']['<col>'] = <function>(<argument>);");
-                state.out.println("set <cf>['<key>']['<super>']['<col>'] = <function>(<argument>);");
-                state.out.println("set <cf>[<key>][<function>(<col>)] = <value> || <function>;");
-                state.out.println("set <cf>[<key>][<function>(<col>) || <col>] = <value> || <function> with ttl = <secs>;");
-                state.out.println("Available functions: " + CliClient.Function.getFunctionNames() + "\n");
-                state.out.println("examples:");
-                state.out.println("set bar['testkey']['my super']['test col']='this is a test';");
-                state.out.println("set baz['testkey']['test col']='this is also a test';");
-                state.out.println("set diz[testkey][testcol] = utf8('this is utf8 string.');");
-                state.out.println("set bar[testkey][timeuuid()] = utf('hello world');");
-                state.out.println("set bar[testkey][timeuuid()] = utf('hello world') with ttl = 30;");
-                state.out.println("set diz[testkey][testcol] = 'this is utf8 string.' with ttl = 150;");
-                break;
-
-            case CliParser.NODE_THRIFT_INCR:
-                state.out.println("incr <cf>['<key>']['<col>'] [by <value>];");
-                state.out.println("incr <cf>['<key>']['<super>']['<col>'] [by <value>];");
-                state.out.println("examples:");
-                state.out.println("incr bar['testkey']['my super']['test col'];");
-                state.out.println("incr bar['testkey']['my super']['test col'] by 42;");
-                state.out.println("incr baz['testkey']['test col'] by -4;");
-                break;
-
-            case CliParser.NODE_THRIFT_DECR:
-                state.out.println("decr <cf>['<key>']['<col>'] [by <value>];");
-                state.out.println("decr <cf>['<key>']['<super>']['<col>'] [by <value>];");
-                state.out.println("examples:");
-                state.out.println("decr bar['testkey']['my super']['test col'];");
-                state.out.println("decr bar['testkey']['my super']['test col'] by 42;");
-                state.out.println("decr baz['testkey']['test col'] by 10;");
-                break;
-
-            case CliParser.NODE_THRIFT_DEL:
-                state.out.println("del <cf>['<key>'];");
-                state.out.println("del <cf>['<key>']['<col>'];");
-                state.out.println("del <cf>['<key>']['<super>']['<col>'];\n");
-                state.out.println("Deletes a record, a column, or a subcolumn.\n");
-                state.out.println("example:");
-                state.out.println("del bar['testkey']['my super']['test col'];");
-                state.out.println("del baz['testkey']['test col'];");
-                state.out.println("del baz['testkey'];");
-                break;
-
-            case CliParser.NODE_THRIFT_COUNT:
-                state.out.println("count <cf>['<key>'];");
-                state.out.println("count <cf>['<key>']['<super>'];\n");
-                state.out.println("Count the number of columns in the specified key or subcolumns in the specified");
-                state.out.println("super column.\n");
-                state.out.println("example:");
-                state.out.println("count bar['testkey']['my super'];");
-                state.out.println("count baz['testkey'];");
-                break;
-
-            case CliParser.NODE_LIST:
-                state.out.println("list <cf>;");
-                state.out.println("list <cf>[<startKey>:];");
-                state.out.println("list <cf>[<startKey>:<endKey>];");
-                state.out.println("list ... limit N;");
-                state.out.println("List a range of rows in the column or supercolumn family.\n");
-                state.out.println("example:");
-                state.out.println("list Users[j:] limit 40;");
-                break;
-
-            case CliParser.NODE_TRUNCATE:
-                state.out.println("truncate <column_family>;");
-                state.out.println("Truncate specified column family.\n");
-                state.out.println("example:");
-                state.out.println("truncate Category;");
-                break;
+public class CliUserHelp
+{
+    public String banner;
 
-            case CliParser.NODE_ASSUME:
-                state.out.println("assume <column_family> comparator as <type>;");
-                state.out.println("assume <column_family> sub_comparator as <type>;");
-                state.out.println("assume <column_family> validator as <type>;");
-                state.out.println("assume <column_family> keys as <type>;\n");
-                state.out.println("Assume one of the attributes (comparator, sub_comparator, validator or keys)");
-                state.out.println("of the given column family to match specified type. Available types: " + CliClient.Function.getFunctionNames());
-                state.out.println("example:");
-                state.out.println("assume Users comparator as lexicaluuid;");
-                break;
-            case CliParser.NODE_CONSISTENCY_LEVEL:
-                state.out.println("consistencylevel as <level>");
-                state.out.println("example:");
-                state.out.println("consistencylevel as QUORUM");
-                break;
-            default:
-                state.out.println("?");
-                break;
-            }
-        }
-        else
-        {
-            state.out.println("List of all CLI commands:");
-            state.out.println("?                                                          Display this message.");
-            state.out.println("help;                                                         Display this help.");
-            state.out.println("help <command>;                         Display detailed, command-specific help.");
-            state.out.println("connect <hostname>/<port> (<username> '<password>')?; Connect to thrift service.");
-            state.out.println("use <keyspace> [<username> 'password'];                    Switch to a keyspace.");
-            state.out.println("describe keyspace (<keyspacename>)?;                          Describe keyspace.");
-            state.out.println("exit;                                                                  Exit CLI.");
-            state.out.println("quit;                                                                  Exit CLI.");
-            state.out.println("describe cluster;                             Display information about cluster.");
-            state.out.println("show cluster name;                                         Display cluster name.");
-            state.out.println("show keyspaces;                                          Show list of keyspaces.");
-            state.out.println("show api version;                                       Show server API version.");
-            state.out.println("create keyspace <keyspace> [with <att1>=<value1> [and <att2>=<value2> ...]];");
-            state.out.println("                Add a new keyspace with the specified attribute(s) and value(s).");
-            state.out.println("update keyspace <keyspace> [with <att1>=<value1> [and <att2>=<value2> ...]];");
-            state.out.println("                 Update a keyspace with the specified attribute(s) and value(s).");
-            state.out.println("create column family <cf> [with <att1>=<value1> [and <att2>=<value2> ...]];");
-            state.out.println("        Create a new column family with the specified attribute(s) and value(s).");
-            state.out.println("update column family <cf> [with <att1>=<value1> [and <att2>=<value2> ...]];");
-            state.out.println("            Update a column family with the specified attribute(s) and value(s).");
-            state.out.println("drop keyspace <keyspace>;                                     Delete a keyspace.");
-            state.out.println("drop column family <cf>;                                 Delete a column family.");
-            state.out.println("get <cf>['<key>'];                                       Get a slice of columns.");
-            state.out.println("get <cf>['<key>']['<super>'];                        Get a slice of sub columns.");
-            state.out.println("get <cf> where <column> = <value> [and <column> > <value> and ...] [limit int]; ");
-            state.out.println("get <cf>['<key>']['<col>'] (as <type>)*;                     Get a column value.");
-            state.out.println("get <cf>['<key>']['<super>']['<col>'] (as <type>)*;      Get a sub column value.");
-            state.out.println("set <cf>['<key>']['<col>'] = <value> (with ttl = <secs>)*;         Set a column.");
-            state.out.println("set <cf>['<key>']['<super>']['<col>'] = <value> (with ttl = <secs>)*;");
-            state.out.println("                                                               Set a sub column.");
-            state.out.println("del <cf>['<key>'];                                                Delete record.");
-            state.out.println("del <cf>['<key>']['<col>'];                                       Delete column.");
-            state.out.println("del <cf>['<key>']['<super>']['<col>'];                        Delete sub column.");
-            state.out.println("count <cf>['<key>'];                                    Count columns in record.");
-            state.out.println("count <cf>['<key>']['<super>'];                 Count columns in a super column.");
-            state.out.println("incr <cf>['<key>']['<col>'] [by <value>];            Increment a counter column.");
-            state.out.println("incr <cf>['<key>']['<super>']['<col>'] [by <value>];");
-            state.out.println("                                                 Increment a counter sub-column.");
-            state.out.println("decr <cf>['<key>']['<col>'] [by <value>];            Decrement a counter column.");
-            state.out.println("decr <cf>['<key>']['<super>']['<col>'] [by <value>];");
-            state.out.println("                                                 Decrement a counter sub-column.");
-            state.out.println("truncate <column_family>;                      Truncate specified column family.");
-            state.out.println("assume <column_family> <attribute> as <type>;");
-            state.out.println("              Assume a given column family attributes to match a specified type.");
-            state.out.println("consistencylevel as <level>;");
-            state.out.println("                  Change the consistency level for set,get, and list operations.");
-            state.out.println("list <cf>;                                   List all rows in the column family.");
-            state.out.println("list <cf>[<startKey>:];");
-            state.out.println("                       List rows in the column family beginning with <startKey>.");
-            state.out.println("list <cf>[<startKey>:<endKey>];");
-            state.out.println("        List rows in the column family in the range from <startKey> to <endKey>.");
-            state.out.println("list ... limit N;                                   Limit the list results to N.");
-        }
-    }
+    public String help;
 
+    public List<CliCommandHelp> commands;
 }

Added: cassandra/branches/cassandra-0.8/src/resources/org/apache/cassandra/cli/CliHelp.yaml
URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.8/src/resources/org/apache/cassandra/cli/CliHelp.yaml?rev=1092430&view=auto
==============================================================================
--- cassandra/branches/cassandra-0.8/src/resources/org/apache/cassandra/cli/CliHelp.yaml (added)
+++ cassandra/branches/cassandra-0.8/src/resources/org/apache/cassandra/cli/CliHelp.yaml Thu Apr 14 18:27:34 2011
@@ -0,0 +1,1146 @@
+# Help file for online commands in Yaml.
+
+banner: |
+    Welcome to the Cassandra CLI.
+
+    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 keyspace       Describe a keyspace and it's column families.
+    drop column family      Remove a column family and it's data.
+    drop keyspace           Remove a keyspace and it's data.
+    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.
+    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_TABLE
+      help: |
+        describe keyspace (<keyspace>)?;
+
+        Describes the settings for the current or named keyspace, and the settings
+        for all column families in the keyspace.
+
+        Optional Parameters:
+        - keyspace: Name of the keyspace to describe.
+
+        Examples:
+        describe keyspace;
+        describe keyspace system;
+    - 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_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.
+
+        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.
+
+        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
+                - BytesType
+                - CounterColumnType (distributed counter column)
+                - 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.
+
+          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:
+                - 0: for a KEYS based index
+
+        - 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
+            - BytesType
+            - CounterColumnType (distributed counter column)
+            - 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.
+
+        - 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
+            - BytesType
+            - CounterColumnType (distributed counter column)
+            - 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.
+
+        - key_valiation_class: Validator to use for keys.
+          Default is BytesType which applies no validation.
+
+          Supported values are:
+            - AsciiType
+            - BytesType
+            - 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
+
+        - keys_cached: Maximum number of keys to cache in memory. Valid values are
+          either a double between 0 and 1 (inclusive on both ends) denoting what
+          fraction should be cached. Or an absolute number of rows to cache.
+          Default value is 200000.
+
+          Each key cache hit saves 1 seek and each row cache hit saves 2 seeks at the
+          minimum, sometimes more. The key cache is fairly tiny for the amount of
+          time it saves, so it's worthwhile to use it at large numbers all the way
+          up to 1.0 (all keys cached). The row cache saves even more time, but must
+          store the whole values of its rows, so it is extremely space-intensive.
+          It's best to only use the row cache if you have hot rows or static rows.
+
+        - keys_cached_save_period: Duration in seconds after which Cassandra should
+          safe the keys cache. Caches are saved to saved_caches_directory as
+          specified in conf/Cassandra.yaml. Default is 14400 or 4 hours.
+
+          Saved caches greatly improve cold-start speeds, and is relatively cheap in
+          terms of I/O for the key cache. Row cache saving is much more expensive and
+          has limited use.
+
+        - memtable_flush_after: Maximum number of minutes to leave a dirty
+          memtable unflushed. This value needs to be large enough that it won't cause
+          a flush storm of all your memtables flushing at once because none have
+          hit the size or count thresholds yet. For production a larger value such
+          as 1440 is recommended. Default is 60.
+
+          NOTE: While any affected column families have unflushed data from a commit
+          log segment, that segment cannot be deleted.
+
+        - memtable_operations: Number of operations in millions before the memtable
+          is flushed. Default is memtable_throughput / 64 * 0.3
+
+        - memtable_throughput: Maximum size in MB to let a memtable get to before
+          it is flushed. Default is to use 1/16 the JVM heap size.
+
+        - read_repair_chance: Probability (0.0-1.0) with which to perform read
+          repairs for any read operation. Default is 1.0 to enable read repair.
+
+        - rows_cached: Maximum number of rows whose entire contents we
+          cache in memory. Valid values are either a double between 0 and 1 (
+          inclusive on both ends) denoting what fraction should be cached. Or an
+          absolute number of rows to cache. Default value is 0, to disable row
+          caching.
+
+          Each key cache hit saves 1 seek and each row cache hit saves 2 seeks at the
+          minimum, sometimes more. The key cache is fairly tiny for the amount of
+          time it saves, so it's worthwhile to use it at large numbers all the way
+          up to 1.0 (all keys cached). The row cache saves even more time, but must
+          store the whole values of its rows, so it is extremely space-intensive.
+          It's best to only use the row cache if you have hot rows or static rows.
+
+        - row_cache_save_period: Duration in seconds after which Cassandra should
+          safe the row cache. Caches are saved to saved_caches_directory as specified
+          in conf/Cassandra.yaml. Default is 0 to disable saving the row cache.
+
+          Saved caches greatly improve cold-start speeds, and is relatively cheap in
+          terms of I/O for the key cache. Row cache saving is much more expensive and
+          has limited use.
+
+        - 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
+            - BytesType
+            - CounterColumnType (distributed counter column)
+            - 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.
+
+        - replicate_on_write: Replicate every counter update from the leader to the
+        follower replicas. Accepts the values true and false.
+
+        - row_cache_provider: The provider for the row cache to use for this
+        column family. Defaults to ConcurrentLinkedHashCacheProvider.  .
+
+        Supported values are:
+            - ConcurrentLinkedHashCacheProvider
+            - SerializingCacheProvider
+
+        It is also valid to specify the fully-qualified class name to a class
+        that implements org.apache.cassandra.cache.IRowCacheProvider.
+
+        ConcurrentLinkedHashCacheProvider provides the same features as the versions
+        prior to Cassandra v0.8. Row data is cached using the Java JVM heap.
+
+        SerializingCacheProvider serialises the contents of the row and
+        stores the data off the JVM Heap. This may reduce the GC pressure.
+        NOTE: Thsi provider requires JNA.jar to be in the class path to
+        enable native methods.
+
+        Examples:
+        create column family Super4
+            with column_type = 'Super'
+            and comparator = 'AsciiType'
+            and rows_cached = 10000;
+        create column family Standard3
+            with comparator = 'LongType'
+            and rows_cached = 10000;
+        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
+            }];
+    - 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
+                - BytesType
+                - CounterColumnType (distributed counter column)
+                - 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.
+
+          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:
+                - 0: for a KEYS based index
+
+        - 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
+            - BytesType
+            - CounterColumnType (distributed counter column)
+            - 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.
+
+        - 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
+            - BytesType
+            - CounterColumnType (distributed counter column)
+            - 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.
+
+        - key_valiation_class: Validator to use for keys.
+          Default is BytesType which applies no validation.
+
+          Supported values are:
+            - AsciiType
+            - BytesType
+            - 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
+
+        - keys_cached: Maximum number of keys to cache in memory. Valid values are
+          either a double between 0 and 1 (inclusive on both ends) denoting what
+          fraction should be cached. Or an absolute number of rows to cache.
+          Default value is 200000.
+
+          Each key cache hit saves 1 seek and each row cache hit saves 2 seeks at the
+          minimum, sometimes more. The key cache is fairly tiny for the amount of
+          time it saves, so it's worthwhile to use it at large numbers all the way
+          up to 1.0 (all keys cached). The row cache saves even more time, but must
+          store the whole values of its rows, so it is extremely space-intensive.
+          It's best to only use the row cache if you have hot rows or static rows.
+
+        - keys_cached_save_period: Duration in seconds after which Cassandra should
+          safe the keys cache. Caches are saved to saved_caches_directory as
+          specified in conf/Cassandra.yaml. Default is 14400 or 4 hours.
+
+          Saved caches greatly improve cold-start speeds, and is relatively cheap in
+          terms of I/O for the key cache. Row cache saving is much more expensive and
+          has limited use.
+
+        - memtable_flush_after: Maximum number of minutes to leave a dirty
+          memtable unflushed. This value needs to be large enough that it won't cause
+          a flush storm of all your memtables flushing at once because none have
+          hit the size or count thresholds yet. For production a larger value such
+          as 1440 is recommended. Default is 60.
+
+          NOTE: While any affected column families have unflushed data from a commit
+          log segment, that segment cannot be deleted.
+
+        - memtable_operations: Number of operations in millions before the memtable
+          is flushed. Default is memtable_throughput / 64 * 0.3
+
+        - memtable_throughput: Maximum size in MB to let a memtable get to before
+          it is flushed. Default is to use 1/16 the JVM heap size.
+
+        - read_repair_chance: Probability (0.0-1.0) with which to perform read
+          repairs for any read operation. Default is 1.0 to enable read repair.
+
+        - rows_cached: Maximum number of rows whose entire contents we
+          cache in memory. Valid values are either a double between 0 and 1 (
+          inclusive on both ends) denoting what fraction should be cached. Or an
+          absolute number of rows to cache. Default value is 0, to disable row
+          caching.
+
+          Each key cache hit saves 1 seek and each row cache hit saves 2 seeks at the
+          minimum, sometimes more. The key cache is fairly tiny for the amount of
+          time it saves, so it's worthwhile to use it at large numbers all the way
+          up to 1.0 (all keys cached). The row cache saves even more time, but must
+          store the whole values of its rows, so it is extremely space-intensive.
+          It's best to only use the row cache if you have hot rows or static rows.
+
+        - row_cache_save_period: Duration in seconds after which Cassandra should
+          safe the row cache. Caches are saved to saved_caches_directory as specified
+          in conf/Cassandra.yaml. Default is 0 to disable saving the row cache.
+
+          Saved caches greatly improve cold-start speeds, and is relatively cheap in
+          terms of I/O for the key cache. Row cache saving is much more expensive and
+          has limited use.
+
+        - 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
+            - BytesType
+            - CounterColumnType (distributed counter column)
+            - 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.
+
+        - replicate_on_write: Replicate every counter update from the leader to the
+        follower replicas. Accepts the values true and false.
+
+        - row_cache_provider: The provider for the row cache to use for this
+        column family. Defaults to ConcurrentLinkedHashCacheProvider.  .
+
+        Supported values are:
+            - ConcurrentLinkedHashCacheProvider
+            - SerializingCacheProvider
+
+        It is also valid to specify the fully-qualified class name to a class
+        that implements org.apache.cassandra.cache.IRowCacheProvider.
+
+        ConcurrentLinkedHashCacheProvider provides the same features as the versions
+        prior to Cassandra v0.8. Row data is cached using the Java JVM heap.
+
+        SerializingCacheProvider serialises the contents of the row and
+        stores the data off the JVM Heap. This may reduce the GC pressure.
+        NOTE: Thsi provider requires JNA.jar to be in the class path to
+        enable native methods.
+
+        Examples:
+        update column family Super4
+            with column_type = 'Super'
+            and comparator = 'AsciiType'
+            and rows_cached = 10000;
+        update column family Standard3
+            with comparator = 'LongType'
+            and rows_cached = 10000;
+        update 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
+            }];
+    - 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_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
+            - 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
+            - BytesType
+            - CounterColumnType (distributed counter column)
+            - 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
+            - 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;
+    - 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
+            - integer
+            - lexicaluuid: if used without arguments generates a new random uuid
+            - long
+            - 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 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.
+
+        - 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;
+    - 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:
+            - AsciiType
+            - BytesType
+            - CounterColumnType (distributed counter column)
+            - 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.
+
+        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.
\ No newline at end of file