You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by br...@apache.org on 2013/10/14 21:20:56 UTC

[1/6] git commit: Add PigUnit unit tests. Patch by Alex Liu, reviewed by brandonwilliams for CASSANDRA-5695

Updated Branches:
  refs/heads/cassandra-1.2 17ae0429c -> 4e3596979
  refs/heads/cassandra-2.0 3a29c7480 -> fe598e77e
  refs/heads/trunk bb13b803c -> 0f1303102


Add PigUnit unit tests.
Patch by Alex Liu, reviewed by brandonwilliams for CASSANDRA-5695


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

Branch: refs/heads/cassandra-1.2
Commit: 4e35969795793159075cd9368f3dd595b265584f
Parents: 17ae042
Author: Brandon Williams <br...@apache.org>
Authored: Mon Oct 14 14:13:18 2013 -0500
Committer: Brandon Williams <br...@apache.org>
Committed: Mon Oct 14 14:13:18 2013 -0500

----------------------------------------------------------------------
 build.xml                                       |   9 +
 .../hadoop/pig/AbstractCassandraStorage.java    |   2 +
 .../cassandra/hadoop/pig/CassandraStorage.java  |  25 +-
 .../apache/cassandra/hadoop/pig/CqlStorage.java |  25 +-
 test/conf/log4j-junit.properties                |   3 +
 .../org/apache/cassandra/pig/CqlTableTest.java  | 236 ++++++
 .../org/apache/cassandra/pig/PigTestBase.java   | 167 ++++
 .../cassandra/pig/ThriftColumnFamilyTest.java   | 809 +++++++++++++++++++
 test/unit/org/apache/pig/test/MiniCluster.java  |  78 ++
 .../org/apache/pig/test/MiniGenericCluster.java | 122 +++
 10 files changed, 1470 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/4e359697/build.xml
----------------------------------------------------------------------
diff --git a/build.xml b/build.xml
index 74c7374..37109c8 100644
--- a/build.xml
+++ b/build.xml
@@ -370,6 +370,7 @@
              <exclusion groupId="commons-lang" artifactId="commons-lang"/>
           </dependency>
           <dependency groupId="org.apache.hadoop" artifactId="hadoop-core" version="1.0.3"/>
+          <dependency groupId="org.apache.hadoop" artifactId="hadoop-minicluster" version="1.0.3"/>
           <dependency groupId="org.apache.pig" artifactId="pig" version="0.10.0"/>
           <dependency groupId="net.sf.jopt-simple" artifactId="jopt-simple" version="3.2"/>
           <dependency groupId="net.java.dev.jna" artifactId="jna" version="3.2.7"/>
@@ -417,6 +418,7 @@
         <dependency groupId="commons-logging" artifactId="commons-logging"/>
         <dependency groupId="org.apache.rat" artifactId="apache-rat"/>
         <dependency groupId="org.apache.hadoop" artifactId="hadoop-core"/>
+      	<dependency groupId="org.apache.hadoop" artifactId="hadoop-minicluster"/>
         <dependency groupId="org.apache.pig" artifactId="pig"/>
         <dependency groupId="net.sf.jopt-simple" artifactId="jopt-simple"/>
 
@@ -481,6 +483,7 @@
         
         <!-- don't need hadoop classes to run, but if you use the hadoop stuff -->
         <dependency groupId="org.apache.hadoop" artifactId="hadoop-core" optional="true"/>
+        <dependency groupId="org.apache.hadoop" artifactId="hadoop-minicluster" optional="true"/>
         <dependency groupId="org.apache.pig" artifactId="pig" optional="true"/>
 
         <!-- don't need jna to run, but nice to have -->
@@ -1160,6 +1163,12 @@
     </testmacro>
   </target>
 
+  <target name="pig-test" depends="build-test" description="Excute Pig tests">
+    <testmacro suitename="unit" inputdir="${test.unit.src}" 
+               timeout="1200000" filter="**/pig/*Test.java">
+    </testmacro>
+  </target>
+
   <!-- instruments the classes to later create code coverage reports -->
   <target name="cobertura-instrument" depends="build,build-test">
     <taskdef resource="tasks.properties">

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4e359697/src/java/org/apache/cassandra/hadoop/pig/AbstractCassandraStorage.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/hadoop/pig/AbstractCassandraStorage.java b/src/java/org/apache/cassandra/hadoop/pig/AbstractCassandraStorage.java
index dbebfb5..a02e309 100644
--- a/src/java/org/apache/cassandra/hadoop/pig/AbstractCassandraStorage.java
+++ b/src/java/org/apache/cassandra/hadoop/pig/AbstractCassandraStorage.java
@@ -98,6 +98,8 @@ public abstract class AbstractCassandraStorage extends LoadFunc implements Store
     protected int splitSize = 64 * 1024;
     protected String partitionerClass;
     protected boolean usePartitionFilter = false;
+    protected String initHostAddress;
+    protected String rpcPort;
 
     public AbstractCassandraStorage()
     {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4e359697/src/java/org/apache/cassandra/hadoop/pig/CassandraStorage.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/hadoop/pig/CassandraStorage.java b/src/java/org/apache/cassandra/hadoop/pig/CassandraStorage.java
index a7cc1ad..0512a37 100644
--- a/src/java/org/apache/cassandra/hadoop/pig/CassandraStorage.java
+++ b/src/java/org/apache/cassandra/hadoop/pig/CassandraStorage.java
@@ -316,6 +316,10 @@ public class CassandraStorage extends AbstractCassandraStorage
             ConfigHelper.setInputSplitSize(conf, splitSize);
         if (partitionerClass!= null)
             ConfigHelper.setInputPartitioner(conf, partitionerClass);
+        if (rpcPort != null)
+            ConfigHelper.setInputRpcPort(conf, rpcPort);
+        if (initHostAddress != null)
+            ConfigHelper.setInputInitialAddress(conf, initHostAddress);
 
         ConfigHelper.setInputColumnFamily(conf, keyspace, column_family, widerows);
         setConnectionInformation();
@@ -346,6 +350,16 @@ public class CassandraStorage extends AbstractCassandraStorage
             ConfigHelper.setInputSplitSize(conf, splitSize);
         if (partitionerClass!= null)
             ConfigHelper.setOutputPartitioner(conf, partitionerClass);
+        if (rpcPort != null)
+        {
+            ConfigHelper.setOutputRpcPort(conf, rpcPort);
+            ConfigHelper.setInputRpcPort(conf, rpcPort);
+        }
+        if (initHostAddress != null)
+        {
+            ConfigHelper.setOutputInitialAddress(conf, initHostAddress);
+            ConfigHelper.setInputInitialAddress(conf, initHostAddress);
+        }
 
         ConfigHelper.setOutputColumnFamily(conf, keyspace, column_family);
         setConnectionInformation();
@@ -781,6 +795,10 @@ public class CassandraStorage extends AbstractCassandraStorage
                     splitSize = Integer.parseInt(urlQuery.get("split_size"));
                 if (urlQuery.containsKey("partitioner"))
                     partitionerClass = urlQuery.get("partitioner");
+                if (urlQuery.containsKey("init_address"))
+                    initHostAddress = urlQuery.get("init_address");
+                if (urlQuery.containsKey("rpc_port"))
+                    rpcPort = urlQuery.get("rpc_port");
             }
             String[] parts = urlParts[0].split("/+");
             String[] credentialsAndKeyspace = parts[1].split("@");
@@ -800,9 +818,10 @@ public class CassandraStorage extends AbstractCassandraStorage
         catch (Exception e)
         {
             throw new IOException("Expected 'cassandra://[username:password@]<keyspace>/<columnfamily>" +
-            		                        "[?slice_start=<start>&slice_end=<end>[&reversed=true][&limit=1]" +
-            		                        "[&allow_deletes=true][&widerows=true][&use_secondary=true]" +
-            		                        "[&comparator=<comparator>][&split_size=<size>][&partitioner=<partitioner>]]': " + e.getMessage());
+                    "[?slice_start=<start>&slice_end=<end>[&reversed=true][&limit=1]" +
+                    "[&allow_deletes=true][&widerows=true][&use_secondary=true]" +
+                    "[&comparator=<comparator>][&split_size=<size>][&partitioner=<partitioner>]" +
+                    "[&init_address=<host>][&rpc_port=<port>]]': " + e.getMessage());
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4e359697/src/java/org/apache/cassandra/hadoop/pig/CqlStorage.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/hadoop/pig/CqlStorage.java b/src/java/org/apache/cassandra/hadoop/pig/CqlStorage.java
index 7e1f56c..e306005 100644
--- a/src/java/org/apache/cassandra/hadoop/pig/CqlStorage.java
+++ b/src/java/org/apache/cassandra/hadoop/pig/CqlStorage.java
@@ -207,6 +207,10 @@ public class CqlStorage extends AbstractCassandraStorage
             ConfigHelper.setInputSplitSize(conf, splitSize);
         if (partitionerClass!= null)
             ConfigHelper.setInputPartitioner(conf, partitionerClass);
+        if (rpcPort != null)
+            ConfigHelper.setInputRpcPort(conf, rpcPort);
+        if (initHostAddress != null)
+            ConfigHelper.setInputInitialAddress(conf, initHostAddress);
 
         ConfigHelper.setInputColumnFamily(conf, keyspace, column_family);
         setConnectionInformation();
@@ -262,6 +266,16 @@ public class CqlStorage extends AbstractCassandraStorage
             ConfigHelper.setInputSplitSize(conf, splitSize);
         if (partitionerClass!= null)
             ConfigHelper.setOutputPartitioner(conf, partitionerClass);
+        if (rpcPort != null)
+        {
+            ConfigHelper.setOutputRpcPort(conf, rpcPort);
+            ConfigHelper.setInputRpcPort(conf, rpcPort);
+        }
+        if (initHostAddress != null)
+        {
+            ConfigHelper.setOutputInitialAddress(conf, initHostAddress);
+            ConfigHelper.setInputInitialAddress(conf, initHostAddress);
+        }
 
         ConfigHelper.setOutputColumnFamily(conf, keyspace, column_family);
         CqlConfigHelper.setOutputCql(conf, outputQuery);
@@ -633,7 +647,11 @@ public class CqlStorage extends AbstractCassandraStorage
                 if (urlQuery.containsKey("partitioner"))
                     partitionerClass = urlQuery.get("partitioner");
                 if (urlQuery.containsKey("use_secondary"))
-                    usePartitionFilter = Boolean.parseBoolean(urlQuery.get("use_secondary")); 
+                    usePartitionFilter = Boolean.parseBoolean(urlQuery.get("use_secondary"));
+                if (urlQuery.containsKey("init_address"))
+                    initHostAddress = urlQuery.get("init_address");
+                if (urlQuery.containsKey("rpc_port"))
+                    rpcPort = urlQuery.get("rpc_port");
             }
             String[] parts = urlParts[0].split("/+");
             String[] credentialsAndKeyspace = parts[1].split("@");
@@ -653,8 +671,9 @@ public class CqlStorage extends AbstractCassandraStorage
         catch (Exception e)
         {
             throw new IOException("Expected 'cql://[username:password@]<keyspace>/<columnfamily>" +
-            		                         "[?[page_size=<size>][&columns=<col1,col2>][&output_query=<prepared_statement>]" +
-            		                         "[&where_clause=<clause>][&split_size=<size>][&partitioner=<partitioner>][&use_secondary=true|false]]': " + e.getMessage());
+                    "[?[page_size=<size>][&columns=<col1,col2>][&output_query=<prepared_statement>]" +
+                    "[&where_clause=<clause>][&split_size=<size>][&partitioner=<partitioner>][&use_secondary=true|false]" +
+                    "[&init_address=<host>][&rpc_port=<port>]]': " + e.getMessage());
         }
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4e359697/test/conf/log4j-junit.properties
----------------------------------------------------------------------
diff --git a/test/conf/log4j-junit.properties b/test/conf/log4j-junit.properties
index 7c73c54..3bc0c1a 100644
--- a/test/conf/log4j-junit.properties
+++ b/test/conf/log4j-junit.properties
@@ -35,3 +35,6 @@ log4j.appender.R.layout=org.apache.log4j.PatternLayout
 log4j.appender.R.layout.ConversionPattern=%5p [%t] %d{ISO8601} %F (line %L) %m%n
 # Edit the next line to point to your logs directory
 log4j.appender.R.File=build/test/logs/system.log
+
+log4j.logger.org.apache.hadoop=ERROR
+

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4e359697/test/unit/org/apache/cassandra/pig/CqlTableTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/pig/CqlTableTest.java b/test/unit/org/apache/cassandra/pig/CqlTableTest.java
new file mode 100644
index 0000000..e9266c9
--- /dev/null
+++ b/test/unit/org/apache/cassandra/pig/CqlTableTest.java
@@ -0,0 +1,236 @@
+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;
+
+public class CqlTableTest extends PigTestBase
+{    
+    private static String[] statements = {
+            "CREATE KEYSPACE cql3ks WITH replication = {'class': 'SimpleStrategy', 'replication_factor' : 1}",
+            "USE cql3ks;",
+
+            "CREATE TABLE cqltable (key1 text, key2 int, column1 int, column2 float, primary key(key1, key2))",
+            "INSERT INTO cqltable (key1, key2, column1, column2) values ('key1', 111, 100, 10.1)",
+            "CREATE TABLE compactcqltable (key1 text, column1 int, column2 float, primary key(key1)) WITH COMPACT STORAGE",
+            "INSERT INTO compactcqltable (key1, column1, column2) values ('key1', 100, 10.1)",
+
+            "CREATE TABLE test (a int PRIMARY KEY, b int);",
+
+            "CREATE TABLE moredata (x int PRIMARY KEY, y int);",
+            "INSERT INTO test (a,b) VALUES (1,1);",
+            "INSERT INTO test (a,b) VALUES (2,2);",
+            "INSERT INTO test (a,b) VALUES (3,3);",
+            "INSERT INTO moredata (x, y) VALUES (4,4);",
+            "INSERT INTO moredata (x, y) VALUES (5,5);",
+            "INSERT INTO moredata (x, y) VALUES (6,6);",
+
+            "CREATE TABLE compotable (a int, b int, c text, d text, PRIMARY KEY (a,b,c));",
+            "INSERT INTO compotable (a, b , c , d ) VALUES ( 1,1,'One','match');",
+            "INSERT INTO compotable (a, b , c , d ) VALUES ( 2,2,'Two','match');",
+            "INSERT INTO compotable (a, b , c , d ) VALUES ( 3,3,'Three','match');",
+            "INSERT INTO compotable (a, b , c , d ) VALUES ( 4,4,'Four','match');",
+
+            "create table compmore (id int PRIMARY KEY, x int, y int, z text, data text);",
+            "INSERT INTO compmore (id, x, y, z,data) VALUES (1,5,6,'Fix','nomatch');",
+            "INSERT INTO compmore (id, x, y, z,data) VALUES (2,6,5,'Sive','nomatch');",
+            "INSERT INTO compmore (id, x, y, z,data) VALUES (3,7,7,'Seven','match');",
+            "INSERT INTO compmore (id, x, y, z,data) VALUES (4,8,8,'Eight','match');",
+            "INSERT INTO compmore (id, x, y, z,data) VALUES (5,9,10,'Ninen','nomatch');",
+
+            "CREATE TABLE collectiontable(m text PRIMARY KEY, n map<text, text>);",
+            "UPDATE collectiontable SET n['key1'] = 'value1' WHERE m = 'book1';",
+            "UPDATE collectiontable SET n['key2'] = 'value2' WHERE m = 'book2';",
+            "UPDATE collectiontable SET n['key3'] = 'value3' WHERE m = 'book3';",
+            "UPDATE collectiontable SET n['key4'] = 'value4' WHERE m = 'book4';",
+    };
+
+    @BeforeClass
+    public static void setup() throws TTransportException, IOException, InterruptedException, ConfigurationException,
+                                      AuthenticationException, AuthorizationException, InvalidRequestException, UnavailableException, TimedOutException, TException, NotFoundException, CharacterCodingException, ClassNotFoundException, NoSuchFieldException, IllegalAccessException, InstantiationException
+    {
+        startCassandra();
+        setupDataByCql(statements);
+        startHadoopCluster();
+    }
+
+    @Test
+    public void testCqlStorageSchema()
+    throws AuthenticationException, AuthorizationException, InvalidRequestException, UnavailableException, TimedOutException, TException, NotFoundException, SchemaDisagreementException, IOException
+    {
+        pig.registerQuery("rows = LOAD 'cql://cql3ks/cqltable?" + defaultParameters + "' USING CqlStorage();");
+        Iterator<Tuple> it = pig.openIterator("rows");
+        if (it.hasNext()) {
+            Tuple t = it.next();
+            Assert.assertEquals(t.get(0).toString(), "key1");
+            Assert.assertEquals(t.get(1), 111);
+            Assert.assertEquals(t.get(2), 100);
+            Assert.assertEquals(t.get(3), 10.1f);
+            Assert.assertEquals(4, t.size());
+        }
+
+        pig.registerQuery("rows = LOAD 'cql://cql3ks/compactcqltable?" + defaultParameters + "' USING CqlStorage();");
+        it = pig.openIterator("rows");
+        if (it.hasNext()) {
+            Tuple t = it.next();
+            Assert.assertEquals(t.get(0).toString(), "key1");
+            Assert.assertEquals(t.get(1), 100);
+            Assert.assertEquals(t.get(2), 10.1f);
+            Assert.assertEquals(3, t.size());
+        }
+    }
+
+    @Test
+    public void testCqlStorageSingleKeyTable()
+    throws AuthenticationException, AuthorizationException, InvalidRequestException, UnavailableException, TimedOutException, TException, NotFoundException, SchemaDisagreementException, IOException
+    {
+        pig.setBatchOn();
+        pig.registerQuery("moretestvalues= LOAD 'cql://cql3ks/moredata?" + defaultParameters + "' USING CqlStorage();");
+        pig.registerQuery("insertformat= FOREACH moretestvalues GENERATE TOTUPLE(TOTUPLE('a',x)),TOTUPLE(y);");
+        pig.registerQuery("STORE insertformat INTO 'cql://cql3ks/test?" + defaultParameters + "&output_query=UPDATE+cql3ks.test+set+b+%3D+%3F' USING CqlStorage();");
+        pig.executeBatch();
+        //(5,5)
+        //(6,6)
+        //(4,4)
+        //(2,2)
+        //(3,3)
+        //(1,1)
+        pig.registerQuery("result= LOAD 'cql://cql3ks/test?" + defaultParameters + "' USING CqlStorage();");
+        Iterator<Tuple> it = pig.openIterator("result");
+        if (it.hasNext()) {
+            Tuple t = it.next();
+            Assert.assertEquals(t.get(0), t.get(1));
+        }
+    }
+
+    @Test
+    public void testCqlStorageCompositeKeyTable()
+    throws AuthenticationException, AuthorizationException, InvalidRequestException, UnavailableException, TimedOutException, TException, NotFoundException, SchemaDisagreementException, IOException
+    {
+        pig.setBatchOn();
+        pig.registerQuery("moredata= LOAD 'cql://cql3ks/compmore?" + defaultParameters + "' USING CqlStorage();");
+        pig.registerQuery("insertformat = FOREACH moredata GENERATE TOTUPLE (TOTUPLE('a',x),TOTUPLE('b',y), TOTUPLE('c',z)),TOTUPLE(data);");
+        pig.registerQuery("STORE insertformat INTO 'cql://cql3ks/compotable?" + defaultParameters + "&output_query=UPDATE%20cql3ks.compotable%20SET%20d%20%3D%20%3F' USING CqlStorage();");
+        pig.executeBatch();
+
+        //(5,6,Fix,nomatch)
+        //(3,3,Three,match)
+        //(1,1,One,match)
+        //(2,2,Two,match)
+        //(7,7,Seven,match)
+        //(8,8,Eight,match)
+        //(6,5,Sive,nomatch)
+        //(4,4,Four,match)
+        //(9,10,Ninen,nomatch)
+        pig.registerQuery("result= LOAD 'cql://cql3ks/compotable?" + defaultParameters + "' USING CqlStorage();");
+        Iterator<Tuple> it = pig.openIterator("result");
+        if (it.hasNext()) {
+            Tuple t = it.next();
+            Assert.assertEquals(t.get(3), "match");
+        }
+    }
+
+    @Test
+    public void testCqlStorageCollectionColumnTable()
+    throws AuthenticationException, AuthorizationException, InvalidRequestException, UnavailableException, TimedOutException, TException, NotFoundException, SchemaDisagreementException, IOException
+    {
+        pig.setBatchOn();
+        pig.registerQuery("collectiontable= LOAD 'cql://cql3ks/collectiontable?" + defaultParameters + "' USING CqlStorage();");
+        pig.registerQuery("recs= FOREACH collectiontable GENERATE TOTUPLE(TOTUPLE('m', m) ), TOTUPLE(TOTUPLE('map', TOTUPLE('m', 'mm'), TOTUPLE('n', 'nn')));");
+        pig.registerQuery("STORE recs INTO 'cql://cql3ks/collectiontable?" + defaultParameters + "&output_query=update+cql3ks.collectiontable+set+n+%3D+%3F' USING CqlStorage();");
+        pig.executeBatch();
+
+        //(book2,((m,mm),(n,nn)))
+        //(book3,((m,mm),(n,nn)))
+        //(book4,((m,mm),(n,nn)))
+        //(book1,((m,mm),(n,nn)))
+        pig.registerQuery("result= LOAD 'cql://cql3ks/collectiontable?" + defaultParameters + "' USING CqlStorage();");
+        Iterator<Tuple> it = pig.openIterator("result");
+        if (it.hasNext()) {
+            Tuple t = it.next();
+            Tuple t1 = (Tuple) t.get(1);
+            Assert.assertEquals(t1.size(), 2);
+            Tuple element1 = (Tuple) t1.get(0);
+            Tuple element2 = (Tuple) t1.get(1);
+            Assert.assertEquals(element1.get(0), "m");
+            Assert.assertEquals(element1.get(1), "mm");
+            Assert.assertEquals(element2.get(0), "n");
+            Assert.assertEquals(element2.get(1), "nn");
+        }
+    }
+
+    @Test
+    public void testCassandraStorageSchema() throws IOException, ClassNotFoundException, TException, TimedOutException, NotFoundException, InvalidRequestException, NoSuchFieldException, UnavailableException, IllegalAccessException, InstantiationException
+    {
+        //results: (key1,{((111,),),((111,column1),100),((111,column2),10.1)})
+        pig.registerQuery("rows = LOAD 'cassandra://cql3ks/cqltable?" + defaultParameters + "' USING CassandraStorage();");
+
+        //schema: {key: chararray,columns: {(name: (),value: bytearray)}}
+        Iterator<Tuple> it = pig.openIterator("rows");
+        if (it.hasNext()) {
+            Tuple t = it.next();
+            String rowKey =  t.get(0).toString();
+            Assert.assertEquals(rowKey, "key1");
+            DataBag columns = (DataBag) t.get(1);
+            Iterator<Tuple> iter = columns.iterator();
+            int i = 0;
+            while(iter.hasNext())
+            {
+                i++;
+                Tuple column = (Tuple) iter.next();
+                if (i==1)
+                {
+                    Assert.assertEquals(((Tuple) column.get(0)).get(0), 111);
+                    Assert.assertEquals(((Tuple) column.get(0)).get(1), "");
+                    Assert.assertEquals(column.get(1).toString(), "");
+                }
+                if (i==2)
+                {
+                    Assert.assertEquals(((Tuple) column.get(0)).get(0), 111);
+                    Assert.assertEquals(((Tuple) column.get(0)).get(1), "column1");
+                    Assert.assertEquals(column.get(1), 100);
+                }
+                if (i==3)
+                {
+                    Assert.assertEquals(((Tuple) column.get(0)).get(0), 111);
+                    Assert.assertEquals(((Tuple) column.get(0)).get(1), "column2");
+                    Assert.assertEquals(column.get(1), 10.1f);
+                }
+            }
+            Assert.assertEquals(3, columns.size());
+        }
+
+        //results: (key1,(column1,100),(column2,10.1))
+        pig.registerQuery("compact_rows = LOAD 'cassandra://cql3ks/compactcqltable?" + defaultParameters + "' USING CassandraStorage();");
+
+        //schema: {key: chararray,column1: (name: chararray,value: int),column2: (name: chararray,value: float)}
+        it = pig.openIterator("compact_rows");
+        if (it.hasNext()) {
+            Tuple t = it.next();
+            String rowKey =  t.get(0).toString();
+            Assert.assertEquals(rowKey, "key1");
+            Tuple column = (Tuple) t.get(1);
+            Assert.assertEquals(column.get(0), "column1");
+            Assert.assertEquals(column.get(1), 100);
+            column = (Tuple) t.get(2);
+            Assert.assertEquals(column.get(0), "column2");
+            Assert.assertEquals(column.get(1), 10.1f);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4e359697/test/unit/org/apache/cassandra/pig/PigTestBase.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/pig/PigTestBase.java b/test/unit/org/apache/cassandra/pig/PigTestBase.java
new file mode 100644
index 0000000..b5f146a
--- /dev/null
+++ b/test/unit/org/apache/cassandra/pig/PigTestBase.java
@@ -0,0 +1,167 @@
+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;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.exceptions.SyntaxException;
+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;
+import org.apache.pig.PigServer;
+import org.apache.pig.backend.hadoop.datastorage.ConfigurationUtil;
+import org.apache.pig.impl.PigContext;
+import org.apache.pig.test.MiniCluster;
+import org.apache.thrift.TException;
+import org.apache.thrift.protocol.TBinaryProtocol;
+import org.apache.thrift.protocol.TProtocol;
+import org.apache.thrift.transport.TFramedTransport;
+import org.apache.thrift.transport.TSocket;
+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
+{
+    protected static EmbeddedCassandraService cassandra;
+    protected static Configuration conf;
+    protected static MiniCluster cluster; 
+    protected static PigServer pig;
+    protected static String defaultParameters= "init_address=localhost&rpc_port=9170&partitioner=org.apache.cassandra.dht.ByteOrderedPartitioner";
+
+    @AfterClass
+    public static void oneTimeTearDown() throws Exception {
+        cluster.shutDown();
+    }
+
+    @Before
+    public void beforeTest() throws Exception {
+        pig = new PigServer(new PigContext(ExecType.LOCAL, ConfigurationUtil.toProperties(conf)));
+        PigContext.initializeImportList("org.apache.cassandra.hadoop.pig");   
+    }
+
+    @After
+    public void tearDown() throws Exception {
+        pig.shutdown();
+    }
+
+    protected static Cassandra.Client getClient() throws TTransportException
+    {
+        TTransport tr = new TFramedTransport(new TSocket("localhost", DatabaseDescriptor.getRpcPort()));
+        TProtocol proto = new TBinaryProtocol(tr);
+        Cassandra.Client client = new Cassandra.Client(proto);
+        tr.open();
+        return client;
+    }
+
+    protected static void startCassandra() throws IOException
+    {
+        Schema.instance.clear(); // Schema are now written on disk and will be reloaded
+        cassandra = new EmbeddedCassandraService();
+        cassandra.start();
+    }
+
+    protected static void startHadoopCluster()
+    {
+        cluster = MiniCluster.buildCluster();
+        conf = cluster.getConfiguration();
+    }
+
+    protected AbstractType parseType(String type) throws IOException
+    {
+        try
+        {
+            return TypeParser.parse(type);
+        }
+        catch (ConfigurationException e)
+        {
+            throw new IOException(e);
+        }
+        catch (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
+    {
+        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();
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4e359697/test/unit/org/apache/cassandra/pig/ThriftColumnFamilyTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/pig/ThriftColumnFamilyTest.java b/test/unit/org/apache/cassandra/pig/ThriftColumnFamilyTest.java
new file mode 100644
index 0000000..9dfa9f9
--- /dev/null
+++ b/test/unit/org/apache/cassandra/pig/ThriftColumnFamilyTest.java
@@ -0,0 +1,809 @@
+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.ColumnParent;
+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;
+
+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;"
+    };
+
+    @BeforeClass
+    public static void setup() throws TTransportException, IOException, InterruptedException, ConfigurationException,
+                                      AuthenticationException, AuthorizationException, InvalidRequestException, UnavailableException, TimedOutException, TException, NotFoundException, CharacterCodingException, ClassNotFoundException, NoSuchFieldException, IllegalAccessException, InstantiationException
+    {
+        startCassandra();
+        setupDataByCli(statements);
+        startHadoopCluster();
+    }
+
+    @Test
+    public void testCqlStorage() throws IOException, ClassNotFoundException, TException, TimedOutException, NotFoundException, InvalidRequestException, NoSuchFieldException, UnavailableException, IllegalAccessException, InstantiationException, AuthenticationException, AuthorizationException
+    {
+        //regular thrift column families
+        pig.registerQuery("data = load 'cql://thriftKs/SomeApp?" + defaultParameters + "' using CqlStorage();");
+
+        //(bar,3.141592653589793,1335890877,User Bar,35.0,9,15000,like)
+        //(baz,1.61803399,1335890877,User Baz,95.3,3,512000,dislike)
+        //(foo,2.718281828459045,1335890877,User Foo,85.0,8,125000,like)
+        //(qux,0.660161815846869,1335890877,User Qux,64.7,2,12000,dislike)
+
+        //{key: chararray,atomic_weight: double,created: long,name: chararray,percent: float,rating: int,score: long,vote_type: chararray}
+        Iterator<Tuple> it = pig.openIterator("data");
+        int count = 0;
+        while (it.hasNext()) {
+            count ++;
+            Tuple t = it.next();
+            if (count == 1)
+            {
+                Assert.assertEquals(t.get(0), "bar");
+                Assert.assertEquals(t.get(1), 3.141592653589793d);
+                Assert.assertEquals(t.get(3), "User Bar");
+                Assert.assertEquals(t.get(4), 35.0f);
+                Assert.assertEquals(t.get(5), 9);
+                Assert.assertEquals(t.get(6), 15000L);
+                Assert.assertEquals(t.get(7), "like");
+            }
+            else if (count == 2)
+            {
+                Assert.assertEquals(t.get(0), "baz");
+                Assert.assertEquals(t.get(1), 1.61803399d);
+                Assert.assertEquals(t.get(3), "User Baz");
+                Assert.assertEquals(t.get(4), 95.3f);
+                Assert.assertEquals(t.get(5), 3);
+                Assert.assertEquals(t.get(6), 512000L);
+                Assert.assertEquals(t.get(7), "dislike");
+            }else if (count == 3)
+            {
+                Assert.assertEquals(t.get(0), "foo");
+                Assert.assertEquals(t.get(1), 2.718281828459045d);
+                Assert.assertEquals(t.get(3), "User Foo");
+                Assert.assertEquals(t.get(4), 85.0f);
+                Assert.assertEquals(t.get(5), 8);
+                Assert.assertEquals(t.get(6), 125000L);
+                Assert.assertEquals(t.get(7), "like");
+            }
+            else if (count == 4)
+            {
+                Assert.assertEquals(t.get(0), "qux");
+                Assert.assertEquals(t.get(1), 0.660161815846869d);
+                Assert.assertEquals(t.get(3), "User Qux");
+                Assert.assertEquals(t.get(4), 64.7f);
+                Assert.assertEquals(t.get(5), 2);
+                Assert.assertEquals(t.get(6), 12000L);
+                Assert.assertEquals(t.get(7), "dislike");
+            }
+        }
+        Assert.assertEquals(count, 4);
+
+        //Test counter colun family
+        pig.registerQuery("cc_data = load 'cql://thriftKs/CC?" + defaultParameters + "' using CqlStorage();");
+
+        //(chuck,fist,1)
+        //(chuck,kick,3)
+
+        // {key: chararray,column1: chararray,value: long}
+        it = pig.openIterator("cc_data");
+        count = 0;
+        while (it.hasNext()) {
+            count ++;
+            Tuple t = it.next();
+            if (count == 1)
+            {
+                Assert.assertEquals(t.get(0), "chuck");
+                Assert.assertEquals(t.get(1), "fist");
+                Assert.assertEquals(t.get(2), 1L);
+            }
+            else if (count == 2)
+            {
+                Assert.assertEquals(t.get(0), "chuck");
+                Assert.assertEquals(t.get(1), "kick");
+                Assert.assertEquals(t.get(2), 3L);
+            }
+        }
+        Assert.assertEquals(count, 2);
+
+        //Test composite column family
+        pig.registerQuery("compo_data = load 'cql://thriftKs/Compo?" + defaultParameters + "' using CqlStorage();");
+
+        //(kick,bruce,bruce,watch it, mate)
+        //(kick,bruce,lee,oww)
+        //(punch,bruce,bruce,hunh?)
+        //(punch,bruce,lee,ouch)
+
+        //{key: chararray,column1: chararray,column2: chararray,value: chararray}
+        it = pig.openIterator("compo_data");
+        count = 0;
+        while (it.hasNext()) {
+            count ++;
+            Tuple t = it.next();
+            if (count == 1)
+            {
+                Assert.assertEquals(t.get(0), "kick");
+                Assert.assertEquals(t.get(1), "bruce");
+                Assert.assertEquals(t.get(2), "bruce");
+                Assert.assertEquals(t.get(3), "watch it, mate");
+            }
+            else if (count == 2)
+            {
+                Assert.assertEquals(t.get(0), "kick");
+                Assert.assertEquals(t.get(1), "bruce");
+                Assert.assertEquals(t.get(2), "lee");
+                Assert.assertEquals(t.get(3), "oww");
+            }
+            else if (count == 3)
+            {
+                Assert.assertEquals(t.get(0), "punch");
+                Assert.assertEquals(t.get(1), "bruce");
+                Assert.assertEquals(t.get(2), "bruce");
+                Assert.assertEquals(t.get(3), "hunh?");
+            }
+            else if (count == 4)
+            {
+                Assert.assertEquals(t.get(0), "punch");
+                Assert.assertEquals(t.get(1), "bruce");
+                Assert.assertEquals(t.get(2), "lee");
+                Assert.assertEquals(t.get(3), "ouch");
+            }
+        }
+        Assert.assertEquals(count, 4);
+    }
+
+    @Test
+    public void testCassandraStorageSchema() throws IOException, ClassNotFoundException, TException, TimedOutException, NotFoundException, InvalidRequestException, NoSuchFieldException, UnavailableException, 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();");
+
+        //schema: {key: chararray,atomic_weight: (name: chararray,value: double),created: (name: chararray,value: long),
+        //name: (name: chararray,value: chararray),percent: (name: chararray,value: float),
+        //rating: (name: chararray,value: int),score: (name: chararray,value: long),
+        //vote_type: (name: chararray,value: chararray),columns: {(name: chararray,value: chararray)}}
+        Iterator<Tuple> it = pig.openIterator("rows");
+        int count = 0;
+        if (it.hasNext()) {
+            Tuple t = it.next();
+            String rowKey =  t.get(0).toString();
+            if ("qux".equals(rowKey))
+            {
+                Tuple column = (Tuple) t.get(1);
+                Assert.assertEquals(column.get(0), "atomic_weight");
+                Assert.assertEquals(column.get(1), 0.660161815846869d);
+                column = (Tuple) t.get(3);
+                Assert.assertEquals(column.get(0), "name");
+                Assert.assertEquals(column.get(1), "User Qux");
+                column = (Tuple) t.get(4);
+                Assert.assertEquals(column.get(0), "percent");
+                Assert.assertEquals(column.get(1), 64.7f);
+                column = (Tuple) t.get(5);
+                Assert.assertEquals(column.get(0), "rating");
+                Assert.assertEquals(column.get(1), 2);
+                column = (Tuple) t.get(6);
+                Assert.assertEquals(column.get(0), "score");
+                Assert.assertEquals(column.get(1), 12000L);
+                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
+    {
+        createColumnFamily("thriftKs", "CopyOfSomeApp", statements[3]);
+        pig.setBatchOn();
+        pig.registerQuery("rows = LOAD 'cassandra://thriftKs/SomeApp?" + defaultParameters + "' USING CassandraStorage();");
+        //full copy
+        pig.registerQuery("STORE rows INTO 'cassandra://thriftKs/CopyOfSomeApp?" + 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"));
+    }
+
+    @Test
+    public void testCassandraStorageSigleTupleCopy() throws IOException, ClassNotFoundException, TException, TimedOutException, NotFoundException, InvalidRequestException, NoSuchFieldException, UnavailableException, IllegalAccessException, InstantiationException, AuthenticationException, AuthorizationException
+    {
+        createColumnFamily("thriftKs", "CopyOfSomeApp", statements[3]);
+        pig.setBatchOn();
+        pig.registerQuery("rows = LOAD 'cassandra://thriftKs/SomeApp?" + 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.executeBatch();
+        String value = null;
+        try
+        {
+            value = getColumnValue("thriftKs", "CopyOfSomeApp", "name", "qux", "UTF8Type");
+        }
+        catch (NotFoundException e)
+        {
+            Assert.assertTrue(true);
+        }
+        if (value != null)
+            Assert.fail();
+        try
+        {
+            value = getColumnValue("thriftKs", "CopyOfSomeApp", "vote_type", "qux", "UTF8Type");
+        }
+        catch (NotFoundException e)
+        {
+            Assert.assertTrue(true);
+        }
+        if (value != null)
+            Assert.fail();
+        Assert.assertEquals("64.7", getColumnValue("thriftKs", "CopyOfSomeApp", "percent", "qux", "FloatType"));
+    }
+
+    @Test
+    public void testCassandraStorageBagOnlyCopy() throws IOException, ClassNotFoundException, TException, TimedOutException, NotFoundException, InvalidRequestException, NoSuchFieldException, UnavailableException, IllegalAccessException, InstantiationException, AuthenticationException, AuthorizationException
+    {
+        createColumnFamily("thriftKs", "CopyOfSomeApp", statements[3]);
+        pig.setBatchOn();
+        pig.registerQuery("rows = LOAD 'cassandra://thriftKs/SomeApp?" + 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.executeBatch();
+        String value = null;
+        try
+        {
+            value = getColumnValue("thriftKs", "CopyOfSomeApp", "name", "qux", "UTF8Type");
+        }
+        catch (NotFoundException e)
+        {
+            Assert.assertTrue(true);
+        }
+        if (value != null)
+            Assert.fail();
+        try
+        {
+            value = getColumnValue("thriftKs", "CopyOfSomeApp", "vote_type", "qux", "UTF8Type");
+        }
+        catch (NotFoundException e)
+        {
+            Assert.assertTrue(true);
+        }
+        if (value != null)
+            Assert.fail();
+        try
+        {
+            value = getColumnValue("thriftKs", "CopyOfSomeApp", "percent", "qux", "FloatType");
+        }
+        catch (NotFoundException e)
+        {
+            Assert.assertTrue(true);
+        }
+        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
+    {
+        createColumnFamily("thriftKs", "CopyOfSomeApp", statements[3]);
+        pig.setBatchOn();
+        pig.registerQuery("rows = LOAD 'cassandra://thriftKs/SomeApp?" + 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.executeBatch();
+
+        Assert.assertEquals("like", getColumnValue("thriftKs", "CopyOfSomeApp", "vote_type", "bar", "UTF8Type"));
+        Assert.assertEquals("like", getColumnValue("thriftKs", "CopyOfSomeApp", "vote_type", "foo", "UTF8Type"));
+        String value = null;
+        try
+        {
+            value = getColumnValue("thriftKs", "CopyOfSomeApp", "vote_type", "qux", "UTF8Type");
+        }
+        catch (NotFoundException e)
+        {
+            Assert.assertTrue(true);
+        }
+        if (value != null)
+            Assert.fail();
+        try
+        {
+            value = getColumnValue("thriftKs", "CopyOfSomeApp", "vote_type", "baz", "UTF8Type");
+        }
+        catch (NotFoundException e)
+        {
+            Assert.assertTrue(true);
+        }
+        if (value != null)
+            Assert.fail();
+
+        createColumnFamily("thriftKs", "CopyOfSomeApp", statements[3]);
+        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.executeBatch();
+        Assert.assertEquals("dislike", getColumnValue("thriftKs", "CopyOfSomeApp", "vote_type", "baz", "UTF8Type"));
+        Assert.assertEquals("dislike", getColumnValue("thriftKs", "CopyOfSomeApp", "vote_type", "qux", "UTF8Type"));
+        value = null;
+        try
+        {
+            value = getColumnValue("thriftKs", "CopyOfSomeApp", "vote_type", "bar", "UTF8Type");
+        }
+        catch (NotFoundException e)
+        {
+            Assert.assertTrue(true);
+        }
+        if (value != null)
+            Assert.fail();
+        try
+        {
+            value = getColumnValue("thriftKs", "CopyOfSomeApp", "vote_type", "foo", "UTF8Type");
+        }
+        catch (NotFoundException e)
+        {
+            Assert.assertTrue(true);
+        }
+        if (value != null)
+            Assert.fail();
+    }
+
+    @Test
+    public void testCassandraStorageJoin() throws IOException, ClassNotFoundException, TException, TimedOutException, NotFoundException, InvalidRequestException, NoSuchFieldException, UnavailableException, IllegalAccessException, InstantiationException, AuthenticationException, AuthorizationException
+    {
+        //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();");
+
+        //cast key to chararray
+        pig.registerQuery("b = foreach Bytes generate (chararray)key, columns;");
+
+        //key in Bytes is a bytearray, U8 chararray
+        //(foo,{(x,Z)},foo,{(x,Z)})
+        pig.registerQuery("a = join Bytes by key, U8 by key;");
+        Iterator<Tuple> it = pig.openIterator("a");
+        if (it.hasNext()) {
+            Tuple t = it.next();
+            Assert.assertEquals(t.get(0), new DataByteArray("foo".getBytes()));
+            DataBag columns = (DataBag) t.get(1);
+            Iterator<Tuple> iter = columns.iterator();
+            Tuple t1 = iter.next();
+            Assert.assertEquals(t1.get(0), "x");
+            Assert.assertEquals(t1.get(1), new DataByteArray("Z".getBytes()));
+            String column = (String) t.get(2);
+            Assert.assertEquals(column, "foo");
+            columns = (DataBag) t.get(3);
+            iter = columns.iterator();
+            Tuple t2 = iter.next();
+            Assert.assertEquals(t2.get(0), "x");
+            Assert.assertEquals(t2.get(1), new DataByteArray("Z".getBytes()));
+        }
+        //key should now be cast into a chararray
+        //(foo,{(x,Z)},foo,{(x,Z)})
+        pig.registerQuery("c = join b by (chararray)key, U8 by (chararray)key;");
+        it = pig.openIterator("c");
+        if (it.hasNext()) {
+            Tuple t = it.next();
+            Assert.assertEquals(t.get(0), "foo");
+            DataBag columns = (DataBag) t.get(1);
+            Iterator<Tuple> iter = columns.iterator();
+            Tuple t1 = iter.next();
+            Assert.assertEquals(t1.get(0), "x");
+            Assert.assertEquals(t1.get(1), new DataByteArray("Z".getBytes()));
+            String column = (String) t.get(2);
+            Assert.assertEquals(column, "foo");
+            columns = (DataBag) t.get(3);
+            iter = columns.iterator();
+            Tuple t2 = iter.next();
+            Assert.assertEquals(t2.get(0), "x");
+            Assert.assertEquals(t2.get(1), new DataByteArray("Z".getBytes()));
+        }
+    }
+
+    @Test
+    public void testCassandraStorageCounterCF() throws IOException, ClassNotFoundException, TException, TimedOutException, NotFoundException, InvalidRequestException, NoSuchFieldException, UnavailableException, IllegalAccessException, InstantiationException, AuthenticationException, AuthorizationException
+    {
+        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("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);
+        }
+    }
+
+    @Test
+    public void testCassandraStorageCompositeColumnCF() throws IOException, ClassNotFoundException, TException, TimedOutException, NotFoundException, InvalidRequestException, NoSuchFieldException, UnavailableException, IllegalAccessException, InstantiationException, AuthenticationException, AuthorizationException
+    {
+        //Test CompositeType
+        pig.registerQuery("compo = load 'cassandra://thriftKs/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');");
+
+        //(kick,(bruce,lee),oww)
+        //(punch,(bruce,lee),ouch)
+        Iterator<Tuple> it = pig.openIterator("lee");
+        int count = 0;
+        while (it.hasNext()) {
+            count ++;
+            Tuple t = it.next();
+            if (count == 1)
+                Assert.assertEquals(t.get(0), "kick");
+            else
+                Assert.assertEquals(t.get(0), "punch");
+            Tuple t1 = (Tuple) t.get(1);
+            Assert.assertEquals(t1.get(0), "bruce");
+            Assert.assertEquals(t1.get(1), "lee");
+            if (count == 1)
+                Assert.assertEquals(t.get(2), "oww");
+            else
+                Assert.assertEquals(t.get(2), "ouch");
+        }
+        Assert.assertEquals(count, 2);
+        pig.registerQuery("night = load 'cassandra://thriftKs/CompoInt?" + 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;");
+
+        //What happens at the darkest hour?
+        pig.registerQuery("darkest = filter night by hour > 2 and hour < 5;");
+
+        //(2.5,daddy?)
+        it = pig.openIterator("darkest");
+        if (it.hasNext()) {
+            Tuple t = it.next();
+            Assert.assertEquals(t.get(0), 2.5d);
+            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.executeBatch();
+        pig.registerQuery("compocopy_int_rows = LOAD 'cassandra://thriftKs/CompoIntCopy?" + defaultParameters + "' using CassandraStorage();");
+        //(clock,{((1,0),z),((1,30),zzzz),((2,30),daddy?),((6,30),coffee...)})
+        it = pig.openIterator("compocopy_int_rows");
+        count = 0;
+        if (it.hasNext()) {
+            Tuple t = it.next();
+            Assert.assertEquals(t.get(0), "clock");
+            DataBag columns = (DataBag) t.get(1);
+            Iterator<Tuple> iter = columns.iterator();
+            while (iter.hasNext())
+            {
+                count ++;
+                Tuple t1 = iter.next();
+                Tuple inner = (Tuple) t1.get(0);
+                if (count == 1)
+                {
+                    Assert.assertEquals(inner.get(0), 1L);
+                    Assert.assertEquals(inner.get(1), 0L);
+                    Assert.assertEquals(t1.get(1), "z");
+                }
+                else if (count == 2)
+                {
+                    Assert.assertEquals(inner.get(0), 1L);
+                    Assert.assertEquals(inner.get(1), 30L);
+                    Assert.assertEquals(t1.get(1), "zzzz");
+                }
+                else if (count == 3)
+                {
+                    Assert.assertEquals(inner.get(0), 2L);
+                    Assert.assertEquals(inner.get(1), 30L);
+                    Assert.assertEquals(t1.get(1), "daddy?");
+                }
+                else if (count == 4)
+                {
+                    Assert.assertEquals(inner.get(0), 6L);
+                    Assert.assertEquals(inner.get(1), 30L);
+                    Assert.assertEquals(t1.get(1), "coffee...");
+                }
+            }
+            Assert.assertEquals(count, 4);
+        }
+    }
+
+    @Test
+    public void testCassandraStorageCompositeKeyCF() throws IOException, ClassNotFoundException, TException, TimedOutException, NotFoundException, InvalidRequestException, NoSuchFieldException, UnavailableException, IllegalAccessException, InstantiationException, AuthenticationException, AuthorizationException
+    {
+        //Test CompositeKey
+        pig.registerQuery("compokeys = load 'cassandra://thriftKs/CompoKey?" + defaultParameters + "' using CassandraStorage();");
+        pig.registerQuery("compokeys = filter compokeys by key.$1 == 40;");
+        //((clock,40),{(6,coffee...)})
+        Iterator<Tuple> it = pig.openIterator("compokeys");
+        if (it.hasNext()) {
+            Tuple t = it.next();
+            Tuple key = (Tuple) t.get(0); 
+            Assert.assertEquals(key.get(0), "clock");
+            Assert.assertEquals(key.get(1), 40L);
+            DataBag columns = (DataBag) t.get(1);
+            Iterator<Tuple> iter = columns.iterator();
+            if (iter.hasNext())
+            {
+                Tuple t1 = iter.next();
+                Assert.assertEquals(t1.get(0), 6L);
+                Assert.assertEquals(t1.get(1), "coffee...");
+            }
+        }
+        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.executeBatch();
+        pig.registerQuery("compo_key_copy_rows = LOAD 'cassandra://thriftKs/CompoKeyCopy?" + defaultParameters + "' using CassandraStorage();");
+        //((clock,10),{(1,z)})
+        //((clock,20),{(1,zzzz)})
+        //((clock,30),{(2,daddy?)})
+        //((clock,40),{(6,coffee...)})
+        it = pig.openIterator("compo_key_copy_rows");
+        int count = 0;
+        while (it.hasNext()) {
+            Tuple t = it.next();
+            count ++;
+            if (count == 1)
+            {
+                Tuple key = (Tuple) t.get(0); 
+                Assert.assertEquals(key.get(0), "clock");
+                Assert.assertEquals(key.get(1), 10L);
+                DataBag columns = (DataBag) t.get(1);
+                Iterator<Tuple> iter = columns.iterator();
+                if (iter.hasNext())
+                {
+                    Tuple t1 = iter.next();
+                    Assert.assertEquals(t1.get(0), 1L);
+                    Assert.assertEquals(t1.get(1), "z");
+                }
+            }
+            else if (count == 2)
+            {
+                Tuple key = (Tuple) t.get(0); 
+                Assert.assertEquals(key.get(0), "clock");
+                Assert.assertEquals(key.get(1), 20L);
+                DataBag columns = (DataBag) t.get(1);
+                Iterator<Tuple> iter = columns.iterator();
+                if (iter.hasNext())
+                {
+                    Tuple t1 = iter.next();
+                    Assert.assertEquals(t1.get(0), 1L);
+                    Assert.assertEquals(t1.get(1), "zzzz");
+                }
+            }
+            else if (count == 3)
+            {
+                Tuple key = (Tuple) t.get(0); 
+                Assert.assertEquals(key.get(0), "clock");
+                Assert.assertEquals(key.get(1), 30L);
+                DataBag columns = (DataBag) t.get(1);
+                Iterator<Tuple> iter = columns.iterator();
+                if (iter.hasNext())
+                {
+                    Tuple t1 = iter.next();
+                    Assert.assertEquals(t1.get(0), 2L);
+                    Assert.assertEquals(t1.get(1), "daddy?");
+                }
+            }
+            else if (count == 4)
+            {
+                Tuple key = (Tuple) t.get(0); 
+                Assert.assertEquals(key.get(0), "clock");
+                Assert.assertEquals(key.get(1), 40L);
+                DataBag columns = (DataBag) t.get(1);
+                Iterator<Tuple> iter = columns.iterator();
+                if (iter.hasNext())
+                {
+                    Tuple t1 = iter.next();
+                    Assert.assertEquals(t1.get(0), 6L);
+                    Assert.assertEquals(t1.get(1), "coffee...");
+                }
+            }
+        }
+        Assert.assertEquals(count, 4);
+    }
+
+    private String getColumnValue(String ks, String cf, String colName, String key, String validator)
+    throws AuthenticationException, AuthorizationException, InvalidRequestException, UnavailableException, TimedOutException, TException, NotFoundException, IOException
+    {
+        Cassandra.Client client = getClient();
+        client.set_keyspace(ks);
+
+        ByteBuffer key_user_id = ByteBufferUtil.bytes(key);
+
+        long timestamp = System.currentTimeMillis();
+        ColumnPath cp = new ColumnPath(cf);
+        ColumnParent par = new ColumnParent(cf);
+        cp.column = ByteBufferUtil.bytes(colName);
+
+        // read
+        ColumnOrSuperColumn got = client.get(key_user_id, cp, ConsistencyLevel.ONE);
+        return parseType(validator).getString(got.getColumn().value);
+    }
+
+    private void createColumnFamily(String ks, String cf, String statement) throws CharacterCodingException, ClassNotFoundException, TException, TimedOutException, NotFoundException, InvalidRequestException, NoSuchFieldException, UnavailableException, IllegalAccessException, InstantiationException
+    {
+        CliMain.connect("127.0.0.1", 9170);
+        try
+        {
+            CliMain.processStatement("use " + ks + ";");
+            CliMain.processStatement("drop column family " + cf + ";");
+        }
+        catch (Exception e)
+        {
+        }
+        CliMain.processStatement(statement);
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4e359697/test/unit/org/apache/pig/test/MiniCluster.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/pig/test/MiniCluster.java b/test/unit/org/apache/pig/test/MiniCluster.java
new file mode 100644
index 0000000..3216392
--- /dev/null
+++ b/test/unit/org/apache/pig/test/MiniCluster.java
@@ -0,0 +1,78 @@
+/*
+ * 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.pig.test;
+
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.mapred.MiniMRCluster;
+
+public class MiniCluster extends MiniGenericCluster {
+    private MiniMRCluster m_mr = null;
+    public MiniCluster() {
+        super();
+    }
+
+    @Override
+    protected void setupMiniDfsAndMrClusters() {
+        try {
+            System.setProperty("hadoop.log.dir", "build/test/logs");
+            final int dataNodes = 4;     // There will be 4 data nodes
+            final int taskTrackers = 4;  // There will be 4 task tracker nodes
+
+            // Create the configuration hadoop-site.xml file
+            File conf_dir = new File("build/classes/");
+            conf_dir.mkdirs();
+            File conf_file = new File(conf_dir, "hadoop-site.xml");
+
+            conf_file.delete();
+
+            // Builds and starts the mini dfs and mapreduce clusters
+            Configuration config = new Configuration();
+            m_dfs = new MiniDFSCluster(config, dataNodes, true, null);
+            m_fileSys = m_dfs.getFileSystem();
+            m_mr = new MiniMRCluster(taskTrackers, m_fileSys.getUri().toString(), 1);
+
+            // Write the necessary config info to hadoop-site.xml
+            m_conf = m_mr.createJobConf();
+            m_conf.setInt("mapred.submit.replication", 2);
+            m_conf.set("dfs.datanode.address", "0.0.0.0:0");
+            m_conf.set("dfs.datanode.http.address", "0.0.0.0:0");
+            m_conf.set("mapred.map.max.attempts", "2");
+            m_conf.set("mapred.reduce.max.attempts", "2");
+            m_conf.set("pig.jobcontrol.sleep", "100");
+            m_conf.writeXml(new FileOutputStream(conf_file));
+
+            // Set the system properties needed by Pig
+            System.setProperty("cluster", m_conf.get("mapred.job.tracker"));
+            System.setProperty("namenode", m_conf.get("fs.default.name"));
+            System.setProperty("junit.hadoop.conf", conf_dir.getPath());
+        } catch (IOException e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+    @Override
+    protected void shutdownMiniMrClusters() {
+        if (m_mr != null) { m_mr.shutdown(); }
+            m_mr = null;
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4e359697/test/unit/org/apache/pig/test/MiniGenericCluster.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/pig/test/MiniGenericCluster.java b/test/unit/org/apache/pig/test/MiniGenericCluster.java
new file mode 100644
index 0000000..ac3f5bc
--- /dev/null
+++ b/test/unit/org/apache/pig/test/MiniGenericCluster.java
@@ -0,0 +1,122 @@
+/**
+ * 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.pig.test;
+
+import java.io.*;
+import java.util.Properties;
+
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.pig.backend.hadoop.datastorage.ConfigurationUtil;
+
+/**
+ * This class builds a single instance of itself with the Singleton 
+ * design pattern. While building the single instance, it sets up a 
+ * mini cluster that actually consists of a mini DFS cluster and a 
+ * mini MapReduce cluster on the local machine and also sets up the 
+ * environment for Pig to run on top of the mini cluster.
+ *
+ * This class is the base class for MiniCluster, which has slightly
+ * difference among different versions of hadoop. MiniCluster implementation
+ * is located in $PIG_HOME/shims.
+ */
+abstract public class MiniGenericCluster {
+    protected MiniDFSCluster m_dfs = null;
+    protected FileSystem m_fileSys = null;
+    protected Configuration m_conf = null;
+    
+    protected final static MiniCluster INSTANCE = new MiniCluster();
+    protected static boolean isSetup = true;
+    
+    protected MiniGenericCluster() {
+        setupMiniDfsAndMrClusters();
+    }
+    
+    abstract protected void setupMiniDfsAndMrClusters();
+    
+    /**
+     * Returns the single instance of class MiniClusterBuilder that
+     * represents the resouces for a mini dfs cluster and a mini 
+     * mapreduce cluster. 
+     */
+    public static MiniCluster buildCluster() {
+        if(! isSetup){
+            INSTANCE.setupMiniDfsAndMrClusters();
+            isSetup = true;
+        }
+        return INSTANCE;
+    }
+
+    public void shutDown(){
+        INSTANCE.shutdownMiniDfsAndMrClusters();
+    }
+    
+    protected void finalize() {
+        shutdownMiniDfsAndMrClusters();
+    }
+    
+    protected void shutdownMiniDfsAndMrClusters() {
+        isSetup = false;
+        shutdownMiniDfsClusters();
+        shutdownMiniMrClusters();
+    }
+    
+    protected void shutdownMiniDfsClusters() {
+        try {
+            if (m_fileSys != null) { m_fileSys.close(); }
+        } catch (IOException e) {
+            e.printStackTrace();
+        }
+        if (m_dfs != null) { m_dfs.shutdown(); }
+        m_fileSys = null;
+        m_dfs = null;
+    }
+    
+    abstract protected void shutdownMiniMrClusters();
+
+    public Properties getProperties() {
+        errorIfNotSetup();
+        return ConfigurationUtil.toProperties(m_conf);
+    }
+
+    public Configuration getConfiguration() {
+        return new Configuration(m_conf);
+    }
+
+    public void setProperty(String name, String value) {
+        errorIfNotSetup();
+        m_conf.set(name, value);
+    }
+    
+    public FileSystem getFileSystem() {
+        errorIfNotSetup();
+        return m_fileSys;
+    }
+    
+    /**
+     * Throw RunTimeException if isSetup is false
+     */
+    private void errorIfNotSetup(){
+        if(isSetup)
+            return;
+        String msg = "function called on MiniCluster that has been shutdown";
+        throw new RuntimeException(msg);
+    }
+}


[3/6] git commit: Add PigUnit unit tests. Patch by Alex Liu, reviewed by brandonwilliams for CASSANDRA-5695

Posted by br...@apache.org.
Add PigUnit unit tests.
Patch by Alex Liu, reviewed by brandonwilliams for CASSANDRA-5695


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

Branch: refs/heads/trunk
Commit: 4e35969795793159075cd9368f3dd595b265584f
Parents: 17ae042
Author: Brandon Williams <br...@apache.org>
Authored: Mon Oct 14 14:13:18 2013 -0500
Committer: Brandon Williams <br...@apache.org>
Committed: Mon Oct 14 14:13:18 2013 -0500

----------------------------------------------------------------------
 build.xml                                       |   9 +
 .../hadoop/pig/AbstractCassandraStorage.java    |   2 +
 .../cassandra/hadoop/pig/CassandraStorage.java  |  25 +-
 .../apache/cassandra/hadoop/pig/CqlStorage.java |  25 +-
 test/conf/log4j-junit.properties                |   3 +
 .../org/apache/cassandra/pig/CqlTableTest.java  | 236 ++++++
 .../org/apache/cassandra/pig/PigTestBase.java   | 167 ++++
 .../cassandra/pig/ThriftColumnFamilyTest.java   | 809 +++++++++++++++++++
 test/unit/org/apache/pig/test/MiniCluster.java  |  78 ++
 .../org/apache/pig/test/MiniGenericCluster.java | 122 +++
 10 files changed, 1470 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/4e359697/build.xml
----------------------------------------------------------------------
diff --git a/build.xml b/build.xml
index 74c7374..37109c8 100644
--- a/build.xml
+++ b/build.xml
@@ -370,6 +370,7 @@
              <exclusion groupId="commons-lang" artifactId="commons-lang"/>
           </dependency>
           <dependency groupId="org.apache.hadoop" artifactId="hadoop-core" version="1.0.3"/>
+          <dependency groupId="org.apache.hadoop" artifactId="hadoop-minicluster" version="1.0.3"/>
           <dependency groupId="org.apache.pig" artifactId="pig" version="0.10.0"/>
           <dependency groupId="net.sf.jopt-simple" artifactId="jopt-simple" version="3.2"/>
           <dependency groupId="net.java.dev.jna" artifactId="jna" version="3.2.7"/>
@@ -417,6 +418,7 @@
         <dependency groupId="commons-logging" artifactId="commons-logging"/>
         <dependency groupId="org.apache.rat" artifactId="apache-rat"/>
         <dependency groupId="org.apache.hadoop" artifactId="hadoop-core"/>
+      	<dependency groupId="org.apache.hadoop" artifactId="hadoop-minicluster"/>
         <dependency groupId="org.apache.pig" artifactId="pig"/>
         <dependency groupId="net.sf.jopt-simple" artifactId="jopt-simple"/>
 
@@ -481,6 +483,7 @@
         
         <!-- don't need hadoop classes to run, but if you use the hadoop stuff -->
         <dependency groupId="org.apache.hadoop" artifactId="hadoop-core" optional="true"/>
+        <dependency groupId="org.apache.hadoop" artifactId="hadoop-minicluster" optional="true"/>
         <dependency groupId="org.apache.pig" artifactId="pig" optional="true"/>
 
         <!-- don't need jna to run, but nice to have -->
@@ -1160,6 +1163,12 @@
     </testmacro>
   </target>
 
+  <target name="pig-test" depends="build-test" description="Excute Pig tests">
+    <testmacro suitename="unit" inputdir="${test.unit.src}" 
+               timeout="1200000" filter="**/pig/*Test.java">
+    </testmacro>
+  </target>
+
   <!-- instruments the classes to later create code coverage reports -->
   <target name="cobertura-instrument" depends="build,build-test">
     <taskdef resource="tasks.properties">

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4e359697/src/java/org/apache/cassandra/hadoop/pig/AbstractCassandraStorage.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/hadoop/pig/AbstractCassandraStorage.java b/src/java/org/apache/cassandra/hadoop/pig/AbstractCassandraStorage.java
index dbebfb5..a02e309 100644
--- a/src/java/org/apache/cassandra/hadoop/pig/AbstractCassandraStorage.java
+++ b/src/java/org/apache/cassandra/hadoop/pig/AbstractCassandraStorage.java
@@ -98,6 +98,8 @@ public abstract class AbstractCassandraStorage extends LoadFunc implements Store
     protected int splitSize = 64 * 1024;
     protected String partitionerClass;
     protected boolean usePartitionFilter = false;
+    protected String initHostAddress;
+    protected String rpcPort;
 
     public AbstractCassandraStorage()
     {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4e359697/src/java/org/apache/cassandra/hadoop/pig/CassandraStorage.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/hadoop/pig/CassandraStorage.java b/src/java/org/apache/cassandra/hadoop/pig/CassandraStorage.java
index a7cc1ad..0512a37 100644
--- a/src/java/org/apache/cassandra/hadoop/pig/CassandraStorage.java
+++ b/src/java/org/apache/cassandra/hadoop/pig/CassandraStorage.java
@@ -316,6 +316,10 @@ public class CassandraStorage extends AbstractCassandraStorage
             ConfigHelper.setInputSplitSize(conf, splitSize);
         if (partitionerClass!= null)
             ConfigHelper.setInputPartitioner(conf, partitionerClass);
+        if (rpcPort != null)
+            ConfigHelper.setInputRpcPort(conf, rpcPort);
+        if (initHostAddress != null)
+            ConfigHelper.setInputInitialAddress(conf, initHostAddress);
 
         ConfigHelper.setInputColumnFamily(conf, keyspace, column_family, widerows);
         setConnectionInformation();
@@ -346,6 +350,16 @@ public class CassandraStorage extends AbstractCassandraStorage
             ConfigHelper.setInputSplitSize(conf, splitSize);
         if (partitionerClass!= null)
             ConfigHelper.setOutputPartitioner(conf, partitionerClass);
+        if (rpcPort != null)
+        {
+            ConfigHelper.setOutputRpcPort(conf, rpcPort);
+            ConfigHelper.setInputRpcPort(conf, rpcPort);
+        }
+        if (initHostAddress != null)
+        {
+            ConfigHelper.setOutputInitialAddress(conf, initHostAddress);
+            ConfigHelper.setInputInitialAddress(conf, initHostAddress);
+        }
 
         ConfigHelper.setOutputColumnFamily(conf, keyspace, column_family);
         setConnectionInformation();
@@ -781,6 +795,10 @@ public class CassandraStorage extends AbstractCassandraStorage
                     splitSize = Integer.parseInt(urlQuery.get("split_size"));
                 if (urlQuery.containsKey("partitioner"))
                     partitionerClass = urlQuery.get("partitioner");
+                if (urlQuery.containsKey("init_address"))
+                    initHostAddress = urlQuery.get("init_address");
+                if (urlQuery.containsKey("rpc_port"))
+                    rpcPort = urlQuery.get("rpc_port");
             }
             String[] parts = urlParts[0].split("/+");
             String[] credentialsAndKeyspace = parts[1].split("@");
@@ -800,9 +818,10 @@ public class CassandraStorage extends AbstractCassandraStorage
         catch (Exception e)
         {
             throw new IOException("Expected 'cassandra://[username:password@]<keyspace>/<columnfamily>" +
-            		                        "[?slice_start=<start>&slice_end=<end>[&reversed=true][&limit=1]" +
-            		                        "[&allow_deletes=true][&widerows=true][&use_secondary=true]" +
-            		                        "[&comparator=<comparator>][&split_size=<size>][&partitioner=<partitioner>]]': " + e.getMessage());
+                    "[?slice_start=<start>&slice_end=<end>[&reversed=true][&limit=1]" +
+                    "[&allow_deletes=true][&widerows=true][&use_secondary=true]" +
+                    "[&comparator=<comparator>][&split_size=<size>][&partitioner=<partitioner>]" +
+                    "[&init_address=<host>][&rpc_port=<port>]]': " + e.getMessage());
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4e359697/src/java/org/apache/cassandra/hadoop/pig/CqlStorage.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/hadoop/pig/CqlStorage.java b/src/java/org/apache/cassandra/hadoop/pig/CqlStorage.java
index 7e1f56c..e306005 100644
--- a/src/java/org/apache/cassandra/hadoop/pig/CqlStorage.java
+++ b/src/java/org/apache/cassandra/hadoop/pig/CqlStorage.java
@@ -207,6 +207,10 @@ public class CqlStorage extends AbstractCassandraStorage
             ConfigHelper.setInputSplitSize(conf, splitSize);
         if (partitionerClass!= null)
             ConfigHelper.setInputPartitioner(conf, partitionerClass);
+        if (rpcPort != null)
+            ConfigHelper.setInputRpcPort(conf, rpcPort);
+        if (initHostAddress != null)
+            ConfigHelper.setInputInitialAddress(conf, initHostAddress);
 
         ConfigHelper.setInputColumnFamily(conf, keyspace, column_family);
         setConnectionInformation();
@@ -262,6 +266,16 @@ public class CqlStorage extends AbstractCassandraStorage
             ConfigHelper.setInputSplitSize(conf, splitSize);
         if (partitionerClass!= null)
             ConfigHelper.setOutputPartitioner(conf, partitionerClass);
+        if (rpcPort != null)
+        {
+            ConfigHelper.setOutputRpcPort(conf, rpcPort);
+            ConfigHelper.setInputRpcPort(conf, rpcPort);
+        }
+        if (initHostAddress != null)
+        {
+            ConfigHelper.setOutputInitialAddress(conf, initHostAddress);
+            ConfigHelper.setInputInitialAddress(conf, initHostAddress);
+        }
 
         ConfigHelper.setOutputColumnFamily(conf, keyspace, column_family);
         CqlConfigHelper.setOutputCql(conf, outputQuery);
@@ -633,7 +647,11 @@ public class CqlStorage extends AbstractCassandraStorage
                 if (urlQuery.containsKey("partitioner"))
                     partitionerClass = urlQuery.get("partitioner");
                 if (urlQuery.containsKey("use_secondary"))
-                    usePartitionFilter = Boolean.parseBoolean(urlQuery.get("use_secondary")); 
+                    usePartitionFilter = Boolean.parseBoolean(urlQuery.get("use_secondary"));
+                if (urlQuery.containsKey("init_address"))
+                    initHostAddress = urlQuery.get("init_address");
+                if (urlQuery.containsKey("rpc_port"))
+                    rpcPort = urlQuery.get("rpc_port");
             }
             String[] parts = urlParts[0].split("/+");
             String[] credentialsAndKeyspace = parts[1].split("@");
@@ -653,8 +671,9 @@ public class CqlStorage extends AbstractCassandraStorage
         catch (Exception e)
         {
             throw new IOException("Expected 'cql://[username:password@]<keyspace>/<columnfamily>" +
-            		                         "[?[page_size=<size>][&columns=<col1,col2>][&output_query=<prepared_statement>]" +
-            		                         "[&where_clause=<clause>][&split_size=<size>][&partitioner=<partitioner>][&use_secondary=true|false]]': " + e.getMessage());
+                    "[?[page_size=<size>][&columns=<col1,col2>][&output_query=<prepared_statement>]" +
+                    "[&where_clause=<clause>][&split_size=<size>][&partitioner=<partitioner>][&use_secondary=true|false]" +
+                    "[&init_address=<host>][&rpc_port=<port>]]': " + e.getMessage());
         }
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4e359697/test/conf/log4j-junit.properties
----------------------------------------------------------------------
diff --git a/test/conf/log4j-junit.properties b/test/conf/log4j-junit.properties
index 7c73c54..3bc0c1a 100644
--- a/test/conf/log4j-junit.properties
+++ b/test/conf/log4j-junit.properties
@@ -35,3 +35,6 @@ log4j.appender.R.layout=org.apache.log4j.PatternLayout
 log4j.appender.R.layout.ConversionPattern=%5p [%t] %d{ISO8601} %F (line %L) %m%n
 # Edit the next line to point to your logs directory
 log4j.appender.R.File=build/test/logs/system.log
+
+log4j.logger.org.apache.hadoop=ERROR
+

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4e359697/test/unit/org/apache/cassandra/pig/CqlTableTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/pig/CqlTableTest.java b/test/unit/org/apache/cassandra/pig/CqlTableTest.java
new file mode 100644
index 0000000..e9266c9
--- /dev/null
+++ b/test/unit/org/apache/cassandra/pig/CqlTableTest.java
@@ -0,0 +1,236 @@
+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;
+
+public class CqlTableTest extends PigTestBase
+{    
+    private static String[] statements = {
+            "CREATE KEYSPACE cql3ks WITH replication = {'class': 'SimpleStrategy', 'replication_factor' : 1}",
+            "USE cql3ks;",
+
+            "CREATE TABLE cqltable (key1 text, key2 int, column1 int, column2 float, primary key(key1, key2))",
+            "INSERT INTO cqltable (key1, key2, column1, column2) values ('key1', 111, 100, 10.1)",
+            "CREATE TABLE compactcqltable (key1 text, column1 int, column2 float, primary key(key1)) WITH COMPACT STORAGE",
+            "INSERT INTO compactcqltable (key1, column1, column2) values ('key1', 100, 10.1)",
+
+            "CREATE TABLE test (a int PRIMARY KEY, b int);",
+
+            "CREATE TABLE moredata (x int PRIMARY KEY, y int);",
+            "INSERT INTO test (a,b) VALUES (1,1);",
+            "INSERT INTO test (a,b) VALUES (2,2);",
+            "INSERT INTO test (a,b) VALUES (3,3);",
+            "INSERT INTO moredata (x, y) VALUES (4,4);",
+            "INSERT INTO moredata (x, y) VALUES (5,5);",
+            "INSERT INTO moredata (x, y) VALUES (6,6);",
+
+            "CREATE TABLE compotable (a int, b int, c text, d text, PRIMARY KEY (a,b,c));",
+            "INSERT INTO compotable (a, b , c , d ) VALUES ( 1,1,'One','match');",
+            "INSERT INTO compotable (a, b , c , d ) VALUES ( 2,2,'Two','match');",
+            "INSERT INTO compotable (a, b , c , d ) VALUES ( 3,3,'Three','match');",
+            "INSERT INTO compotable (a, b , c , d ) VALUES ( 4,4,'Four','match');",
+
+            "create table compmore (id int PRIMARY KEY, x int, y int, z text, data text);",
+            "INSERT INTO compmore (id, x, y, z,data) VALUES (1,5,6,'Fix','nomatch');",
+            "INSERT INTO compmore (id, x, y, z,data) VALUES (2,6,5,'Sive','nomatch');",
+            "INSERT INTO compmore (id, x, y, z,data) VALUES (3,7,7,'Seven','match');",
+            "INSERT INTO compmore (id, x, y, z,data) VALUES (4,8,8,'Eight','match');",
+            "INSERT INTO compmore (id, x, y, z,data) VALUES (5,9,10,'Ninen','nomatch');",
+
+            "CREATE TABLE collectiontable(m text PRIMARY KEY, n map<text, text>);",
+            "UPDATE collectiontable SET n['key1'] = 'value1' WHERE m = 'book1';",
+            "UPDATE collectiontable SET n['key2'] = 'value2' WHERE m = 'book2';",
+            "UPDATE collectiontable SET n['key3'] = 'value3' WHERE m = 'book3';",
+            "UPDATE collectiontable SET n['key4'] = 'value4' WHERE m = 'book4';",
+    };
+
+    @BeforeClass
+    public static void setup() throws TTransportException, IOException, InterruptedException, ConfigurationException,
+                                      AuthenticationException, AuthorizationException, InvalidRequestException, UnavailableException, TimedOutException, TException, NotFoundException, CharacterCodingException, ClassNotFoundException, NoSuchFieldException, IllegalAccessException, InstantiationException
+    {
+        startCassandra();
+        setupDataByCql(statements);
+        startHadoopCluster();
+    }
+
+    @Test
+    public void testCqlStorageSchema()
+    throws AuthenticationException, AuthorizationException, InvalidRequestException, UnavailableException, TimedOutException, TException, NotFoundException, SchemaDisagreementException, IOException
+    {
+        pig.registerQuery("rows = LOAD 'cql://cql3ks/cqltable?" + defaultParameters + "' USING CqlStorage();");
+        Iterator<Tuple> it = pig.openIterator("rows");
+        if (it.hasNext()) {
+            Tuple t = it.next();
+            Assert.assertEquals(t.get(0).toString(), "key1");
+            Assert.assertEquals(t.get(1), 111);
+            Assert.assertEquals(t.get(2), 100);
+            Assert.assertEquals(t.get(3), 10.1f);
+            Assert.assertEquals(4, t.size());
+        }
+
+        pig.registerQuery("rows = LOAD 'cql://cql3ks/compactcqltable?" + defaultParameters + "' USING CqlStorage();");
+        it = pig.openIterator("rows");
+        if (it.hasNext()) {
+            Tuple t = it.next();
+            Assert.assertEquals(t.get(0).toString(), "key1");
+            Assert.assertEquals(t.get(1), 100);
+            Assert.assertEquals(t.get(2), 10.1f);
+            Assert.assertEquals(3, t.size());
+        }
+    }
+
+    @Test
+    public void testCqlStorageSingleKeyTable()
+    throws AuthenticationException, AuthorizationException, InvalidRequestException, UnavailableException, TimedOutException, TException, NotFoundException, SchemaDisagreementException, IOException
+    {
+        pig.setBatchOn();
+        pig.registerQuery("moretestvalues= LOAD 'cql://cql3ks/moredata?" + defaultParameters + "' USING CqlStorage();");
+        pig.registerQuery("insertformat= FOREACH moretestvalues GENERATE TOTUPLE(TOTUPLE('a',x)),TOTUPLE(y);");
+        pig.registerQuery("STORE insertformat INTO 'cql://cql3ks/test?" + defaultParameters + "&output_query=UPDATE+cql3ks.test+set+b+%3D+%3F' USING CqlStorage();");
+        pig.executeBatch();
+        //(5,5)
+        //(6,6)
+        //(4,4)
+        //(2,2)
+        //(3,3)
+        //(1,1)
+        pig.registerQuery("result= LOAD 'cql://cql3ks/test?" + defaultParameters + "' USING CqlStorage();");
+        Iterator<Tuple> it = pig.openIterator("result");
+        if (it.hasNext()) {
+            Tuple t = it.next();
+            Assert.assertEquals(t.get(0), t.get(1));
+        }
+    }
+
+    @Test
+    public void testCqlStorageCompositeKeyTable()
+    throws AuthenticationException, AuthorizationException, InvalidRequestException, UnavailableException, TimedOutException, TException, NotFoundException, SchemaDisagreementException, IOException
+    {
+        pig.setBatchOn();
+        pig.registerQuery("moredata= LOAD 'cql://cql3ks/compmore?" + defaultParameters + "' USING CqlStorage();");
+        pig.registerQuery("insertformat = FOREACH moredata GENERATE TOTUPLE (TOTUPLE('a',x),TOTUPLE('b',y), TOTUPLE('c',z)),TOTUPLE(data);");
+        pig.registerQuery("STORE insertformat INTO 'cql://cql3ks/compotable?" + defaultParameters + "&output_query=UPDATE%20cql3ks.compotable%20SET%20d%20%3D%20%3F' USING CqlStorage();");
+        pig.executeBatch();
+
+        //(5,6,Fix,nomatch)
+        //(3,3,Three,match)
+        //(1,1,One,match)
+        //(2,2,Two,match)
+        //(7,7,Seven,match)
+        //(8,8,Eight,match)
+        //(6,5,Sive,nomatch)
+        //(4,4,Four,match)
+        //(9,10,Ninen,nomatch)
+        pig.registerQuery("result= LOAD 'cql://cql3ks/compotable?" + defaultParameters + "' USING CqlStorage();");
+        Iterator<Tuple> it = pig.openIterator("result");
+        if (it.hasNext()) {
+            Tuple t = it.next();
+            Assert.assertEquals(t.get(3), "match");
+        }
+    }
+
+    @Test
+    public void testCqlStorageCollectionColumnTable()
+    throws AuthenticationException, AuthorizationException, InvalidRequestException, UnavailableException, TimedOutException, TException, NotFoundException, SchemaDisagreementException, IOException
+    {
+        pig.setBatchOn();
+        pig.registerQuery("collectiontable= LOAD 'cql://cql3ks/collectiontable?" + defaultParameters + "' USING CqlStorage();");
+        pig.registerQuery("recs= FOREACH collectiontable GENERATE TOTUPLE(TOTUPLE('m', m) ), TOTUPLE(TOTUPLE('map', TOTUPLE('m', 'mm'), TOTUPLE('n', 'nn')));");
+        pig.registerQuery("STORE recs INTO 'cql://cql3ks/collectiontable?" + defaultParameters + "&output_query=update+cql3ks.collectiontable+set+n+%3D+%3F' USING CqlStorage();");
+        pig.executeBatch();
+
+        //(book2,((m,mm),(n,nn)))
+        //(book3,((m,mm),(n,nn)))
+        //(book4,((m,mm),(n,nn)))
+        //(book1,((m,mm),(n,nn)))
+        pig.registerQuery("result= LOAD 'cql://cql3ks/collectiontable?" + defaultParameters + "' USING CqlStorage();");
+        Iterator<Tuple> it = pig.openIterator("result");
+        if (it.hasNext()) {
+            Tuple t = it.next();
+            Tuple t1 = (Tuple) t.get(1);
+            Assert.assertEquals(t1.size(), 2);
+            Tuple element1 = (Tuple) t1.get(0);
+            Tuple element2 = (Tuple) t1.get(1);
+            Assert.assertEquals(element1.get(0), "m");
+            Assert.assertEquals(element1.get(1), "mm");
+            Assert.assertEquals(element2.get(0), "n");
+            Assert.assertEquals(element2.get(1), "nn");
+        }
+    }
+
+    @Test
+    public void testCassandraStorageSchema() throws IOException, ClassNotFoundException, TException, TimedOutException, NotFoundException, InvalidRequestException, NoSuchFieldException, UnavailableException, IllegalAccessException, InstantiationException
+    {
+        //results: (key1,{((111,),),((111,column1),100),((111,column2),10.1)})
+        pig.registerQuery("rows = LOAD 'cassandra://cql3ks/cqltable?" + defaultParameters + "' USING CassandraStorage();");
+
+        //schema: {key: chararray,columns: {(name: (),value: bytearray)}}
+        Iterator<Tuple> it = pig.openIterator("rows");
+        if (it.hasNext()) {
+            Tuple t = it.next();
+            String rowKey =  t.get(0).toString();
+            Assert.assertEquals(rowKey, "key1");
+            DataBag columns = (DataBag) t.get(1);
+            Iterator<Tuple> iter = columns.iterator();
+            int i = 0;
+            while(iter.hasNext())
+            {
+                i++;
+                Tuple column = (Tuple) iter.next();
+                if (i==1)
+                {
+                    Assert.assertEquals(((Tuple) column.get(0)).get(0), 111);
+                    Assert.assertEquals(((Tuple) column.get(0)).get(1), "");
+                    Assert.assertEquals(column.get(1).toString(), "");
+                }
+                if (i==2)
+                {
+                    Assert.assertEquals(((Tuple) column.get(0)).get(0), 111);
+                    Assert.assertEquals(((Tuple) column.get(0)).get(1), "column1");
+                    Assert.assertEquals(column.get(1), 100);
+                }
+                if (i==3)
+                {
+                    Assert.assertEquals(((Tuple) column.get(0)).get(0), 111);
+                    Assert.assertEquals(((Tuple) column.get(0)).get(1), "column2");
+                    Assert.assertEquals(column.get(1), 10.1f);
+                }
+            }
+            Assert.assertEquals(3, columns.size());
+        }
+
+        //results: (key1,(column1,100),(column2,10.1))
+        pig.registerQuery("compact_rows = LOAD 'cassandra://cql3ks/compactcqltable?" + defaultParameters + "' USING CassandraStorage();");
+
+        //schema: {key: chararray,column1: (name: chararray,value: int),column2: (name: chararray,value: float)}
+        it = pig.openIterator("compact_rows");
+        if (it.hasNext()) {
+            Tuple t = it.next();
+            String rowKey =  t.get(0).toString();
+            Assert.assertEquals(rowKey, "key1");
+            Tuple column = (Tuple) t.get(1);
+            Assert.assertEquals(column.get(0), "column1");
+            Assert.assertEquals(column.get(1), 100);
+            column = (Tuple) t.get(2);
+            Assert.assertEquals(column.get(0), "column2");
+            Assert.assertEquals(column.get(1), 10.1f);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4e359697/test/unit/org/apache/cassandra/pig/PigTestBase.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/pig/PigTestBase.java b/test/unit/org/apache/cassandra/pig/PigTestBase.java
new file mode 100644
index 0000000..b5f146a
--- /dev/null
+++ b/test/unit/org/apache/cassandra/pig/PigTestBase.java
@@ -0,0 +1,167 @@
+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;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.exceptions.SyntaxException;
+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;
+import org.apache.pig.PigServer;
+import org.apache.pig.backend.hadoop.datastorage.ConfigurationUtil;
+import org.apache.pig.impl.PigContext;
+import org.apache.pig.test.MiniCluster;
+import org.apache.thrift.TException;
+import org.apache.thrift.protocol.TBinaryProtocol;
+import org.apache.thrift.protocol.TProtocol;
+import org.apache.thrift.transport.TFramedTransport;
+import org.apache.thrift.transport.TSocket;
+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
+{
+    protected static EmbeddedCassandraService cassandra;
+    protected static Configuration conf;
+    protected static MiniCluster cluster; 
+    protected static PigServer pig;
+    protected static String defaultParameters= "init_address=localhost&rpc_port=9170&partitioner=org.apache.cassandra.dht.ByteOrderedPartitioner";
+
+    @AfterClass
+    public static void oneTimeTearDown() throws Exception {
+        cluster.shutDown();
+    }
+
+    @Before
+    public void beforeTest() throws Exception {
+        pig = new PigServer(new PigContext(ExecType.LOCAL, ConfigurationUtil.toProperties(conf)));
+        PigContext.initializeImportList("org.apache.cassandra.hadoop.pig");   
+    }
+
+    @After
+    public void tearDown() throws Exception {
+        pig.shutdown();
+    }
+
+    protected static Cassandra.Client getClient() throws TTransportException
+    {
+        TTransport tr = new TFramedTransport(new TSocket("localhost", DatabaseDescriptor.getRpcPort()));
+        TProtocol proto = new TBinaryProtocol(tr);
+        Cassandra.Client client = new Cassandra.Client(proto);
+        tr.open();
+        return client;
+    }
+
+    protected static void startCassandra() throws IOException
+    {
+        Schema.instance.clear(); // Schema are now written on disk and will be reloaded
+        cassandra = new EmbeddedCassandraService();
+        cassandra.start();
+    }
+
+    protected static void startHadoopCluster()
+    {
+        cluster = MiniCluster.buildCluster();
+        conf = cluster.getConfiguration();
+    }
+
+    protected AbstractType parseType(String type) throws IOException
+    {
+        try
+        {
+            return TypeParser.parse(type);
+        }
+        catch (ConfigurationException e)
+        {
+            throw new IOException(e);
+        }
+        catch (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
+    {
+        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();
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4e359697/test/unit/org/apache/cassandra/pig/ThriftColumnFamilyTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/pig/ThriftColumnFamilyTest.java b/test/unit/org/apache/cassandra/pig/ThriftColumnFamilyTest.java
new file mode 100644
index 0000000..9dfa9f9
--- /dev/null
+++ b/test/unit/org/apache/cassandra/pig/ThriftColumnFamilyTest.java
@@ -0,0 +1,809 @@
+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.ColumnParent;
+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;
+
+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;"
+    };
+
+    @BeforeClass
+    public static void setup() throws TTransportException, IOException, InterruptedException, ConfigurationException,
+                                      AuthenticationException, AuthorizationException, InvalidRequestException, UnavailableException, TimedOutException, TException, NotFoundException, CharacterCodingException, ClassNotFoundException, NoSuchFieldException, IllegalAccessException, InstantiationException
+    {
+        startCassandra();
+        setupDataByCli(statements);
+        startHadoopCluster();
+    }
+
+    @Test
+    public void testCqlStorage() throws IOException, ClassNotFoundException, TException, TimedOutException, NotFoundException, InvalidRequestException, NoSuchFieldException, UnavailableException, IllegalAccessException, InstantiationException, AuthenticationException, AuthorizationException
+    {
+        //regular thrift column families
+        pig.registerQuery("data = load 'cql://thriftKs/SomeApp?" + defaultParameters + "' using CqlStorage();");
+
+        //(bar,3.141592653589793,1335890877,User Bar,35.0,9,15000,like)
+        //(baz,1.61803399,1335890877,User Baz,95.3,3,512000,dislike)
+        //(foo,2.718281828459045,1335890877,User Foo,85.0,8,125000,like)
+        //(qux,0.660161815846869,1335890877,User Qux,64.7,2,12000,dislike)
+
+        //{key: chararray,atomic_weight: double,created: long,name: chararray,percent: float,rating: int,score: long,vote_type: chararray}
+        Iterator<Tuple> it = pig.openIterator("data");
+        int count = 0;
+        while (it.hasNext()) {
+            count ++;
+            Tuple t = it.next();
+            if (count == 1)
+            {
+                Assert.assertEquals(t.get(0), "bar");
+                Assert.assertEquals(t.get(1), 3.141592653589793d);
+                Assert.assertEquals(t.get(3), "User Bar");
+                Assert.assertEquals(t.get(4), 35.0f);
+                Assert.assertEquals(t.get(5), 9);
+                Assert.assertEquals(t.get(6), 15000L);
+                Assert.assertEquals(t.get(7), "like");
+            }
+            else if (count == 2)
+            {
+                Assert.assertEquals(t.get(0), "baz");
+                Assert.assertEquals(t.get(1), 1.61803399d);
+                Assert.assertEquals(t.get(3), "User Baz");
+                Assert.assertEquals(t.get(4), 95.3f);
+                Assert.assertEquals(t.get(5), 3);
+                Assert.assertEquals(t.get(6), 512000L);
+                Assert.assertEquals(t.get(7), "dislike");
+            }else if (count == 3)
+            {
+                Assert.assertEquals(t.get(0), "foo");
+                Assert.assertEquals(t.get(1), 2.718281828459045d);
+                Assert.assertEquals(t.get(3), "User Foo");
+                Assert.assertEquals(t.get(4), 85.0f);
+                Assert.assertEquals(t.get(5), 8);
+                Assert.assertEquals(t.get(6), 125000L);
+                Assert.assertEquals(t.get(7), "like");
+            }
+            else if (count == 4)
+            {
+                Assert.assertEquals(t.get(0), "qux");
+                Assert.assertEquals(t.get(1), 0.660161815846869d);
+                Assert.assertEquals(t.get(3), "User Qux");
+                Assert.assertEquals(t.get(4), 64.7f);
+                Assert.assertEquals(t.get(5), 2);
+                Assert.assertEquals(t.get(6), 12000L);
+                Assert.assertEquals(t.get(7), "dislike");
+            }
+        }
+        Assert.assertEquals(count, 4);
+
+        //Test counter colun family
+        pig.registerQuery("cc_data = load 'cql://thriftKs/CC?" + defaultParameters + "' using CqlStorage();");
+
+        //(chuck,fist,1)
+        //(chuck,kick,3)
+
+        // {key: chararray,column1: chararray,value: long}
+        it = pig.openIterator("cc_data");
+        count = 0;
+        while (it.hasNext()) {
+            count ++;
+            Tuple t = it.next();
+            if (count == 1)
+            {
+                Assert.assertEquals(t.get(0), "chuck");
+                Assert.assertEquals(t.get(1), "fist");
+                Assert.assertEquals(t.get(2), 1L);
+            }
+            else if (count == 2)
+            {
+                Assert.assertEquals(t.get(0), "chuck");
+                Assert.assertEquals(t.get(1), "kick");
+                Assert.assertEquals(t.get(2), 3L);
+            }
+        }
+        Assert.assertEquals(count, 2);
+
+        //Test composite column family
+        pig.registerQuery("compo_data = load 'cql://thriftKs/Compo?" + defaultParameters + "' using CqlStorage();");
+
+        //(kick,bruce,bruce,watch it, mate)
+        //(kick,bruce,lee,oww)
+        //(punch,bruce,bruce,hunh?)
+        //(punch,bruce,lee,ouch)
+
+        //{key: chararray,column1: chararray,column2: chararray,value: chararray}
+        it = pig.openIterator("compo_data");
+        count = 0;
+        while (it.hasNext()) {
+            count ++;
+            Tuple t = it.next();
+            if (count == 1)
+            {
+                Assert.assertEquals(t.get(0), "kick");
+                Assert.assertEquals(t.get(1), "bruce");
+                Assert.assertEquals(t.get(2), "bruce");
+                Assert.assertEquals(t.get(3), "watch it, mate");
+            }
+            else if (count == 2)
+            {
+                Assert.assertEquals(t.get(0), "kick");
+                Assert.assertEquals(t.get(1), "bruce");
+                Assert.assertEquals(t.get(2), "lee");
+                Assert.assertEquals(t.get(3), "oww");
+            }
+            else if (count == 3)
+            {
+                Assert.assertEquals(t.get(0), "punch");
+                Assert.assertEquals(t.get(1), "bruce");
+                Assert.assertEquals(t.get(2), "bruce");
+                Assert.assertEquals(t.get(3), "hunh?");
+            }
+            else if (count == 4)
+            {
+                Assert.assertEquals(t.get(0), "punch");
+                Assert.assertEquals(t.get(1), "bruce");
+                Assert.assertEquals(t.get(2), "lee");
+                Assert.assertEquals(t.get(3), "ouch");
+            }
+        }
+        Assert.assertEquals(count, 4);
+    }
+
+    @Test
+    public void testCassandraStorageSchema() throws IOException, ClassNotFoundException, TException, TimedOutException, NotFoundException, InvalidRequestException, NoSuchFieldException, UnavailableException, 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();");
+
+        //schema: {key: chararray,atomic_weight: (name: chararray,value: double),created: (name: chararray,value: long),
+        //name: (name: chararray,value: chararray),percent: (name: chararray,value: float),
+        //rating: (name: chararray,value: int),score: (name: chararray,value: long),
+        //vote_type: (name: chararray,value: chararray),columns: {(name: chararray,value: chararray)}}
+        Iterator<Tuple> it = pig.openIterator("rows");
+        int count = 0;
+        if (it.hasNext()) {
+            Tuple t = it.next();
+            String rowKey =  t.get(0).toString();
+            if ("qux".equals(rowKey))
+            {
+                Tuple column = (Tuple) t.get(1);
+                Assert.assertEquals(column.get(0), "atomic_weight");
+                Assert.assertEquals(column.get(1), 0.660161815846869d);
+                column = (Tuple) t.get(3);
+                Assert.assertEquals(column.get(0), "name");
+                Assert.assertEquals(column.get(1), "User Qux");
+                column = (Tuple) t.get(4);
+                Assert.assertEquals(column.get(0), "percent");
+                Assert.assertEquals(column.get(1), 64.7f);
+                column = (Tuple) t.get(5);
+                Assert.assertEquals(column.get(0), "rating");
+                Assert.assertEquals(column.get(1), 2);
+                column = (Tuple) t.get(6);
+                Assert.assertEquals(column.get(0), "score");
+                Assert.assertEquals(column.get(1), 12000L);
+                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
+    {
+        createColumnFamily("thriftKs", "CopyOfSomeApp", statements[3]);
+        pig.setBatchOn();
+        pig.registerQuery("rows = LOAD 'cassandra://thriftKs/SomeApp?" + defaultParameters + "' USING CassandraStorage();");
+        //full copy
+        pig.registerQuery("STORE rows INTO 'cassandra://thriftKs/CopyOfSomeApp?" + 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"));
+    }
+
+    @Test
+    public void testCassandraStorageSigleTupleCopy() throws IOException, ClassNotFoundException, TException, TimedOutException, NotFoundException, InvalidRequestException, NoSuchFieldException, UnavailableException, IllegalAccessException, InstantiationException, AuthenticationException, AuthorizationException
+    {
+        createColumnFamily("thriftKs", "CopyOfSomeApp", statements[3]);
+        pig.setBatchOn();
+        pig.registerQuery("rows = LOAD 'cassandra://thriftKs/SomeApp?" + 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.executeBatch();
+        String value = null;
+        try
+        {
+            value = getColumnValue("thriftKs", "CopyOfSomeApp", "name", "qux", "UTF8Type");
+        }
+        catch (NotFoundException e)
+        {
+            Assert.assertTrue(true);
+        }
+        if (value != null)
+            Assert.fail();
+        try
+        {
+            value = getColumnValue("thriftKs", "CopyOfSomeApp", "vote_type", "qux", "UTF8Type");
+        }
+        catch (NotFoundException e)
+        {
+            Assert.assertTrue(true);
+        }
+        if (value != null)
+            Assert.fail();
+        Assert.assertEquals("64.7", getColumnValue("thriftKs", "CopyOfSomeApp", "percent", "qux", "FloatType"));
+    }
+
+    @Test
+    public void testCassandraStorageBagOnlyCopy() throws IOException, ClassNotFoundException, TException, TimedOutException, NotFoundException, InvalidRequestException, NoSuchFieldException, UnavailableException, IllegalAccessException, InstantiationException, AuthenticationException, AuthorizationException
+    {
+        createColumnFamily("thriftKs", "CopyOfSomeApp", statements[3]);
+        pig.setBatchOn();
+        pig.registerQuery("rows = LOAD 'cassandra://thriftKs/SomeApp?" + 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.executeBatch();
+        String value = null;
+        try
+        {
+            value = getColumnValue("thriftKs", "CopyOfSomeApp", "name", "qux", "UTF8Type");
+        }
+        catch (NotFoundException e)
+        {
+            Assert.assertTrue(true);
+        }
+        if (value != null)
+            Assert.fail();
+        try
+        {
+            value = getColumnValue("thriftKs", "CopyOfSomeApp", "vote_type", "qux", "UTF8Type");
+        }
+        catch (NotFoundException e)
+        {
+            Assert.assertTrue(true);
+        }
+        if (value != null)
+            Assert.fail();
+        try
+        {
+            value = getColumnValue("thriftKs", "CopyOfSomeApp", "percent", "qux", "FloatType");
+        }
+        catch (NotFoundException e)
+        {
+            Assert.assertTrue(true);
+        }
+        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
+    {
+        createColumnFamily("thriftKs", "CopyOfSomeApp", statements[3]);
+        pig.setBatchOn();
+        pig.registerQuery("rows = LOAD 'cassandra://thriftKs/SomeApp?" + 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.executeBatch();
+
+        Assert.assertEquals("like", getColumnValue("thriftKs", "CopyOfSomeApp", "vote_type", "bar", "UTF8Type"));
+        Assert.assertEquals("like", getColumnValue("thriftKs", "CopyOfSomeApp", "vote_type", "foo", "UTF8Type"));
+        String value = null;
+        try
+        {
+            value = getColumnValue("thriftKs", "CopyOfSomeApp", "vote_type", "qux", "UTF8Type");
+        }
+        catch (NotFoundException e)
+        {
+            Assert.assertTrue(true);
+        }
+        if (value != null)
+            Assert.fail();
+        try
+        {
+            value = getColumnValue("thriftKs", "CopyOfSomeApp", "vote_type", "baz", "UTF8Type");
+        }
+        catch (NotFoundException e)
+        {
+            Assert.assertTrue(true);
+        }
+        if (value != null)
+            Assert.fail();
+
+        createColumnFamily("thriftKs", "CopyOfSomeApp", statements[3]);
+        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.executeBatch();
+        Assert.assertEquals("dislike", getColumnValue("thriftKs", "CopyOfSomeApp", "vote_type", "baz", "UTF8Type"));
+        Assert.assertEquals("dislike", getColumnValue("thriftKs", "CopyOfSomeApp", "vote_type", "qux", "UTF8Type"));
+        value = null;
+        try
+        {
+            value = getColumnValue("thriftKs", "CopyOfSomeApp", "vote_type", "bar", "UTF8Type");
+        }
+        catch (NotFoundException e)
+        {
+            Assert.assertTrue(true);
+        }
+        if (value != null)
+            Assert.fail();
+        try
+        {
+            value = getColumnValue("thriftKs", "CopyOfSomeApp", "vote_type", "foo", "UTF8Type");
+        }
+        catch (NotFoundException e)
+        {
+            Assert.assertTrue(true);
+        }
+        if (value != null)
+            Assert.fail();
+    }
+
+    @Test
+    public void testCassandraStorageJoin() throws IOException, ClassNotFoundException, TException, TimedOutException, NotFoundException, InvalidRequestException, NoSuchFieldException, UnavailableException, IllegalAccessException, InstantiationException, AuthenticationException, AuthorizationException
+    {
+        //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();");
+
+        //cast key to chararray
+        pig.registerQuery("b = foreach Bytes generate (chararray)key, columns;");
+
+        //key in Bytes is a bytearray, U8 chararray
+        //(foo,{(x,Z)},foo,{(x,Z)})
+        pig.registerQuery("a = join Bytes by key, U8 by key;");
+        Iterator<Tuple> it = pig.openIterator("a");
+        if (it.hasNext()) {
+            Tuple t = it.next();
+            Assert.assertEquals(t.get(0), new DataByteArray("foo".getBytes()));
+            DataBag columns = (DataBag) t.get(1);
+            Iterator<Tuple> iter = columns.iterator();
+            Tuple t1 = iter.next();
+            Assert.assertEquals(t1.get(0), "x");
+            Assert.assertEquals(t1.get(1), new DataByteArray("Z".getBytes()));
+            String column = (String) t.get(2);
+            Assert.assertEquals(column, "foo");
+            columns = (DataBag) t.get(3);
+            iter = columns.iterator();
+            Tuple t2 = iter.next();
+            Assert.assertEquals(t2.get(0), "x");
+            Assert.assertEquals(t2.get(1), new DataByteArray("Z".getBytes()));
+        }
+        //key should now be cast into a chararray
+        //(foo,{(x,Z)},foo,{(x,Z)})
+        pig.registerQuery("c = join b by (chararray)key, U8 by (chararray)key;");
+        it = pig.openIterator("c");
+        if (it.hasNext()) {
+            Tuple t = it.next();
+            Assert.assertEquals(t.get(0), "foo");
+            DataBag columns = (DataBag) t.get(1);
+            Iterator<Tuple> iter = columns.iterator();
+            Tuple t1 = iter.next();
+            Assert.assertEquals(t1.get(0), "x");
+            Assert.assertEquals(t1.get(1), new DataByteArray("Z".getBytes()));
+            String column = (String) t.get(2);
+            Assert.assertEquals(column, "foo");
+            columns = (DataBag) t.get(3);
+            iter = columns.iterator();
+            Tuple t2 = iter.next();
+            Assert.assertEquals(t2.get(0), "x");
+            Assert.assertEquals(t2.get(1), new DataByteArray("Z".getBytes()));
+        }
+    }
+
+    @Test
+    public void testCassandraStorageCounterCF() throws IOException, ClassNotFoundException, TException, TimedOutException, NotFoundException, InvalidRequestException, NoSuchFieldException, UnavailableException, IllegalAccessException, InstantiationException, AuthenticationException, AuthorizationException
+    {
+        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("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);
+        }
+    }
+
+    @Test
+    public void testCassandraStorageCompositeColumnCF() throws IOException, ClassNotFoundException, TException, TimedOutException, NotFoundException, InvalidRequestException, NoSuchFieldException, UnavailableException, IllegalAccessException, InstantiationException, AuthenticationException, AuthorizationException
+    {
+        //Test CompositeType
+        pig.registerQuery("compo = load 'cassandra://thriftKs/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');");
+
+        //(kick,(bruce,lee),oww)
+        //(punch,(bruce,lee),ouch)
+        Iterator<Tuple> it = pig.openIterator("lee");
+        int count = 0;
+        while (it.hasNext()) {
+            count ++;
+            Tuple t = it.next();
+            if (count == 1)
+                Assert.assertEquals(t.get(0), "kick");
+            else
+                Assert.assertEquals(t.get(0), "punch");
+            Tuple t1 = (Tuple) t.get(1);
+            Assert.assertEquals(t1.get(0), "bruce");
+            Assert.assertEquals(t1.get(1), "lee");
+            if (count == 1)
+                Assert.assertEquals(t.get(2), "oww");
+            else
+                Assert.assertEquals(t.get(2), "ouch");
+        }
+        Assert.assertEquals(count, 2);
+        pig.registerQuery("night = load 'cassandra://thriftKs/CompoInt?" + 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;");
+
+        //What happens at the darkest hour?
+        pig.registerQuery("darkest = filter night by hour > 2 and hour < 5;");
+
+        //(2.5,daddy?)
+        it = pig.openIterator("darkest");
+        if (it.hasNext()) {
+            Tuple t = it.next();
+            Assert.assertEquals(t.get(0), 2.5d);
+            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.executeBatch();
+        pig.registerQuery("compocopy_int_rows = LOAD 'cassandra://thriftKs/CompoIntCopy?" + defaultParameters + "' using CassandraStorage();");
+        //(clock,{((1,0),z),((1,30),zzzz),((2,30),daddy?),((6,30),coffee...)})
+        it = pig.openIterator("compocopy_int_rows");
+        count = 0;
+        if (it.hasNext()) {
+            Tuple t = it.next();
+            Assert.assertEquals(t.get(0), "clock");
+            DataBag columns = (DataBag) t.get(1);
+            Iterator<Tuple> iter = columns.iterator();
+            while (iter.hasNext())
+            {
+                count ++;
+                Tuple t1 = iter.next();
+                Tuple inner = (Tuple) t1.get(0);
+                if (count == 1)
+                {
+                    Assert.assertEquals(inner.get(0), 1L);
+                    Assert.assertEquals(inner.get(1), 0L);
+                    Assert.assertEquals(t1.get(1), "z");
+                }
+                else if (count == 2)
+                {
+                    Assert.assertEquals(inner.get(0), 1L);
+                    Assert.assertEquals(inner.get(1), 30L);
+                    Assert.assertEquals(t1.get(1), "zzzz");
+                }
+                else if (count == 3)
+                {
+                    Assert.assertEquals(inner.get(0), 2L);
+                    Assert.assertEquals(inner.get(1), 30L);
+                    Assert.assertEquals(t1.get(1), "daddy?");
+                }
+                else if (count == 4)
+                {
+                    Assert.assertEquals(inner.get(0), 6L);
+                    Assert.assertEquals(inner.get(1), 30L);
+                    Assert.assertEquals(t1.get(1), "coffee...");
+                }
+            }
+            Assert.assertEquals(count, 4);
+        }
+    }
+
+    @Test
+    public void testCassandraStorageCompositeKeyCF() throws IOException, ClassNotFoundException, TException, TimedOutException, NotFoundException, InvalidRequestException, NoSuchFieldException, UnavailableException, IllegalAccessException, InstantiationException, AuthenticationException, AuthorizationException
+    {
+        //Test CompositeKey
+        pig.registerQuery("compokeys = load 'cassandra://thriftKs/CompoKey?" + defaultParameters + "' using CassandraStorage();");
+        pig.registerQuery("compokeys = filter compokeys by key.$1 == 40;");
+        //((clock,40),{(6,coffee...)})
+        Iterator<Tuple> it = pig.openIterator("compokeys");
+        if (it.hasNext()) {
+            Tuple t = it.next();
+            Tuple key = (Tuple) t.get(0); 
+            Assert.assertEquals(key.get(0), "clock");
+            Assert.assertEquals(key.get(1), 40L);
+            DataBag columns = (DataBag) t.get(1);
+            Iterator<Tuple> iter = columns.iterator();
+            if (iter.hasNext())
+            {
+                Tuple t1 = iter.next();
+                Assert.assertEquals(t1.get(0), 6L);
+                Assert.assertEquals(t1.get(1), "coffee...");
+            }
+        }
+        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.executeBatch();
+        pig.registerQuery("compo_key_copy_rows = LOAD 'cassandra://thriftKs/CompoKeyCopy?" + defaultParameters + "' using CassandraStorage();");
+        //((clock,10),{(1,z)})
+        //((clock,20),{(1,zzzz)})
+        //((clock,30),{(2,daddy?)})
+        //((clock,40),{(6,coffee...)})
+        it = pig.openIterator("compo_key_copy_rows");
+        int count = 0;
+        while (it.hasNext()) {
+            Tuple t = it.next();
+            count ++;
+            if (count == 1)
+            {
+                Tuple key = (Tuple) t.get(0); 
+                Assert.assertEquals(key.get(0), "clock");
+                Assert.assertEquals(key.get(1), 10L);
+                DataBag columns = (DataBag) t.get(1);
+                Iterator<Tuple> iter = columns.iterator();
+                if (iter.hasNext())
+                {
+                    Tuple t1 = iter.next();
+                    Assert.assertEquals(t1.get(0), 1L);
+                    Assert.assertEquals(t1.get(1), "z");
+                }
+            }
+            else if (count == 2)
+            {
+                Tuple key = (Tuple) t.get(0); 
+                Assert.assertEquals(key.get(0), "clock");
+                Assert.assertEquals(key.get(1), 20L);
+                DataBag columns = (DataBag) t.get(1);
+                Iterator<Tuple> iter = columns.iterator();
+                if (iter.hasNext())
+                {
+                    Tuple t1 = iter.next();
+                    Assert.assertEquals(t1.get(0), 1L);
+                    Assert.assertEquals(t1.get(1), "zzzz");
+                }
+            }
+            else if (count == 3)
+            {
+                Tuple key = (Tuple) t.get(0); 
+                Assert.assertEquals(key.get(0), "clock");
+                Assert.assertEquals(key.get(1), 30L);
+                DataBag columns = (DataBag) t.get(1);
+                Iterator<Tuple> iter = columns.iterator();
+                if (iter.hasNext())
+                {
+                    Tuple t1 = iter.next();
+                    Assert.assertEquals(t1.get(0), 2L);
+                    Assert.assertEquals(t1.get(1), "daddy?");
+                }
+            }
+            else if (count == 4)
+            {
+                Tuple key = (Tuple) t.get(0); 
+                Assert.assertEquals(key.get(0), "clock");
+                Assert.assertEquals(key.get(1), 40L);
+                DataBag columns = (DataBag) t.get(1);
+                Iterator<Tuple> iter = columns.iterator();
+                if (iter.hasNext())
+                {
+                    Tuple t1 = iter.next();
+                    Assert.assertEquals(t1.get(0), 6L);
+                    Assert.assertEquals(t1.get(1), "coffee...");
+                }
+            }
+        }
+        Assert.assertEquals(count, 4);
+    }
+
+    private String getColumnValue(String ks, String cf, String colName, String key, String validator)
+    throws AuthenticationException, AuthorizationException, InvalidRequestException, UnavailableException, TimedOutException, TException, NotFoundException, IOException
+    {
+        Cassandra.Client client = getClient();
+        client.set_keyspace(ks);
+
+        ByteBuffer key_user_id = ByteBufferUtil.bytes(key);
+
+        long timestamp = System.currentTimeMillis();
+        ColumnPath cp = new ColumnPath(cf);
+        ColumnParent par = new ColumnParent(cf);
+        cp.column = ByteBufferUtil.bytes(colName);
+
+        // read
+        ColumnOrSuperColumn got = client.get(key_user_id, cp, ConsistencyLevel.ONE);
+        return parseType(validator).getString(got.getColumn().value);
+    }
+
+    private void createColumnFamily(String ks, String cf, String statement) throws CharacterCodingException, ClassNotFoundException, TException, TimedOutException, NotFoundException, InvalidRequestException, NoSuchFieldException, UnavailableException, IllegalAccessException, InstantiationException
+    {
+        CliMain.connect("127.0.0.1", 9170);
+        try
+        {
+            CliMain.processStatement("use " + ks + ";");
+            CliMain.processStatement("drop column family " + cf + ";");
+        }
+        catch (Exception e)
+        {
+        }
+        CliMain.processStatement(statement);
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4e359697/test/unit/org/apache/pig/test/MiniCluster.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/pig/test/MiniCluster.java b/test/unit/org/apache/pig/test/MiniCluster.java
new file mode 100644
index 0000000..3216392
--- /dev/null
+++ b/test/unit/org/apache/pig/test/MiniCluster.java
@@ -0,0 +1,78 @@
+/*
+ * 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.pig.test;
+
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.mapred.MiniMRCluster;
+
+public class MiniCluster extends MiniGenericCluster {
+    private MiniMRCluster m_mr = null;
+    public MiniCluster() {
+        super();
+    }
+
+    @Override
+    protected void setupMiniDfsAndMrClusters() {
+        try {
+            System.setProperty("hadoop.log.dir", "build/test/logs");
+            final int dataNodes = 4;     // There will be 4 data nodes
+            final int taskTrackers = 4;  // There will be 4 task tracker nodes
+
+            // Create the configuration hadoop-site.xml file
+            File conf_dir = new File("build/classes/");
+            conf_dir.mkdirs();
+            File conf_file = new File(conf_dir, "hadoop-site.xml");
+
+            conf_file.delete();
+
+            // Builds and starts the mini dfs and mapreduce clusters
+            Configuration config = new Configuration();
+            m_dfs = new MiniDFSCluster(config, dataNodes, true, null);
+            m_fileSys = m_dfs.getFileSystem();
+            m_mr = new MiniMRCluster(taskTrackers, m_fileSys.getUri().toString(), 1);
+
+            // Write the necessary config info to hadoop-site.xml
+            m_conf = m_mr.createJobConf();
+            m_conf.setInt("mapred.submit.replication", 2);
+            m_conf.set("dfs.datanode.address", "0.0.0.0:0");
+            m_conf.set("dfs.datanode.http.address", "0.0.0.0:0");
+            m_conf.set("mapred.map.max.attempts", "2");
+            m_conf.set("mapred.reduce.max.attempts", "2");
+            m_conf.set("pig.jobcontrol.sleep", "100");
+            m_conf.writeXml(new FileOutputStream(conf_file));
+
+            // Set the system properties needed by Pig
+            System.setProperty("cluster", m_conf.get("mapred.job.tracker"));
+            System.setProperty("namenode", m_conf.get("fs.default.name"));
+            System.setProperty("junit.hadoop.conf", conf_dir.getPath());
+        } catch (IOException e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+    @Override
+    protected void shutdownMiniMrClusters() {
+        if (m_mr != null) { m_mr.shutdown(); }
+            m_mr = null;
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4e359697/test/unit/org/apache/pig/test/MiniGenericCluster.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/pig/test/MiniGenericCluster.java b/test/unit/org/apache/pig/test/MiniGenericCluster.java
new file mode 100644
index 0000000..ac3f5bc
--- /dev/null
+++ b/test/unit/org/apache/pig/test/MiniGenericCluster.java
@@ -0,0 +1,122 @@
+/**
+ * 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.pig.test;
+
+import java.io.*;
+import java.util.Properties;
+
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.pig.backend.hadoop.datastorage.ConfigurationUtil;
+
+/**
+ * This class builds a single instance of itself with the Singleton 
+ * design pattern. While building the single instance, it sets up a 
+ * mini cluster that actually consists of a mini DFS cluster and a 
+ * mini MapReduce cluster on the local machine and also sets up the 
+ * environment for Pig to run on top of the mini cluster.
+ *
+ * This class is the base class for MiniCluster, which has slightly
+ * difference among different versions of hadoop. MiniCluster implementation
+ * is located in $PIG_HOME/shims.
+ */
+abstract public class MiniGenericCluster {
+    protected MiniDFSCluster m_dfs = null;
+    protected FileSystem m_fileSys = null;
+    protected Configuration m_conf = null;
+    
+    protected final static MiniCluster INSTANCE = new MiniCluster();
+    protected static boolean isSetup = true;
+    
+    protected MiniGenericCluster() {
+        setupMiniDfsAndMrClusters();
+    }
+    
+    abstract protected void setupMiniDfsAndMrClusters();
+    
+    /**
+     * Returns the single instance of class MiniClusterBuilder that
+     * represents the resouces for a mini dfs cluster and a mini 
+     * mapreduce cluster. 
+     */
+    public static MiniCluster buildCluster() {
+        if(! isSetup){
+            INSTANCE.setupMiniDfsAndMrClusters();
+            isSetup = true;
+        }
+        return INSTANCE;
+    }
+
+    public void shutDown(){
+        INSTANCE.shutdownMiniDfsAndMrClusters();
+    }
+    
+    protected void finalize() {
+        shutdownMiniDfsAndMrClusters();
+    }
+    
+    protected void shutdownMiniDfsAndMrClusters() {
+        isSetup = false;
+        shutdownMiniDfsClusters();
+        shutdownMiniMrClusters();
+    }
+    
+    protected void shutdownMiniDfsClusters() {
+        try {
+            if (m_fileSys != null) { m_fileSys.close(); }
+        } catch (IOException e) {
+            e.printStackTrace();
+        }
+        if (m_dfs != null) { m_dfs.shutdown(); }
+        m_fileSys = null;
+        m_dfs = null;
+    }
+    
+    abstract protected void shutdownMiniMrClusters();
+
+    public Properties getProperties() {
+        errorIfNotSetup();
+        return ConfigurationUtil.toProperties(m_conf);
+    }
+
+    public Configuration getConfiguration() {
+        return new Configuration(m_conf);
+    }
+
+    public void setProperty(String name, String value) {
+        errorIfNotSetup();
+        m_conf.set(name, value);
+    }
+    
+    public FileSystem getFileSystem() {
+        errorIfNotSetup();
+        return m_fileSys;
+    }
+    
+    /**
+     * Throw RunTimeException if isSetup is false
+     */
+    private void errorIfNotSetup(){
+        if(isSetup)
+            return;
+        String msg = "function called on MiniCluster that has been shutdown";
+        throw new RuntimeException(msg);
+    }
+}


[2/6] git commit: Add PigUnit unit tests. Patch by Alex Liu, reviewed by brandonwilliams for CASSANDRA-5695

Posted by br...@apache.org.
Add PigUnit unit tests.
Patch by Alex Liu, reviewed by brandonwilliams for CASSANDRA-5695


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

Branch: refs/heads/cassandra-2.0
Commit: 4e35969795793159075cd9368f3dd595b265584f
Parents: 17ae042
Author: Brandon Williams <br...@apache.org>
Authored: Mon Oct 14 14:13:18 2013 -0500
Committer: Brandon Williams <br...@apache.org>
Committed: Mon Oct 14 14:13:18 2013 -0500

----------------------------------------------------------------------
 build.xml                                       |   9 +
 .../hadoop/pig/AbstractCassandraStorage.java    |   2 +
 .../cassandra/hadoop/pig/CassandraStorage.java  |  25 +-
 .../apache/cassandra/hadoop/pig/CqlStorage.java |  25 +-
 test/conf/log4j-junit.properties                |   3 +
 .../org/apache/cassandra/pig/CqlTableTest.java  | 236 ++++++
 .../org/apache/cassandra/pig/PigTestBase.java   | 167 ++++
 .../cassandra/pig/ThriftColumnFamilyTest.java   | 809 +++++++++++++++++++
 test/unit/org/apache/pig/test/MiniCluster.java  |  78 ++
 .../org/apache/pig/test/MiniGenericCluster.java | 122 +++
 10 files changed, 1470 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/4e359697/build.xml
----------------------------------------------------------------------
diff --git a/build.xml b/build.xml
index 74c7374..37109c8 100644
--- a/build.xml
+++ b/build.xml
@@ -370,6 +370,7 @@
              <exclusion groupId="commons-lang" artifactId="commons-lang"/>
           </dependency>
           <dependency groupId="org.apache.hadoop" artifactId="hadoop-core" version="1.0.3"/>
+          <dependency groupId="org.apache.hadoop" artifactId="hadoop-minicluster" version="1.0.3"/>
           <dependency groupId="org.apache.pig" artifactId="pig" version="0.10.0"/>
           <dependency groupId="net.sf.jopt-simple" artifactId="jopt-simple" version="3.2"/>
           <dependency groupId="net.java.dev.jna" artifactId="jna" version="3.2.7"/>
@@ -417,6 +418,7 @@
         <dependency groupId="commons-logging" artifactId="commons-logging"/>
         <dependency groupId="org.apache.rat" artifactId="apache-rat"/>
         <dependency groupId="org.apache.hadoop" artifactId="hadoop-core"/>
+      	<dependency groupId="org.apache.hadoop" artifactId="hadoop-minicluster"/>
         <dependency groupId="org.apache.pig" artifactId="pig"/>
         <dependency groupId="net.sf.jopt-simple" artifactId="jopt-simple"/>
 
@@ -481,6 +483,7 @@
         
         <!-- don't need hadoop classes to run, but if you use the hadoop stuff -->
         <dependency groupId="org.apache.hadoop" artifactId="hadoop-core" optional="true"/>
+        <dependency groupId="org.apache.hadoop" artifactId="hadoop-minicluster" optional="true"/>
         <dependency groupId="org.apache.pig" artifactId="pig" optional="true"/>
 
         <!-- don't need jna to run, but nice to have -->
@@ -1160,6 +1163,12 @@
     </testmacro>
   </target>
 
+  <target name="pig-test" depends="build-test" description="Excute Pig tests">
+    <testmacro suitename="unit" inputdir="${test.unit.src}" 
+               timeout="1200000" filter="**/pig/*Test.java">
+    </testmacro>
+  </target>
+
   <!-- instruments the classes to later create code coverage reports -->
   <target name="cobertura-instrument" depends="build,build-test">
     <taskdef resource="tasks.properties">

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4e359697/src/java/org/apache/cassandra/hadoop/pig/AbstractCassandraStorage.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/hadoop/pig/AbstractCassandraStorage.java b/src/java/org/apache/cassandra/hadoop/pig/AbstractCassandraStorage.java
index dbebfb5..a02e309 100644
--- a/src/java/org/apache/cassandra/hadoop/pig/AbstractCassandraStorage.java
+++ b/src/java/org/apache/cassandra/hadoop/pig/AbstractCassandraStorage.java
@@ -98,6 +98,8 @@ public abstract class AbstractCassandraStorage extends LoadFunc implements Store
     protected int splitSize = 64 * 1024;
     protected String partitionerClass;
     protected boolean usePartitionFilter = false;
+    protected String initHostAddress;
+    protected String rpcPort;
 
     public AbstractCassandraStorage()
     {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4e359697/src/java/org/apache/cassandra/hadoop/pig/CassandraStorage.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/hadoop/pig/CassandraStorage.java b/src/java/org/apache/cassandra/hadoop/pig/CassandraStorage.java
index a7cc1ad..0512a37 100644
--- a/src/java/org/apache/cassandra/hadoop/pig/CassandraStorage.java
+++ b/src/java/org/apache/cassandra/hadoop/pig/CassandraStorage.java
@@ -316,6 +316,10 @@ public class CassandraStorage extends AbstractCassandraStorage
             ConfigHelper.setInputSplitSize(conf, splitSize);
         if (partitionerClass!= null)
             ConfigHelper.setInputPartitioner(conf, partitionerClass);
+        if (rpcPort != null)
+            ConfigHelper.setInputRpcPort(conf, rpcPort);
+        if (initHostAddress != null)
+            ConfigHelper.setInputInitialAddress(conf, initHostAddress);
 
         ConfigHelper.setInputColumnFamily(conf, keyspace, column_family, widerows);
         setConnectionInformation();
@@ -346,6 +350,16 @@ public class CassandraStorage extends AbstractCassandraStorage
             ConfigHelper.setInputSplitSize(conf, splitSize);
         if (partitionerClass!= null)
             ConfigHelper.setOutputPartitioner(conf, partitionerClass);
+        if (rpcPort != null)
+        {
+            ConfigHelper.setOutputRpcPort(conf, rpcPort);
+            ConfigHelper.setInputRpcPort(conf, rpcPort);
+        }
+        if (initHostAddress != null)
+        {
+            ConfigHelper.setOutputInitialAddress(conf, initHostAddress);
+            ConfigHelper.setInputInitialAddress(conf, initHostAddress);
+        }
 
         ConfigHelper.setOutputColumnFamily(conf, keyspace, column_family);
         setConnectionInformation();
@@ -781,6 +795,10 @@ public class CassandraStorage extends AbstractCassandraStorage
                     splitSize = Integer.parseInt(urlQuery.get("split_size"));
                 if (urlQuery.containsKey("partitioner"))
                     partitionerClass = urlQuery.get("partitioner");
+                if (urlQuery.containsKey("init_address"))
+                    initHostAddress = urlQuery.get("init_address");
+                if (urlQuery.containsKey("rpc_port"))
+                    rpcPort = urlQuery.get("rpc_port");
             }
             String[] parts = urlParts[0].split("/+");
             String[] credentialsAndKeyspace = parts[1].split("@");
@@ -800,9 +818,10 @@ public class CassandraStorage extends AbstractCassandraStorage
         catch (Exception e)
         {
             throw new IOException("Expected 'cassandra://[username:password@]<keyspace>/<columnfamily>" +
-            		                        "[?slice_start=<start>&slice_end=<end>[&reversed=true][&limit=1]" +
-            		                        "[&allow_deletes=true][&widerows=true][&use_secondary=true]" +
-            		                        "[&comparator=<comparator>][&split_size=<size>][&partitioner=<partitioner>]]': " + e.getMessage());
+                    "[?slice_start=<start>&slice_end=<end>[&reversed=true][&limit=1]" +
+                    "[&allow_deletes=true][&widerows=true][&use_secondary=true]" +
+                    "[&comparator=<comparator>][&split_size=<size>][&partitioner=<partitioner>]" +
+                    "[&init_address=<host>][&rpc_port=<port>]]': " + e.getMessage());
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4e359697/src/java/org/apache/cassandra/hadoop/pig/CqlStorage.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/hadoop/pig/CqlStorage.java b/src/java/org/apache/cassandra/hadoop/pig/CqlStorage.java
index 7e1f56c..e306005 100644
--- a/src/java/org/apache/cassandra/hadoop/pig/CqlStorage.java
+++ b/src/java/org/apache/cassandra/hadoop/pig/CqlStorage.java
@@ -207,6 +207,10 @@ public class CqlStorage extends AbstractCassandraStorage
             ConfigHelper.setInputSplitSize(conf, splitSize);
         if (partitionerClass!= null)
             ConfigHelper.setInputPartitioner(conf, partitionerClass);
+        if (rpcPort != null)
+            ConfigHelper.setInputRpcPort(conf, rpcPort);
+        if (initHostAddress != null)
+            ConfigHelper.setInputInitialAddress(conf, initHostAddress);
 
         ConfigHelper.setInputColumnFamily(conf, keyspace, column_family);
         setConnectionInformation();
@@ -262,6 +266,16 @@ public class CqlStorage extends AbstractCassandraStorage
             ConfigHelper.setInputSplitSize(conf, splitSize);
         if (partitionerClass!= null)
             ConfigHelper.setOutputPartitioner(conf, partitionerClass);
+        if (rpcPort != null)
+        {
+            ConfigHelper.setOutputRpcPort(conf, rpcPort);
+            ConfigHelper.setInputRpcPort(conf, rpcPort);
+        }
+        if (initHostAddress != null)
+        {
+            ConfigHelper.setOutputInitialAddress(conf, initHostAddress);
+            ConfigHelper.setInputInitialAddress(conf, initHostAddress);
+        }
 
         ConfigHelper.setOutputColumnFamily(conf, keyspace, column_family);
         CqlConfigHelper.setOutputCql(conf, outputQuery);
@@ -633,7 +647,11 @@ public class CqlStorage extends AbstractCassandraStorage
                 if (urlQuery.containsKey("partitioner"))
                     partitionerClass = urlQuery.get("partitioner");
                 if (urlQuery.containsKey("use_secondary"))
-                    usePartitionFilter = Boolean.parseBoolean(urlQuery.get("use_secondary")); 
+                    usePartitionFilter = Boolean.parseBoolean(urlQuery.get("use_secondary"));
+                if (urlQuery.containsKey("init_address"))
+                    initHostAddress = urlQuery.get("init_address");
+                if (urlQuery.containsKey("rpc_port"))
+                    rpcPort = urlQuery.get("rpc_port");
             }
             String[] parts = urlParts[0].split("/+");
             String[] credentialsAndKeyspace = parts[1].split("@");
@@ -653,8 +671,9 @@ public class CqlStorage extends AbstractCassandraStorage
         catch (Exception e)
         {
             throw new IOException("Expected 'cql://[username:password@]<keyspace>/<columnfamily>" +
-            		                         "[?[page_size=<size>][&columns=<col1,col2>][&output_query=<prepared_statement>]" +
-            		                         "[&where_clause=<clause>][&split_size=<size>][&partitioner=<partitioner>][&use_secondary=true|false]]': " + e.getMessage());
+                    "[?[page_size=<size>][&columns=<col1,col2>][&output_query=<prepared_statement>]" +
+                    "[&where_clause=<clause>][&split_size=<size>][&partitioner=<partitioner>][&use_secondary=true|false]" +
+                    "[&init_address=<host>][&rpc_port=<port>]]': " + e.getMessage());
         }
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4e359697/test/conf/log4j-junit.properties
----------------------------------------------------------------------
diff --git a/test/conf/log4j-junit.properties b/test/conf/log4j-junit.properties
index 7c73c54..3bc0c1a 100644
--- a/test/conf/log4j-junit.properties
+++ b/test/conf/log4j-junit.properties
@@ -35,3 +35,6 @@ log4j.appender.R.layout=org.apache.log4j.PatternLayout
 log4j.appender.R.layout.ConversionPattern=%5p [%t] %d{ISO8601} %F (line %L) %m%n
 # Edit the next line to point to your logs directory
 log4j.appender.R.File=build/test/logs/system.log
+
+log4j.logger.org.apache.hadoop=ERROR
+

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4e359697/test/unit/org/apache/cassandra/pig/CqlTableTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/pig/CqlTableTest.java b/test/unit/org/apache/cassandra/pig/CqlTableTest.java
new file mode 100644
index 0000000..e9266c9
--- /dev/null
+++ b/test/unit/org/apache/cassandra/pig/CqlTableTest.java
@@ -0,0 +1,236 @@
+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;
+
+public class CqlTableTest extends PigTestBase
+{    
+    private static String[] statements = {
+            "CREATE KEYSPACE cql3ks WITH replication = {'class': 'SimpleStrategy', 'replication_factor' : 1}",
+            "USE cql3ks;",
+
+            "CREATE TABLE cqltable (key1 text, key2 int, column1 int, column2 float, primary key(key1, key2))",
+            "INSERT INTO cqltable (key1, key2, column1, column2) values ('key1', 111, 100, 10.1)",
+            "CREATE TABLE compactcqltable (key1 text, column1 int, column2 float, primary key(key1)) WITH COMPACT STORAGE",
+            "INSERT INTO compactcqltable (key1, column1, column2) values ('key1', 100, 10.1)",
+
+            "CREATE TABLE test (a int PRIMARY KEY, b int);",
+
+            "CREATE TABLE moredata (x int PRIMARY KEY, y int);",
+            "INSERT INTO test (a,b) VALUES (1,1);",
+            "INSERT INTO test (a,b) VALUES (2,2);",
+            "INSERT INTO test (a,b) VALUES (3,3);",
+            "INSERT INTO moredata (x, y) VALUES (4,4);",
+            "INSERT INTO moredata (x, y) VALUES (5,5);",
+            "INSERT INTO moredata (x, y) VALUES (6,6);",
+
+            "CREATE TABLE compotable (a int, b int, c text, d text, PRIMARY KEY (a,b,c));",
+            "INSERT INTO compotable (a, b , c , d ) VALUES ( 1,1,'One','match');",
+            "INSERT INTO compotable (a, b , c , d ) VALUES ( 2,2,'Two','match');",
+            "INSERT INTO compotable (a, b , c , d ) VALUES ( 3,3,'Three','match');",
+            "INSERT INTO compotable (a, b , c , d ) VALUES ( 4,4,'Four','match');",
+
+            "create table compmore (id int PRIMARY KEY, x int, y int, z text, data text);",
+            "INSERT INTO compmore (id, x, y, z,data) VALUES (1,5,6,'Fix','nomatch');",
+            "INSERT INTO compmore (id, x, y, z,data) VALUES (2,6,5,'Sive','nomatch');",
+            "INSERT INTO compmore (id, x, y, z,data) VALUES (3,7,7,'Seven','match');",
+            "INSERT INTO compmore (id, x, y, z,data) VALUES (4,8,8,'Eight','match');",
+            "INSERT INTO compmore (id, x, y, z,data) VALUES (5,9,10,'Ninen','nomatch');",
+
+            "CREATE TABLE collectiontable(m text PRIMARY KEY, n map<text, text>);",
+            "UPDATE collectiontable SET n['key1'] = 'value1' WHERE m = 'book1';",
+            "UPDATE collectiontable SET n['key2'] = 'value2' WHERE m = 'book2';",
+            "UPDATE collectiontable SET n['key3'] = 'value3' WHERE m = 'book3';",
+            "UPDATE collectiontable SET n['key4'] = 'value4' WHERE m = 'book4';",
+    };
+
+    @BeforeClass
+    public static void setup() throws TTransportException, IOException, InterruptedException, ConfigurationException,
+                                      AuthenticationException, AuthorizationException, InvalidRequestException, UnavailableException, TimedOutException, TException, NotFoundException, CharacterCodingException, ClassNotFoundException, NoSuchFieldException, IllegalAccessException, InstantiationException
+    {
+        startCassandra();
+        setupDataByCql(statements);
+        startHadoopCluster();
+    }
+
+    @Test
+    public void testCqlStorageSchema()
+    throws AuthenticationException, AuthorizationException, InvalidRequestException, UnavailableException, TimedOutException, TException, NotFoundException, SchemaDisagreementException, IOException
+    {
+        pig.registerQuery("rows = LOAD 'cql://cql3ks/cqltable?" + defaultParameters + "' USING CqlStorage();");
+        Iterator<Tuple> it = pig.openIterator("rows");
+        if (it.hasNext()) {
+            Tuple t = it.next();
+            Assert.assertEquals(t.get(0).toString(), "key1");
+            Assert.assertEquals(t.get(1), 111);
+            Assert.assertEquals(t.get(2), 100);
+            Assert.assertEquals(t.get(3), 10.1f);
+            Assert.assertEquals(4, t.size());
+        }
+
+        pig.registerQuery("rows = LOAD 'cql://cql3ks/compactcqltable?" + defaultParameters + "' USING CqlStorage();");
+        it = pig.openIterator("rows");
+        if (it.hasNext()) {
+            Tuple t = it.next();
+            Assert.assertEquals(t.get(0).toString(), "key1");
+            Assert.assertEquals(t.get(1), 100);
+            Assert.assertEquals(t.get(2), 10.1f);
+            Assert.assertEquals(3, t.size());
+        }
+    }
+
+    @Test
+    public void testCqlStorageSingleKeyTable()
+    throws AuthenticationException, AuthorizationException, InvalidRequestException, UnavailableException, TimedOutException, TException, NotFoundException, SchemaDisagreementException, IOException
+    {
+        pig.setBatchOn();
+        pig.registerQuery("moretestvalues= LOAD 'cql://cql3ks/moredata?" + defaultParameters + "' USING CqlStorage();");
+        pig.registerQuery("insertformat= FOREACH moretestvalues GENERATE TOTUPLE(TOTUPLE('a',x)),TOTUPLE(y);");
+        pig.registerQuery("STORE insertformat INTO 'cql://cql3ks/test?" + defaultParameters + "&output_query=UPDATE+cql3ks.test+set+b+%3D+%3F' USING CqlStorage();");
+        pig.executeBatch();
+        //(5,5)
+        //(6,6)
+        //(4,4)
+        //(2,2)
+        //(3,3)
+        //(1,1)
+        pig.registerQuery("result= LOAD 'cql://cql3ks/test?" + defaultParameters + "' USING CqlStorage();");
+        Iterator<Tuple> it = pig.openIterator("result");
+        if (it.hasNext()) {
+            Tuple t = it.next();
+            Assert.assertEquals(t.get(0), t.get(1));
+        }
+    }
+
+    @Test
+    public void testCqlStorageCompositeKeyTable()
+    throws AuthenticationException, AuthorizationException, InvalidRequestException, UnavailableException, TimedOutException, TException, NotFoundException, SchemaDisagreementException, IOException
+    {
+        pig.setBatchOn();
+        pig.registerQuery("moredata= LOAD 'cql://cql3ks/compmore?" + defaultParameters + "' USING CqlStorage();");
+        pig.registerQuery("insertformat = FOREACH moredata GENERATE TOTUPLE (TOTUPLE('a',x),TOTUPLE('b',y), TOTUPLE('c',z)),TOTUPLE(data);");
+        pig.registerQuery("STORE insertformat INTO 'cql://cql3ks/compotable?" + defaultParameters + "&output_query=UPDATE%20cql3ks.compotable%20SET%20d%20%3D%20%3F' USING CqlStorage();");
+        pig.executeBatch();
+
+        //(5,6,Fix,nomatch)
+        //(3,3,Three,match)
+        //(1,1,One,match)
+        //(2,2,Two,match)
+        //(7,7,Seven,match)
+        //(8,8,Eight,match)
+        //(6,5,Sive,nomatch)
+        //(4,4,Four,match)
+        //(9,10,Ninen,nomatch)
+        pig.registerQuery("result= LOAD 'cql://cql3ks/compotable?" + defaultParameters + "' USING CqlStorage();");
+        Iterator<Tuple> it = pig.openIterator("result");
+        if (it.hasNext()) {
+            Tuple t = it.next();
+            Assert.assertEquals(t.get(3), "match");
+        }
+    }
+
+    @Test
+    public void testCqlStorageCollectionColumnTable()
+    throws AuthenticationException, AuthorizationException, InvalidRequestException, UnavailableException, TimedOutException, TException, NotFoundException, SchemaDisagreementException, IOException
+    {
+        pig.setBatchOn();
+        pig.registerQuery("collectiontable= LOAD 'cql://cql3ks/collectiontable?" + defaultParameters + "' USING CqlStorage();");
+        pig.registerQuery("recs= FOREACH collectiontable GENERATE TOTUPLE(TOTUPLE('m', m) ), TOTUPLE(TOTUPLE('map', TOTUPLE('m', 'mm'), TOTUPLE('n', 'nn')));");
+        pig.registerQuery("STORE recs INTO 'cql://cql3ks/collectiontable?" + defaultParameters + "&output_query=update+cql3ks.collectiontable+set+n+%3D+%3F' USING CqlStorage();");
+        pig.executeBatch();
+
+        //(book2,((m,mm),(n,nn)))
+        //(book3,((m,mm),(n,nn)))
+        //(book4,((m,mm),(n,nn)))
+        //(book1,((m,mm),(n,nn)))
+        pig.registerQuery("result= LOAD 'cql://cql3ks/collectiontable?" + defaultParameters + "' USING CqlStorage();");
+        Iterator<Tuple> it = pig.openIterator("result");
+        if (it.hasNext()) {
+            Tuple t = it.next();
+            Tuple t1 = (Tuple) t.get(1);
+            Assert.assertEquals(t1.size(), 2);
+            Tuple element1 = (Tuple) t1.get(0);
+            Tuple element2 = (Tuple) t1.get(1);
+            Assert.assertEquals(element1.get(0), "m");
+            Assert.assertEquals(element1.get(1), "mm");
+            Assert.assertEquals(element2.get(0), "n");
+            Assert.assertEquals(element2.get(1), "nn");
+        }
+    }
+
+    @Test
+    public void testCassandraStorageSchema() throws IOException, ClassNotFoundException, TException, TimedOutException, NotFoundException, InvalidRequestException, NoSuchFieldException, UnavailableException, IllegalAccessException, InstantiationException
+    {
+        //results: (key1,{((111,),),((111,column1),100),((111,column2),10.1)})
+        pig.registerQuery("rows = LOAD 'cassandra://cql3ks/cqltable?" + defaultParameters + "' USING CassandraStorage();");
+
+        //schema: {key: chararray,columns: {(name: (),value: bytearray)}}
+        Iterator<Tuple> it = pig.openIterator("rows");
+        if (it.hasNext()) {
+            Tuple t = it.next();
+            String rowKey =  t.get(0).toString();
+            Assert.assertEquals(rowKey, "key1");
+            DataBag columns = (DataBag) t.get(1);
+            Iterator<Tuple> iter = columns.iterator();
+            int i = 0;
+            while(iter.hasNext())
+            {
+                i++;
+                Tuple column = (Tuple) iter.next();
+                if (i==1)
+                {
+                    Assert.assertEquals(((Tuple) column.get(0)).get(0), 111);
+                    Assert.assertEquals(((Tuple) column.get(0)).get(1), "");
+                    Assert.assertEquals(column.get(1).toString(), "");
+                }
+                if (i==2)
+                {
+                    Assert.assertEquals(((Tuple) column.get(0)).get(0), 111);
+                    Assert.assertEquals(((Tuple) column.get(0)).get(1), "column1");
+                    Assert.assertEquals(column.get(1), 100);
+                }
+                if (i==3)
+                {
+                    Assert.assertEquals(((Tuple) column.get(0)).get(0), 111);
+                    Assert.assertEquals(((Tuple) column.get(0)).get(1), "column2");
+                    Assert.assertEquals(column.get(1), 10.1f);
+                }
+            }
+            Assert.assertEquals(3, columns.size());
+        }
+
+        //results: (key1,(column1,100),(column2,10.1))
+        pig.registerQuery("compact_rows = LOAD 'cassandra://cql3ks/compactcqltable?" + defaultParameters + "' USING CassandraStorage();");
+
+        //schema: {key: chararray,column1: (name: chararray,value: int),column2: (name: chararray,value: float)}
+        it = pig.openIterator("compact_rows");
+        if (it.hasNext()) {
+            Tuple t = it.next();
+            String rowKey =  t.get(0).toString();
+            Assert.assertEquals(rowKey, "key1");
+            Tuple column = (Tuple) t.get(1);
+            Assert.assertEquals(column.get(0), "column1");
+            Assert.assertEquals(column.get(1), 100);
+            column = (Tuple) t.get(2);
+            Assert.assertEquals(column.get(0), "column2");
+            Assert.assertEquals(column.get(1), 10.1f);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4e359697/test/unit/org/apache/cassandra/pig/PigTestBase.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/pig/PigTestBase.java b/test/unit/org/apache/cassandra/pig/PigTestBase.java
new file mode 100644
index 0000000..b5f146a
--- /dev/null
+++ b/test/unit/org/apache/cassandra/pig/PigTestBase.java
@@ -0,0 +1,167 @@
+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;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.exceptions.SyntaxException;
+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;
+import org.apache.pig.PigServer;
+import org.apache.pig.backend.hadoop.datastorage.ConfigurationUtil;
+import org.apache.pig.impl.PigContext;
+import org.apache.pig.test.MiniCluster;
+import org.apache.thrift.TException;
+import org.apache.thrift.protocol.TBinaryProtocol;
+import org.apache.thrift.protocol.TProtocol;
+import org.apache.thrift.transport.TFramedTransport;
+import org.apache.thrift.transport.TSocket;
+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
+{
+    protected static EmbeddedCassandraService cassandra;
+    protected static Configuration conf;
+    protected static MiniCluster cluster; 
+    protected static PigServer pig;
+    protected static String defaultParameters= "init_address=localhost&rpc_port=9170&partitioner=org.apache.cassandra.dht.ByteOrderedPartitioner";
+
+    @AfterClass
+    public static void oneTimeTearDown() throws Exception {
+        cluster.shutDown();
+    }
+
+    @Before
+    public void beforeTest() throws Exception {
+        pig = new PigServer(new PigContext(ExecType.LOCAL, ConfigurationUtil.toProperties(conf)));
+        PigContext.initializeImportList("org.apache.cassandra.hadoop.pig");   
+    }
+
+    @After
+    public void tearDown() throws Exception {
+        pig.shutdown();
+    }
+
+    protected static Cassandra.Client getClient() throws TTransportException
+    {
+        TTransport tr = new TFramedTransport(new TSocket("localhost", DatabaseDescriptor.getRpcPort()));
+        TProtocol proto = new TBinaryProtocol(tr);
+        Cassandra.Client client = new Cassandra.Client(proto);
+        tr.open();
+        return client;
+    }
+
+    protected static void startCassandra() throws IOException
+    {
+        Schema.instance.clear(); // Schema are now written on disk and will be reloaded
+        cassandra = new EmbeddedCassandraService();
+        cassandra.start();
+    }
+
+    protected static void startHadoopCluster()
+    {
+        cluster = MiniCluster.buildCluster();
+        conf = cluster.getConfiguration();
+    }
+
+    protected AbstractType parseType(String type) throws IOException
+    {
+        try
+        {
+            return TypeParser.parse(type);
+        }
+        catch (ConfigurationException e)
+        {
+            throw new IOException(e);
+        }
+        catch (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
+    {
+        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();
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4e359697/test/unit/org/apache/cassandra/pig/ThriftColumnFamilyTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/pig/ThriftColumnFamilyTest.java b/test/unit/org/apache/cassandra/pig/ThriftColumnFamilyTest.java
new file mode 100644
index 0000000..9dfa9f9
--- /dev/null
+++ b/test/unit/org/apache/cassandra/pig/ThriftColumnFamilyTest.java
@@ -0,0 +1,809 @@
+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.ColumnParent;
+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;
+
+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;"
+    };
+
+    @BeforeClass
+    public static void setup() throws TTransportException, IOException, InterruptedException, ConfigurationException,
+                                      AuthenticationException, AuthorizationException, InvalidRequestException, UnavailableException, TimedOutException, TException, NotFoundException, CharacterCodingException, ClassNotFoundException, NoSuchFieldException, IllegalAccessException, InstantiationException
+    {
+        startCassandra();
+        setupDataByCli(statements);
+        startHadoopCluster();
+    }
+
+    @Test
+    public void testCqlStorage() throws IOException, ClassNotFoundException, TException, TimedOutException, NotFoundException, InvalidRequestException, NoSuchFieldException, UnavailableException, IllegalAccessException, InstantiationException, AuthenticationException, AuthorizationException
+    {
+        //regular thrift column families
+        pig.registerQuery("data = load 'cql://thriftKs/SomeApp?" + defaultParameters + "' using CqlStorage();");
+
+        //(bar,3.141592653589793,1335890877,User Bar,35.0,9,15000,like)
+        //(baz,1.61803399,1335890877,User Baz,95.3,3,512000,dislike)
+        //(foo,2.718281828459045,1335890877,User Foo,85.0,8,125000,like)
+        //(qux,0.660161815846869,1335890877,User Qux,64.7,2,12000,dislike)
+
+        //{key: chararray,atomic_weight: double,created: long,name: chararray,percent: float,rating: int,score: long,vote_type: chararray}
+        Iterator<Tuple> it = pig.openIterator("data");
+        int count = 0;
+        while (it.hasNext()) {
+            count ++;
+            Tuple t = it.next();
+            if (count == 1)
+            {
+                Assert.assertEquals(t.get(0), "bar");
+                Assert.assertEquals(t.get(1), 3.141592653589793d);
+                Assert.assertEquals(t.get(3), "User Bar");
+                Assert.assertEquals(t.get(4), 35.0f);
+                Assert.assertEquals(t.get(5), 9);
+                Assert.assertEquals(t.get(6), 15000L);
+                Assert.assertEquals(t.get(7), "like");
+            }
+            else if (count == 2)
+            {
+                Assert.assertEquals(t.get(0), "baz");
+                Assert.assertEquals(t.get(1), 1.61803399d);
+                Assert.assertEquals(t.get(3), "User Baz");
+                Assert.assertEquals(t.get(4), 95.3f);
+                Assert.assertEquals(t.get(5), 3);
+                Assert.assertEquals(t.get(6), 512000L);
+                Assert.assertEquals(t.get(7), "dislike");
+            }else if (count == 3)
+            {
+                Assert.assertEquals(t.get(0), "foo");
+                Assert.assertEquals(t.get(1), 2.718281828459045d);
+                Assert.assertEquals(t.get(3), "User Foo");
+                Assert.assertEquals(t.get(4), 85.0f);
+                Assert.assertEquals(t.get(5), 8);
+                Assert.assertEquals(t.get(6), 125000L);
+                Assert.assertEquals(t.get(7), "like");
+            }
+            else if (count == 4)
+            {
+                Assert.assertEquals(t.get(0), "qux");
+                Assert.assertEquals(t.get(1), 0.660161815846869d);
+                Assert.assertEquals(t.get(3), "User Qux");
+                Assert.assertEquals(t.get(4), 64.7f);
+                Assert.assertEquals(t.get(5), 2);
+                Assert.assertEquals(t.get(6), 12000L);
+                Assert.assertEquals(t.get(7), "dislike");
+            }
+        }
+        Assert.assertEquals(count, 4);
+
+        //Test counter colun family
+        pig.registerQuery("cc_data = load 'cql://thriftKs/CC?" + defaultParameters + "' using CqlStorage();");
+
+        //(chuck,fist,1)
+        //(chuck,kick,3)
+
+        // {key: chararray,column1: chararray,value: long}
+        it = pig.openIterator("cc_data");
+        count = 0;
+        while (it.hasNext()) {
+            count ++;
+            Tuple t = it.next();
+            if (count == 1)
+            {
+                Assert.assertEquals(t.get(0), "chuck");
+                Assert.assertEquals(t.get(1), "fist");
+                Assert.assertEquals(t.get(2), 1L);
+            }
+            else if (count == 2)
+            {
+                Assert.assertEquals(t.get(0), "chuck");
+                Assert.assertEquals(t.get(1), "kick");
+                Assert.assertEquals(t.get(2), 3L);
+            }
+        }
+        Assert.assertEquals(count, 2);
+
+        //Test composite column family
+        pig.registerQuery("compo_data = load 'cql://thriftKs/Compo?" + defaultParameters + "' using CqlStorage();");
+
+        //(kick,bruce,bruce,watch it, mate)
+        //(kick,bruce,lee,oww)
+        //(punch,bruce,bruce,hunh?)
+        //(punch,bruce,lee,ouch)
+
+        //{key: chararray,column1: chararray,column2: chararray,value: chararray}
+        it = pig.openIterator("compo_data");
+        count = 0;
+        while (it.hasNext()) {
+            count ++;
+            Tuple t = it.next();
+            if (count == 1)
+            {
+                Assert.assertEquals(t.get(0), "kick");
+                Assert.assertEquals(t.get(1), "bruce");
+                Assert.assertEquals(t.get(2), "bruce");
+                Assert.assertEquals(t.get(3), "watch it, mate");
+            }
+            else if (count == 2)
+            {
+                Assert.assertEquals(t.get(0), "kick");
+                Assert.assertEquals(t.get(1), "bruce");
+                Assert.assertEquals(t.get(2), "lee");
+                Assert.assertEquals(t.get(3), "oww");
+            }
+            else if (count == 3)
+            {
+                Assert.assertEquals(t.get(0), "punch");
+                Assert.assertEquals(t.get(1), "bruce");
+                Assert.assertEquals(t.get(2), "bruce");
+                Assert.assertEquals(t.get(3), "hunh?");
+            }
+            else if (count == 4)
+            {
+                Assert.assertEquals(t.get(0), "punch");
+                Assert.assertEquals(t.get(1), "bruce");
+                Assert.assertEquals(t.get(2), "lee");
+                Assert.assertEquals(t.get(3), "ouch");
+            }
+        }
+        Assert.assertEquals(count, 4);
+    }
+
+    @Test
+    public void testCassandraStorageSchema() throws IOException, ClassNotFoundException, TException, TimedOutException, NotFoundException, InvalidRequestException, NoSuchFieldException, UnavailableException, 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();");
+
+        //schema: {key: chararray,atomic_weight: (name: chararray,value: double),created: (name: chararray,value: long),
+        //name: (name: chararray,value: chararray),percent: (name: chararray,value: float),
+        //rating: (name: chararray,value: int),score: (name: chararray,value: long),
+        //vote_type: (name: chararray,value: chararray),columns: {(name: chararray,value: chararray)}}
+        Iterator<Tuple> it = pig.openIterator("rows");
+        int count = 0;
+        if (it.hasNext()) {
+            Tuple t = it.next();
+            String rowKey =  t.get(0).toString();
+            if ("qux".equals(rowKey))
+            {
+                Tuple column = (Tuple) t.get(1);
+                Assert.assertEquals(column.get(0), "atomic_weight");
+                Assert.assertEquals(column.get(1), 0.660161815846869d);
+                column = (Tuple) t.get(3);
+                Assert.assertEquals(column.get(0), "name");
+                Assert.assertEquals(column.get(1), "User Qux");
+                column = (Tuple) t.get(4);
+                Assert.assertEquals(column.get(0), "percent");
+                Assert.assertEquals(column.get(1), 64.7f);
+                column = (Tuple) t.get(5);
+                Assert.assertEquals(column.get(0), "rating");
+                Assert.assertEquals(column.get(1), 2);
+                column = (Tuple) t.get(6);
+                Assert.assertEquals(column.get(0), "score");
+                Assert.assertEquals(column.get(1), 12000L);
+                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
+    {
+        createColumnFamily("thriftKs", "CopyOfSomeApp", statements[3]);
+        pig.setBatchOn();
+        pig.registerQuery("rows = LOAD 'cassandra://thriftKs/SomeApp?" + defaultParameters + "' USING CassandraStorage();");
+        //full copy
+        pig.registerQuery("STORE rows INTO 'cassandra://thriftKs/CopyOfSomeApp?" + 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"));
+    }
+
+    @Test
+    public void testCassandraStorageSigleTupleCopy() throws IOException, ClassNotFoundException, TException, TimedOutException, NotFoundException, InvalidRequestException, NoSuchFieldException, UnavailableException, IllegalAccessException, InstantiationException, AuthenticationException, AuthorizationException
+    {
+        createColumnFamily("thriftKs", "CopyOfSomeApp", statements[3]);
+        pig.setBatchOn();
+        pig.registerQuery("rows = LOAD 'cassandra://thriftKs/SomeApp?" + 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.executeBatch();
+        String value = null;
+        try
+        {
+            value = getColumnValue("thriftKs", "CopyOfSomeApp", "name", "qux", "UTF8Type");
+        }
+        catch (NotFoundException e)
+        {
+            Assert.assertTrue(true);
+        }
+        if (value != null)
+            Assert.fail();
+        try
+        {
+            value = getColumnValue("thriftKs", "CopyOfSomeApp", "vote_type", "qux", "UTF8Type");
+        }
+        catch (NotFoundException e)
+        {
+            Assert.assertTrue(true);
+        }
+        if (value != null)
+            Assert.fail();
+        Assert.assertEquals("64.7", getColumnValue("thriftKs", "CopyOfSomeApp", "percent", "qux", "FloatType"));
+    }
+
+    @Test
+    public void testCassandraStorageBagOnlyCopy() throws IOException, ClassNotFoundException, TException, TimedOutException, NotFoundException, InvalidRequestException, NoSuchFieldException, UnavailableException, IllegalAccessException, InstantiationException, AuthenticationException, AuthorizationException
+    {
+        createColumnFamily("thriftKs", "CopyOfSomeApp", statements[3]);
+        pig.setBatchOn();
+        pig.registerQuery("rows = LOAD 'cassandra://thriftKs/SomeApp?" + 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.executeBatch();
+        String value = null;
+        try
+        {
+            value = getColumnValue("thriftKs", "CopyOfSomeApp", "name", "qux", "UTF8Type");
+        }
+        catch (NotFoundException e)
+        {
+            Assert.assertTrue(true);
+        }
+        if (value != null)
+            Assert.fail();
+        try
+        {
+            value = getColumnValue("thriftKs", "CopyOfSomeApp", "vote_type", "qux", "UTF8Type");
+        }
+        catch (NotFoundException e)
+        {
+            Assert.assertTrue(true);
+        }
+        if (value != null)
+            Assert.fail();
+        try
+        {
+            value = getColumnValue("thriftKs", "CopyOfSomeApp", "percent", "qux", "FloatType");
+        }
+        catch (NotFoundException e)
+        {
+            Assert.assertTrue(true);
+        }
+        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
+    {
+        createColumnFamily("thriftKs", "CopyOfSomeApp", statements[3]);
+        pig.setBatchOn();
+        pig.registerQuery("rows = LOAD 'cassandra://thriftKs/SomeApp?" + 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.executeBatch();
+
+        Assert.assertEquals("like", getColumnValue("thriftKs", "CopyOfSomeApp", "vote_type", "bar", "UTF8Type"));
+        Assert.assertEquals("like", getColumnValue("thriftKs", "CopyOfSomeApp", "vote_type", "foo", "UTF8Type"));
+        String value = null;
+        try
+        {
+            value = getColumnValue("thriftKs", "CopyOfSomeApp", "vote_type", "qux", "UTF8Type");
+        }
+        catch (NotFoundException e)
+        {
+            Assert.assertTrue(true);
+        }
+        if (value != null)
+            Assert.fail();
+        try
+        {
+            value = getColumnValue("thriftKs", "CopyOfSomeApp", "vote_type", "baz", "UTF8Type");
+        }
+        catch (NotFoundException e)
+        {
+            Assert.assertTrue(true);
+        }
+        if (value != null)
+            Assert.fail();
+
+        createColumnFamily("thriftKs", "CopyOfSomeApp", statements[3]);
+        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.executeBatch();
+        Assert.assertEquals("dislike", getColumnValue("thriftKs", "CopyOfSomeApp", "vote_type", "baz", "UTF8Type"));
+        Assert.assertEquals("dislike", getColumnValue("thriftKs", "CopyOfSomeApp", "vote_type", "qux", "UTF8Type"));
+        value = null;
+        try
+        {
+            value = getColumnValue("thriftKs", "CopyOfSomeApp", "vote_type", "bar", "UTF8Type");
+        }
+        catch (NotFoundException e)
+        {
+            Assert.assertTrue(true);
+        }
+        if (value != null)
+            Assert.fail();
+        try
+        {
+            value = getColumnValue("thriftKs", "CopyOfSomeApp", "vote_type", "foo", "UTF8Type");
+        }
+        catch (NotFoundException e)
+        {
+            Assert.assertTrue(true);
+        }
+        if (value != null)
+            Assert.fail();
+    }
+
+    @Test
+    public void testCassandraStorageJoin() throws IOException, ClassNotFoundException, TException, TimedOutException, NotFoundException, InvalidRequestException, NoSuchFieldException, UnavailableException, IllegalAccessException, InstantiationException, AuthenticationException, AuthorizationException
+    {
+        //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();");
+
+        //cast key to chararray
+        pig.registerQuery("b = foreach Bytes generate (chararray)key, columns;");
+
+        //key in Bytes is a bytearray, U8 chararray
+        //(foo,{(x,Z)},foo,{(x,Z)})
+        pig.registerQuery("a = join Bytes by key, U8 by key;");
+        Iterator<Tuple> it = pig.openIterator("a");
+        if (it.hasNext()) {
+            Tuple t = it.next();
+            Assert.assertEquals(t.get(0), new DataByteArray("foo".getBytes()));
+            DataBag columns = (DataBag) t.get(1);
+            Iterator<Tuple> iter = columns.iterator();
+            Tuple t1 = iter.next();
+            Assert.assertEquals(t1.get(0), "x");
+            Assert.assertEquals(t1.get(1), new DataByteArray("Z".getBytes()));
+            String column = (String) t.get(2);
+            Assert.assertEquals(column, "foo");
+            columns = (DataBag) t.get(3);
+            iter = columns.iterator();
+            Tuple t2 = iter.next();
+            Assert.assertEquals(t2.get(0), "x");
+            Assert.assertEquals(t2.get(1), new DataByteArray("Z".getBytes()));
+        }
+        //key should now be cast into a chararray
+        //(foo,{(x,Z)},foo,{(x,Z)})
+        pig.registerQuery("c = join b by (chararray)key, U8 by (chararray)key;");
+        it = pig.openIterator("c");
+        if (it.hasNext()) {
+            Tuple t = it.next();
+            Assert.assertEquals(t.get(0), "foo");
+            DataBag columns = (DataBag) t.get(1);
+            Iterator<Tuple> iter = columns.iterator();
+            Tuple t1 = iter.next();
+            Assert.assertEquals(t1.get(0), "x");
+            Assert.assertEquals(t1.get(1), new DataByteArray("Z".getBytes()));
+            String column = (String) t.get(2);
+            Assert.assertEquals(column, "foo");
+            columns = (DataBag) t.get(3);
+            iter = columns.iterator();
+            Tuple t2 = iter.next();
+            Assert.assertEquals(t2.get(0), "x");
+            Assert.assertEquals(t2.get(1), new DataByteArray("Z".getBytes()));
+        }
+    }
+
+    @Test
+    public void testCassandraStorageCounterCF() throws IOException, ClassNotFoundException, TException, TimedOutException, NotFoundException, InvalidRequestException, NoSuchFieldException, UnavailableException, IllegalAccessException, InstantiationException, AuthenticationException, AuthorizationException
+    {
+        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("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);
+        }
+    }
+
+    @Test
+    public void testCassandraStorageCompositeColumnCF() throws IOException, ClassNotFoundException, TException, TimedOutException, NotFoundException, InvalidRequestException, NoSuchFieldException, UnavailableException, IllegalAccessException, InstantiationException, AuthenticationException, AuthorizationException
+    {
+        //Test CompositeType
+        pig.registerQuery("compo = load 'cassandra://thriftKs/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');");
+
+        //(kick,(bruce,lee),oww)
+        //(punch,(bruce,lee),ouch)
+        Iterator<Tuple> it = pig.openIterator("lee");
+        int count = 0;
+        while (it.hasNext()) {
+            count ++;
+            Tuple t = it.next();
+            if (count == 1)
+                Assert.assertEquals(t.get(0), "kick");
+            else
+                Assert.assertEquals(t.get(0), "punch");
+            Tuple t1 = (Tuple) t.get(1);
+            Assert.assertEquals(t1.get(0), "bruce");
+            Assert.assertEquals(t1.get(1), "lee");
+            if (count == 1)
+                Assert.assertEquals(t.get(2), "oww");
+            else
+                Assert.assertEquals(t.get(2), "ouch");
+        }
+        Assert.assertEquals(count, 2);
+        pig.registerQuery("night = load 'cassandra://thriftKs/CompoInt?" + 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;");
+
+        //What happens at the darkest hour?
+        pig.registerQuery("darkest = filter night by hour > 2 and hour < 5;");
+
+        //(2.5,daddy?)
+        it = pig.openIterator("darkest");
+        if (it.hasNext()) {
+            Tuple t = it.next();
+            Assert.assertEquals(t.get(0), 2.5d);
+            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.executeBatch();
+        pig.registerQuery("compocopy_int_rows = LOAD 'cassandra://thriftKs/CompoIntCopy?" + defaultParameters + "' using CassandraStorage();");
+        //(clock,{((1,0),z),((1,30),zzzz),((2,30),daddy?),((6,30),coffee...)})
+        it = pig.openIterator("compocopy_int_rows");
+        count = 0;
+        if (it.hasNext()) {
+            Tuple t = it.next();
+            Assert.assertEquals(t.get(0), "clock");
+            DataBag columns = (DataBag) t.get(1);
+            Iterator<Tuple> iter = columns.iterator();
+            while (iter.hasNext())
+            {
+                count ++;
+                Tuple t1 = iter.next();
+                Tuple inner = (Tuple) t1.get(0);
+                if (count == 1)
+                {
+                    Assert.assertEquals(inner.get(0), 1L);
+                    Assert.assertEquals(inner.get(1), 0L);
+                    Assert.assertEquals(t1.get(1), "z");
+                }
+                else if (count == 2)
+                {
+                    Assert.assertEquals(inner.get(0), 1L);
+                    Assert.assertEquals(inner.get(1), 30L);
+                    Assert.assertEquals(t1.get(1), "zzzz");
+                }
+                else if (count == 3)
+                {
+                    Assert.assertEquals(inner.get(0), 2L);
+                    Assert.assertEquals(inner.get(1), 30L);
+                    Assert.assertEquals(t1.get(1), "daddy?");
+                }
+                else if (count == 4)
+                {
+                    Assert.assertEquals(inner.get(0), 6L);
+                    Assert.assertEquals(inner.get(1), 30L);
+                    Assert.assertEquals(t1.get(1), "coffee...");
+                }
+            }
+            Assert.assertEquals(count, 4);
+        }
+    }
+
+    @Test
+    public void testCassandraStorageCompositeKeyCF() throws IOException, ClassNotFoundException, TException, TimedOutException, NotFoundException, InvalidRequestException, NoSuchFieldException, UnavailableException, IllegalAccessException, InstantiationException, AuthenticationException, AuthorizationException
+    {
+        //Test CompositeKey
+        pig.registerQuery("compokeys = load 'cassandra://thriftKs/CompoKey?" + defaultParameters + "' using CassandraStorage();");
+        pig.registerQuery("compokeys = filter compokeys by key.$1 == 40;");
+        //((clock,40),{(6,coffee...)})
+        Iterator<Tuple> it = pig.openIterator("compokeys");
+        if (it.hasNext()) {
+            Tuple t = it.next();
+            Tuple key = (Tuple) t.get(0); 
+            Assert.assertEquals(key.get(0), "clock");
+            Assert.assertEquals(key.get(1), 40L);
+            DataBag columns = (DataBag) t.get(1);
+            Iterator<Tuple> iter = columns.iterator();
+            if (iter.hasNext())
+            {
+                Tuple t1 = iter.next();
+                Assert.assertEquals(t1.get(0), 6L);
+                Assert.assertEquals(t1.get(1), "coffee...");
+            }
+        }
+        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.executeBatch();
+        pig.registerQuery("compo_key_copy_rows = LOAD 'cassandra://thriftKs/CompoKeyCopy?" + defaultParameters + "' using CassandraStorage();");
+        //((clock,10),{(1,z)})
+        //((clock,20),{(1,zzzz)})
+        //((clock,30),{(2,daddy?)})
+        //((clock,40),{(6,coffee...)})
+        it = pig.openIterator("compo_key_copy_rows");
+        int count = 0;
+        while (it.hasNext()) {
+            Tuple t = it.next();
+            count ++;
+            if (count == 1)
+            {
+                Tuple key = (Tuple) t.get(0); 
+                Assert.assertEquals(key.get(0), "clock");
+                Assert.assertEquals(key.get(1), 10L);
+                DataBag columns = (DataBag) t.get(1);
+                Iterator<Tuple> iter = columns.iterator();
+                if (iter.hasNext())
+                {
+                    Tuple t1 = iter.next();
+                    Assert.assertEquals(t1.get(0), 1L);
+                    Assert.assertEquals(t1.get(1), "z");
+                }
+            }
+            else if (count == 2)
+            {
+                Tuple key = (Tuple) t.get(0); 
+                Assert.assertEquals(key.get(0), "clock");
+                Assert.assertEquals(key.get(1), 20L);
+                DataBag columns = (DataBag) t.get(1);
+                Iterator<Tuple> iter = columns.iterator();
+                if (iter.hasNext())
+                {
+                    Tuple t1 = iter.next();
+                    Assert.assertEquals(t1.get(0), 1L);
+                    Assert.assertEquals(t1.get(1), "zzzz");
+                }
+            }
+            else if (count == 3)
+            {
+                Tuple key = (Tuple) t.get(0); 
+                Assert.assertEquals(key.get(0), "clock");
+                Assert.assertEquals(key.get(1), 30L);
+                DataBag columns = (DataBag) t.get(1);
+                Iterator<Tuple> iter = columns.iterator();
+                if (iter.hasNext())
+                {
+                    Tuple t1 = iter.next();
+                    Assert.assertEquals(t1.get(0), 2L);
+                    Assert.assertEquals(t1.get(1), "daddy?");
+                }
+            }
+            else if (count == 4)
+            {
+                Tuple key = (Tuple) t.get(0); 
+                Assert.assertEquals(key.get(0), "clock");
+                Assert.assertEquals(key.get(1), 40L);
+                DataBag columns = (DataBag) t.get(1);
+                Iterator<Tuple> iter = columns.iterator();
+                if (iter.hasNext())
+                {
+                    Tuple t1 = iter.next();
+                    Assert.assertEquals(t1.get(0), 6L);
+                    Assert.assertEquals(t1.get(1), "coffee...");
+                }
+            }
+        }
+        Assert.assertEquals(count, 4);
+    }
+
+    private String getColumnValue(String ks, String cf, String colName, String key, String validator)
+    throws AuthenticationException, AuthorizationException, InvalidRequestException, UnavailableException, TimedOutException, TException, NotFoundException, IOException
+    {
+        Cassandra.Client client = getClient();
+        client.set_keyspace(ks);
+
+        ByteBuffer key_user_id = ByteBufferUtil.bytes(key);
+
+        long timestamp = System.currentTimeMillis();
+        ColumnPath cp = new ColumnPath(cf);
+        ColumnParent par = new ColumnParent(cf);
+        cp.column = ByteBufferUtil.bytes(colName);
+
+        // read
+        ColumnOrSuperColumn got = client.get(key_user_id, cp, ConsistencyLevel.ONE);
+        return parseType(validator).getString(got.getColumn().value);
+    }
+
+    private void createColumnFamily(String ks, String cf, String statement) throws CharacterCodingException, ClassNotFoundException, TException, TimedOutException, NotFoundException, InvalidRequestException, NoSuchFieldException, UnavailableException, IllegalAccessException, InstantiationException
+    {
+        CliMain.connect("127.0.0.1", 9170);
+        try
+        {
+            CliMain.processStatement("use " + ks + ";");
+            CliMain.processStatement("drop column family " + cf + ";");
+        }
+        catch (Exception e)
+        {
+        }
+        CliMain.processStatement(statement);
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4e359697/test/unit/org/apache/pig/test/MiniCluster.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/pig/test/MiniCluster.java b/test/unit/org/apache/pig/test/MiniCluster.java
new file mode 100644
index 0000000..3216392
--- /dev/null
+++ b/test/unit/org/apache/pig/test/MiniCluster.java
@@ -0,0 +1,78 @@
+/*
+ * 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.pig.test;
+
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.mapred.MiniMRCluster;
+
+public class MiniCluster extends MiniGenericCluster {
+    private MiniMRCluster m_mr = null;
+    public MiniCluster() {
+        super();
+    }
+
+    @Override
+    protected void setupMiniDfsAndMrClusters() {
+        try {
+            System.setProperty("hadoop.log.dir", "build/test/logs");
+            final int dataNodes = 4;     // There will be 4 data nodes
+            final int taskTrackers = 4;  // There will be 4 task tracker nodes
+
+            // Create the configuration hadoop-site.xml file
+            File conf_dir = new File("build/classes/");
+            conf_dir.mkdirs();
+            File conf_file = new File(conf_dir, "hadoop-site.xml");
+
+            conf_file.delete();
+
+            // Builds and starts the mini dfs and mapreduce clusters
+            Configuration config = new Configuration();
+            m_dfs = new MiniDFSCluster(config, dataNodes, true, null);
+            m_fileSys = m_dfs.getFileSystem();
+            m_mr = new MiniMRCluster(taskTrackers, m_fileSys.getUri().toString(), 1);
+
+            // Write the necessary config info to hadoop-site.xml
+            m_conf = m_mr.createJobConf();
+            m_conf.setInt("mapred.submit.replication", 2);
+            m_conf.set("dfs.datanode.address", "0.0.0.0:0");
+            m_conf.set("dfs.datanode.http.address", "0.0.0.0:0");
+            m_conf.set("mapred.map.max.attempts", "2");
+            m_conf.set("mapred.reduce.max.attempts", "2");
+            m_conf.set("pig.jobcontrol.sleep", "100");
+            m_conf.writeXml(new FileOutputStream(conf_file));
+
+            // Set the system properties needed by Pig
+            System.setProperty("cluster", m_conf.get("mapred.job.tracker"));
+            System.setProperty("namenode", m_conf.get("fs.default.name"));
+            System.setProperty("junit.hadoop.conf", conf_dir.getPath());
+        } catch (IOException e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+    @Override
+    protected void shutdownMiniMrClusters() {
+        if (m_mr != null) { m_mr.shutdown(); }
+            m_mr = null;
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4e359697/test/unit/org/apache/pig/test/MiniGenericCluster.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/pig/test/MiniGenericCluster.java b/test/unit/org/apache/pig/test/MiniGenericCluster.java
new file mode 100644
index 0000000..ac3f5bc
--- /dev/null
+++ b/test/unit/org/apache/pig/test/MiniGenericCluster.java
@@ -0,0 +1,122 @@
+/**
+ * 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.pig.test;
+
+import java.io.*;
+import java.util.Properties;
+
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.pig.backend.hadoop.datastorage.ConfigurationUtil;
+
+/**
+ * This class builds a single instance of itself with the Singleton 
+ * design pattern. While building the single instance, it sets up a 
+ * mini cluster that actually consists of a mini DFS cluster and a 
+ * mini MapReduce cluster on the local machine and also sets up the 
+ * environment for Pig to run on top of the mini cluster.
+ *
+ * This class is the base class for MiniCluster, which has slightly
+ * difference among different versions of hadoop. MiniCluster implementation
+ * is located in $PIG_HOME/shims.
+ */
+abstract public class MiniGenericCluster {
+    protected MiniDFSCluster m_dfs = null;
+    protected FileSystem m_fileSys = null;
+    protected Configuration m_conf = null;
+    
+    protected final static MiniCluster INSTANCE = new MiniCluster();
+    protected static boolean isSetup = true;
+    
+    protected MiniGenericCluster() {
+        setupMiniDfsAndMrClusters();
+    }
+    
+    abstract protected void setupMiniDfsAndMrClusters();
+    
+    /**
+     * Returns the single instance of class MiniClusterBuilder that
+     * represents the resouces for a mini dfs cluster and a mini 
+     * mapreduce cluster. 
+     */
+    public static MiniCluster buildCluster() {
+        if(! isSetup){
+            INSTANCE.setupMiniDfsAndMrClusters();
+            isSetup = true;
+        }
+        return INSTANCE;
+    }
+
+    public void shutDown(){
+        INSTANCE.shutdownMiniDfsAndMrClusters();
+    }
+    
+    protected void finalize() {
+        shutdownMiniDfsAndMrClusters();
+    }
+    
+    protected void shutdownMiniDfsAndMrClusters() {
+        isSetup = false;
+        shutdownMiniDfsClusters();
+        shutdownMiniMrClusters();
+    }
+    
+    protected void shutdownMiniDfsClusters() {
+        try {
+            if (m_fileSys != null) { m_fileSys.close(); }
+        } catch (IOException e) {
+            e.printStackTrace();
+        }
+        if (m_dfs != null) { m_dfs.shutdown(); }
+        m_fileSys = null;
+        m_dfs = null;
+    }
+    
+    abstract protected void shutdownMiniMrClusters();
+
+    public Properties getProperties() {
+        errorIfNotSetup();
+        return ConfigurationUtil.toProperties(m_conf);
+    }
+
+    public Configuration getConfiguration() {
+        return new Configuration(m_conf);
+    }
+
+    public void setProperty(String name, String value) {
+        errorIfNotSetup();
+        m_conf.set(name, value);
+    }
+    
+    public FileSystem getFileSystem() {
+        errorIfNotSetup();
+        return m_fileSys;
+    }
+    
+    /**
+     * Throw RunTimeException if isSetup is false
+     */
+    private void errorIfNotSetup(){
+        if(isSetup)
+            return;
+        String msg = "function called on MiniCluster that has been shutdown";
+        throw new RuntimeException(msg);
+    }
+}


[4/6] git commit: Merge branch 'cassandra-1.2' into cassandra-2.0

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

Conflicts:
	build.xml
	src/java/org/apache/cassandra/hadoop/pig/AbstractCassandraStorage.java


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

Branch: refs/heads/trunk
Commit: fe598e77e6e06a6408ea8e8b1fa80b25977e572d
Parents: 3a29c74 4e35969
Author: Brandon Williams <br...@apache.org>
Authored: Mon Oct 14 14:17:05 2013 -0500
Committer: Brandon Williams <br...@apache.org>
Committed: Mon Oct 14 14:17:05 2013 -0500

----------------------------------------------------------------------
 build.xml                                       |   8 +
 .../hadoop/pig/AbstractCassandraStorage.java    |   4 +-
 .../cassandra/hadoop/pig/CassandraStorage.java  |  25 +-
 .../apache/cassandra/hadoop/pig/CqlStorage.java |  25 +-
 test/conf/log4j-junit.properties                |   3 +
 .../org/apache/cassandra/pig/CqlTableTest.java  | 236 ++++++
 .../org/apache/cassandra/pig/PigTestBase.java   | 167 ++++
 .../cassandra/pig/ThriftColumnFamilyTest.java   | 809 +++++++++++++++++++
 test/unit/org/apache/pig/test/MiniCluster.java  |  78 ++
 .../org/apache/pig/test/MiniGenericCluster.java | 122 +++
 10 files changed, 1470 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/fe598e77/build.xml
----------------------------------------------------------------------
diff --cc build.xml
index 4be1d10,37109c8..5e7f269
--- a/build.xml
+++ b/build.xml
@@@ -364,7 -370,9 +364,8 @@@
               <exclusion groupId="commons-lang" artifactId="commons-lang"/>
            </dependency>
            <dependency groupId="org.apache.hadoop" artifactId="hadoop-core" version="1.0.3"/>
+           <dependency groupId="org.apache.hadoop" artifactId="hadoop-minicluster" version="1.0.3"/>
            <dependency groupId="org.apache.pig" artifactId="pig" version="0.10.0"/>
 -          <dependency groupId="net.sf.jopt-simple" artifactId="jopt-simple" version="3.2"/>
            <dependency groupId="net.java.dev.jna" artifactId="jna" version="3.2.7"/>
  
            <dependency groupId="net.sourceforge.cobertura" artifactId="cobertura" version="${cobertura.version}"/>
@@@ -404,7 -418,9 +405,8 @@@
          <dependency groupId="commons-logging" artifactId="commons-logging"/>
          <dependency groupId="org.apache.rat" artifactId="apache-rat"/>
          <dependency groupId="org.apache.hadoop" artifactId="hadoop-core"/>
+       	<dependency groupId="org.apache.hadoop" artifactId="hadoop-minicluster"/>
          <dependency groupId="org.apache.pig" artifactId="pig"/>
 -        <dependency groupId="net.sf.jopt-simple" artifactId="jopt-simple"/>
  
          <dependency groupId="net.java.dev.jna" artifactId="jna"/>
        </artifact:pom>
@@@ -1120,9 -1163,11 +1123,14 @@@
      </testmacro>
    </target>
  
 +  <target name="test-all" 
 +          depends="test,long-test,test-compression,test-clientutil-jar" 
 +          description="Run all tests" />
+   <target name="pig-test" depends="build-test" description="Excute Pig tests">
+     <testmacro suitename="unit" inputdir="${test.unit.src}" 
+                timeout="1200000" filter="**/pig/*Test.java">
+     </testmacro>
+   </target>
  
    <!-- instruments the classes to later create code coverage reports -->
    <target name="cobertura-instrument" depends="build,build-test">

http://git-wip-us.apache.org/repos/asf/cassandra/blob/fe598e77/src/java/org/apache/cassandra/hadoop/pig/AbstractCassandraStorage.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/fe598e77/src/java/org/apache/cassandra/hadoop/pig/CassandraStorage.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/hadoop/pig/CassandraStorage.java
index 5357709,0512a37..4708359
--- a/src/java/org/apache/cassandra/hadoop/pig/CassandraStorage.java
+++ b/src/java/org/apache/cassandra/hadoop/pig/CassandraStorage.java
@@@ -792,11 -818,11 +810,12 @@@ public class CassandraStorage extends A
          catch (Exception e)
          {
              throw new IOException("Expected 'cassandra://[username:password@]<keyspace>/<columnfamily>" +
-             		                        "[?slice_start=<start>&slice_end=<end>[&reversed=true][&limit=1]" +
-             		                        "[&allow_deletes=true][&widerows=true][&use_secondary=true]" +
-             		                        "[&comparator=<comparator>][&split_size=<size>][&partitioner=<partitioner>]]': " + e.getMessage());
+                     "[?slice_start=<start>&slice_end=<end>[&reversed=true][&limit=1]" +
+                     "[&allow_deletes=true][&widerows=true][&use_secondary=true]" +
+                     "[&comparator=<comparator>][&split_size=<size>][&partitioner=<partitioner>]" +
+                     "[&init_address=<host>][&rpc_port=<port>]]': " + e.getMessage());
          }
      }
 +
  }
  

http://git-wip-us.apache.org/repos/asf/cassandra/blob/fe598e77/src/java/org/apache/cassandra/hadoop/pig/CqlStorage.java
----------------------------------------------------------------------


[5/6] git commit: Merge branch 'cassandra-1.2' into cassandra-2.0

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

Conflicts:
	build.xml
	src/java/org/apache/cassandra/hadoop/pig/AbstractCassandraStorage.java


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

Branch: refs/heads/cassandra-2.0
Commit: fe598e77e6e06a6408ea8e8b1fa80b25977e572d
Parents: 3a29c74 4e35969
Author: Brandon Williams <br...@apache.org>
Authored: Mon Oct 14 14:17:05 2013 -0500
Committer: Brandon Williams <br...@apache.org>
Committed: Mon Oct 14 14:17:05 2013 -0500

----------------------------------------------------------------------
 build.xml                                       |   8 +
 .../hadoop/pig/AbstractCassandraStorage.java    |   4 +-
 .../cassandra/hadoop/pig/CassandraStorage.java  |  25 +-
 .../apache/cassandra/hadoop/pig/CqlStorage.java |  25 +-
 test/conf/log4j-junit.properties                |   3 +
 .../org/apache/cassandra/pig/CqlTableTest.java  | 236 ++++++
 .../org/apache/cassandra/pig/PigTestBase.java   | 167 ++++
 .../cassandra/pig/ThriftColumnFamilyTest.java   | 809 +++++++++++++++++++
 test/unit/org/apache/pig/test/MiniCluster.java  |  78 ++
 .../org/apache/pig/test/MiniGenericCluster.java | 122 +++
 10 files changed, 1470 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/fe598e77/build.xml
----------------------------------------------------------------------
diff --cc build.xml
index 4be1d10,37109c8..5e7f269
--- a/build.xml
+++ b/build.xml
@@@ -364,7 -370,9 +364,8 @@@
               <exclusion groupId="commons-lang" artifactId="commons-lang"/>
            </dependency>
            <dependency groupId="org.apache.hadoop" artifactId="hadoop-core" version="1.0.3"/>
+           <dependency groupId="org.apache.hadoop" artifactId="hadoop-minicluster" version="1.0.3"/>
            <dependency groupId="org.apache.pig" artifactId="pig" version="0.10.0"/>
 -          <dependency groupId="net.sf.jopt-simple" artifactId="jopt-simple" version="3.2"/>
            <dependency groupId="net.java.dev.jna" artifactId="jna" version="3.2.7"/>
  
            <dependency groupId="net.sourceforge.cobertura" artifactId="cobertura" version="${cobertura.version}"/>
@@@ -404,7 -418,9 +405,8 @@@
          <dependency groupId="commons-logging" artifactId="commons-logging"/>
          <dependency groupId="org.apache.rat" artifactId="apache-rat"/>
          <dependency groupId="org.apache.hadoop" artifactId="hadoop-core"/>
+       	<dependency groupId="org.apache.hadoop" artifactId="hadoop-minicluster"/>
          <dependency groupId="org.apache.pig" artifactId="pig"/>
 -        <dependency groupId="net.sf.jopt-simple" artifactId="jopt-simple"/>
  
          <dependency groupId="net.java.dev.jna" artifactId="jna"/>
        </artifact:pom>
@@@ -1120,9 -1163,11 +1123,14 @@@
      </testmacro>
    </target>
  
 +  <target name="test-all" 
 +          depends="test,long-test,test-compression,test-clientutil-jar" 
 +          description="Run all tests" />
+   <target name="pig-test" depends="build-test" description="Excute Pig tests">
+     <testmacro suitename="unit" inputdir="${test.unit.src}" 
+                timeout="1200000" filter="**/pig/*Test.java">
+     </testmacro>
+   </target>
  
    <!-- instruments the classes to later create code coverage reports -->
    <target name="cobertura-instrument" depends="build,build-test">

http://git-wip-us.apache.org/repos/asf/cassandra/blob/fe598e77/src/java/org/apache/cassandra/hadoop/pig/AbstractCassandraStorage.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/fe598e77/src/java/org/apache/cassandra/hadoop/pig/CassandraStorage.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/hadoop/pig/CassandraStorage.java
index 5357709,0512a37..4708359
--- a/src/java/org/apache/cassandra/hadoop/pig/CassandraStorage.java
+++ b/src/java/org/apache/cassandra/hadoop/pig/CassandraStorage.java
@@@ -792,11 -818,11 +810,12 @@@ public class CassandraStorage extends A
          catch (Exception e)
          {
              throw new IOException("Expected 'cassandra://[username:password@]<keyspace>/<columnfamily>" +
-             		                        "[?slice_start=<start>&slice_end=<end>[&reversed=true][&limit=1]" +
-             		                        "[&allow_deletes=true][&widerows=true][&use_secondary=true]" +
-             		                        "[&comparator=<comparator>][&split_size=<size>][&partitioner=<partitioner>]]': " + e.getMessage());
+                     "[?slice_start=<start>&slice_end=<end>[&reversed=true][&limit=1]" +
+                     "[&allow_deletes=true][&widerows=true][&use_secondary=true]" +
+                     "[&comparator=<comparator>][&split_size=<size>][&partitioner=<partitioner>]" +
+                     "[&init_address=<host>][&rpc_port=<port>]]': " + e.getMessage());
          }
      }
 +
  }
  

http://git-wip-us.apache.org/repos/asf/cassandra/blob/fe598e77/src/java/org/apache/cassandra/hadoop/pig/CqlStorage.java
----------------------------------------------------------------------


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

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

Conflicts:
	build.xml
	test/conf/log4j-junit.properties


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

Branch: refs/heads/trunk
Commit: 0f13031023ea588b8b71f3aab3108797c3a114d6
Parents: bb13b80 fe598e7
Author: Brandon Williams <br...@apache.org>
Authored: Mon Oct 14 14:19:20 2013 -0500
Committer: Brandon Williams <br...@apache.org>
Committed: Mon Oct 14 14:19:20 2013 -0500

----------------------------------------------------------------------
 build.xml                                       |   8 +
 .../hadoop/pig/AbstractCassandraStorage.java    |   4 +-
 .../cassandra/hadoop/pig/CassandraStorage.java  |  25 +-
 .../apache/cassandra/hadoop/pig/CqlStorage.java |  25 +-
 test/conf/log4j-junit.properties                |  40 +
 .../org/apache/cassandra/pig/CqlTableTest.java  | 236 ++++++
 .../org/apache/cassandra/pig/PigTestBase.java   | 167 ++++
 .../cassandra/pig/ThriftColumnFamilyTest.java   | 809 +++++++++++++++++++
 test/unit/org/apache/pig/test/MiniCluster.java  |  78 ++
 .../org/apache/pig/test/MiniGenericCluster.java | 122 +++
 10 files changed, 1507 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/0f130310/build.xml
----------------------------------------------------------------------
diff --cc build.xml
index 6e22525,5e7f269..fe1a5f7
--- a/build.xml
+++ b/build.xml
@@@ -363,13 -360,12 +363,14 @@@
  
            <dependency groupId="junit" artifactId="junit" version="4.6" />
            <dependency groupId="commons-logging" artifactId="commons-logging" version="1.1.1"/>
 -          <dependency groupId="org.apache.rat" artifactId="apache-rat" version="0.6">
 +          <dependency groupId="org.apache.rat" artifactId="apache-rat" version="0.10">
               <exclusion groupId="commons-lang" artifactId="commons-lang"/>
            </dependency>
 -          <dependency groupId="org.apache.hadoop" artifactId="hadoop-core" version="1.0.3"/>
 +          <dependency groupId="org.apache.hadoop" artifactId="hadoop-core" version="1.0.3">
 +          	<exclusion groupId="org.mortbay.jetty" artifactId="servlet-api"/>
 +          </dependency>
+           <dependency groupId="org.apache.hadoop" artifactId="hadoop-minicluster" version="1.0.3"/>
 -          <dependency groupId="org.apache.pig" artifactId="pig" version="0.10.0"/>
 +          <dependency groupId="org.apache.pig" artifactId="pig" version="0.11.1"/>
            <dependency groupId="net.java.dev.jna" artifactId="jna" version="3.2.7"/>
  
            <dependency groupId="net.sourceforge.cobertura" artifactId="cobertura" version="${cobertura.version}"/>