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/12/05 23:20:04 UTC

[2/3] git commit: Pig: fix duplicate schema alias Patch by Alex Liu, reviewed by brandonwilliams for CASSANDRA-6309

Pig: fix duplicate schema alias
Patch by Alex Liu, reviewed by brandonwilliams for CASSANDRA-6309


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

Branch: refs/heads/trunk
Commit: f7efaffadace3e344eeb4a1384fa72c73d8422b0
Parents: 667e3db
Author: Brandon Williams <br...@apache.org>
Authored: Thu Dec 5 16:14:43 2013 -0600
Committer: Brandon Williams <br...@apache.org>
Committed: Thu Dec 5 16:14:43 2013 -0600

----------------------------------------------------------------------
 build.xml                                       |   1 +
 .../apache/cassandra/db/ConsistencyLevel.java   |   2 -
 .../hadoop/pig/AbstractCassandraStorage.java    |  21 +---
 .../cassandra/pig/CqlTableDataTypeTest.java     |  35 ++----
 .../org/apache/cassandra/pig/CqlTableTest.java  |   9 +-
 .../pig/ThriftColumnFamilyDataTypeTest.java     |  21 ----
 .../cassandra/pig/ThriftColumnFamilyTest.java   | 121 +++++--------------
 7 files changed, 48 insertions(+), 162 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7efaffa/build.xml
----------------------------------------------------------------------
diff --git a/build.xml b/build.xml
index 41a7fb0..e21091a 100644
--- a/build.xml
+++ b/build.xml
@@ -990,6 +990,7 @@
       </classpath>
       <src path="${test.unit.src}"/>
       <src path="${test.long.src}"/>
+      <src path="${test.pig.src}"/>
     </javac>
 
     <!-- Non-java resources needed by the test suite -->

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7efaffa/src/java/org/apache/cassandra/db/ConsistencyLevel.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/ConsistencyLevel.java b/src/java/org/apache/cassandra/db/ConsistencyLevel.java
index cbb4bb1..0f6aba7 100644
--- a/src/java/org/apache/cassandra/db/ConsistencyLevel.java
+++ b/src/java/org/apache/cassandra/db/ConsistencyLevel.java
@@ -285,9 +285,7 @@ public enum ConsistencyLevel
     {
         switch (this)
         {
-            case LOCAL_QUORUM:
             case EACH_QUORUM:
-            case LOCAL_ONE:
                 requireNetworkTopologyStrategy(keyspaceName);
                 break;
             case SERIAL:

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7efaffa/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 9e2e301..3fb1c5a 100644
--- a/src/java/org/apache/cassandra/hadoop/pig/AbstractCassandraStorage.java
+++ b/src/java/org/apache/cassandra/hadoop/pig/AbstractCassandraStorage.java
@@ -614,20 +614,7 @@ public abstract class AbstractCassandraStorage extends LoadFunc implements Store
             cfDef.default_validation_class = ByteBufferUtil.string(cqlRow.columns.get(3).value);
             cfDef.key_validation_class = ByteBufferUtil.string(cqlRow.columns.get(4).value);
             String keyAliases = ByteBufferUtil.string(cqlRow.columns.get(5).value);
-            List<String> keys = FBUtilities.fromJsonList(keyAliases);
-            // classis thrift tables
-            if (keys.size() == 0)
-            {
-                CFDefinition cfDefinition = getCfDefinition(keyspace, column_family, client);
-                for (ColumnIdentifier column : cfDefinition.keys.keySet())
-                {
-                    String key = column.toString();
-                    String type = cfDefinition.keys.get(column).type.toString();
-                    logger.debug("name: {}, type: {} ", key, type);
-                    keys.add(key);
-                }
-            }
-            else
+            if (FBUtilities.fromJsonList(keyAliases).size() > 0)
                 cql3Table = true;
         }
         cfDef.column_metadata = getColumnMetadata(client);
@@ -666,7 +653,8 @@ public abstract class AbstractCassandraStorage extends LoadFunc implements Store
     {
         String query = "SELECT column_name, " +
                        "       validator, " +
-                       "       index_type " +
+                       "       index_type, " +
+                       "       type " +
                        "FROM system.schema_columns " +
                        "WHERE keyspace_name = '%s' " +
                        "  AND columnfamily_name = '%s'";
@@ -717,6 +705,9 @@ public abstract class AbstractCassandraStorage extends LoadFunc implements Store
         {
             CqlRow row = iterator.next();
             ColumnDef cDef = new ColumnDef();
+            String type = ByteBufferUtil.string(row.getColumns().get(3).value);
+            if (!type.equals("regular"))
+                continue;
             cDef.setName(ByteBufferUtil.clone(row.getColumns().get(0).value));
             cDef.validation_class = ByteBufferUtil.string(row.getColumns().get(1).value);
             ByteBuffer indexType = row.getColumns().get(2).value;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7efaffa/test/pig/org/apache/cassandra/pig/CqlTableDataTypeTest.java
----------------------------------------------------------------------
diff --git a/test/pig/org/apache/cassandra/pig/CqlTableDataTypeTest.java b/test/pig/org/apache/cassandra/pig/CqlTableDataTypeTest.java
index 1ae9806..2020b0a 100644
--- a/test/pig/org/apache/cassandra/pig/CqlTableDataTypeTest.java
+++ b/test/pig/org/apache/cassandra/pig/CqlTableDataTypeTest.java
@@ -7,37 +7,16 @@
  * "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
+ *     http://www.apache.org/licenses/LICENSE-2.0
  *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.cassandra.pig;
-/*
- * 
- * 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.
- * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
  */
 
+package org.apache.cassandra.pig;
 
 import java.io.IOException;
 import java.nio.charset.CharacterCodingException;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7efaffa/test/pig/org/apache/cassandra/pig/CqlTableTest.java
----------------------------------------------------------------------
diff --git a/test/pig/org/apache/cassandra/pig/CqlTableTest.java b/test/pig/org/apache/cassandra/pig/CqlTableTest.java
index 785d819..e33364f 100644
--- a/test/pig/org/apache/cassandra/pig/CqlTableTest.java
+++ b/test/pig/org/apache/cassandra/pig/CqlTableTest.java
@@ -50,6 +50,7 @@ public class CqlTableTest extends PigTestBase
             "INSERT INTO compactcqltable (key1, column1, column2) values ('key1', 100, 10.1)",
 
             "CREATE TABLE test (a int PRIMARY KEY, b int);",
+            "CREATE INDEX test_b on test (b);",
 
             "CREATE TABLE moredata (x int PRIMARY KEY, y int);",
             "INSERT INTO test (a,b) VALUES (1,1);",
@@ -158,10 +159,12 @@ public class CqlTableTest extends PigTestBase
         //(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");
+        int count = 0;
+        while (it.hasNext()) {
+            it.next();
+            count ++;
         }
+        Assert.assertEquals(count, 9);
     }
 
     @Test

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7efaffa/test/pig/org/apache/cassandra/pig/ThriftColumnFamilyDataTypeTest.java
----------------------------------------------------------------------
diff --git a/test/pig/org/apache/cassandra/pig/ThriftColumnFamilyDataTypeTest.java b/test/pig/org/apache/cassandra/pig/ThriftColumnFamilyDataTypeTest.java
index 7bccc23..cc54620 100644
--- a/test/pig/org/apache/cassandra/pig/ThriftColumnFamilyDataTypeTest.java
+++ b/test/pig/org/apache/cassandra/pig/ThriftColumnFamilyDataTypeTest.java
@@ -17,27 +17,6 @@
  * under the License.
  */
 package org.apache.cassandra.pig;
-/*
- * 
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *   http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- * 
- */
-
 
 import java.io.IOException;
 import java.nio.charset.CharacterCodingException;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f7efaffa/test/pig/org/apache/cassandra/pig/ThriftColumnFamilyTest.java
----------------------------------------------------------------------
diff --git a/test/pig/org/apache/cassandra/pig/ThriftColumnFamilyTest.java b/test/pig/org/apache/cassandra/pig/ThriftColumnFamilyTest.java
index 223cbf4..e114d37 100644
--- a/test/pig/org/apache/cassandra/pig/ThriftColumnFamilyTest.java
+++ b/test/pig/org/apache/cassandra/pig/ThriftColumnFamilyTest.java
@@ -29,7 +29,6 @@ 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;
@@ -210,9 +209,8 @@ public class ThriftColumnFamilyTest extends PigTestBase
         while (it.hasNext()) {
             count ++;
             Tuple t = it.next();
-            if (count == 1)
+            if ("bar".equals(t.get(0)))
             {
-                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);
@@ -220,16 +218,16 @@ public class ThriftColumnFamilyTest extends PigTestBase
                 Assert.assertEquals(t.get(6), 15000L);
                 Assert.assertEquals(t.get(7), "like");
             }
-            else if (count == 2)
+            else if ("baz".equals(t.get(0)))
             {
-                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)
+            }
+            else if ("foo".equals(t.get(0)))
             {
                 Assert.assertEquals(t.get(0), "foo");
                 Assert.assertEquals(t.get(1), 2.718281828459045d);
@@ -239,7 +237,7 @@ public class ThriftColumnFamilyTest extends PigTestBase
                 Assert.assertEquals(t.get(6), 125000L);
                 Assert.assertEquals(t.get(7), "like");
             }
-            else if (count == 4)
+            else if ("qux".equals(t.get(0)))
             {
                 Assert.assertEquals(t.get(0), "qux");
                 Assert.assertEquals(t.get(1), 0.660161815846869d);
@@ -264,18 +262,10 @@ public class ThriftColumnFamilyTest extends PigTestBase
         while (it.hasNext()) {
             count ++;
             Tuple t = it.next();
-            if (count == 1)
-            {
-                Assert.assertEquals(t.get(0), "chuck");
-                Assert.assertEquals(t.get(1), "fist");
+            if ("chuck".equals(t.get(0)) && "fist".equals(t.get(1)))
                 Assert.assertEquals(t.get(2), 1L);
-            }
-            else if (count == 2)
-            {
-                Assert.assertEquals(t.get(0), "chuck");
-                Assert.assertEquals(t.get(1), "kick");
+            else if ("chuck".equals(t.get(0)) && "kick".equals(t.get(1)))
                 Assert.assertEquals(t.get(2), 3L);
-            }
         }
         Assert.assertEquals(count, 2);
 
@@ -293,34 +283,14 @@ public class ThriftColumnFamilyTest extends PigTestBase
         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");
+            if ("kick".equals(t.get(0)) && "bruce".equals(t.get(1)) && "bruce".equals(t.get(2)))
                 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");
+            else if ("kick".equals(t.get(0)) && "bruce".equals(t.get(1)) && "lee".equals(t.get(2)))
                 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");
+            else if ("punch".equals(t.get(0)) && "bruce".equals(t.get(1)) && "bruce".equals(t.get(2)))
                 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");
+            else if ("punch".equals(t.get(0)) && "bruce".equals(t.get(1)) && "lee".equals(t.get(2)))
                 Assert.assertEquals(t.get(3), "ouch");
-            }
         }
         Assert.assertEquals(count, 4);
     }
@@ -340,7 +310,6 @@ public class ThriftColumnFamilyTest extends PigTestBase
         //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();
@@ -623,16 +592,12 @@ public class ThriftColumnFamilyTest extends PigTestBase
         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)
+            if ("kick".equals(t.get(0)))
                 Assert.assertEquals(t.get(2), "oww");
-            else
+            else if ("kick".equals(t.get(0)))
                 Assert.assertEquals(t.get(2), "ouch");
         }
         Assert.assertEquals(count, 2);
@@ -668,30 +633,14 @@ public class ThriftColumnFamilyTest extends PigTestBase
                 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);
+                if ((Long) inner.get(0) == 1L && (Long) 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);
+                else if ((Long) inner.get(0) == 1L && (Long) 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);
+                else if ((Long) inner.get(0) == 2L && (Long) 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);
+                else if ((Long) inner.get(0) == 6L && (Long) inner.get(1) == 30L)
                     Assert.assertEquals(t1.get(1), "coffee...");
-                }
             }
             Assert.assertEquals(count, 4);
         }
@@ -733,11 +682,9 @@ public class ThriftColumnFamilyTest extends PigTestBase
         while (it.hasNext()) {
             Tuple t = it.next();
             count ++;
-            if (count == 1)
+            Tuple key = (Tuple) t.get(0); 
+            if ("clock".equals(key.get(0)) && (Long) key.get(1) == 10L)
             {
-                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())
@@ -747,46 +694,37 @@ public class ThriftColumnFamilyTest extends PigTestBase
                     Assert.assertEquals(t1.get(1), "z");
                 }
             }
-            else if (count == 2)
+            else if ("clock".equals(key.get(0)) && (Long) key.get(1) == 40L)
             {
-                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");
+                    Assert.assertEquals(t1.get(0), 6L);
+                    Assert.assertEquals(t1.get(1), "coffee...");
                 }
             }
-            else if (count == 3)
+            else if ("clock".equals(key.get(0)) && (Long) key.get(1) == 20L)
             {
-                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?");
+                    Assert.assertEquals(t1.get(0), 1L);
+                    Assert.assertEquals(t1.get(1), "zzzz");
                 }
             }
-            else if (count == 4)
+            else if ("clock".equals(key.get(0)) && (Long) key.get(1) == 30L)
             {
-                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(t1.get(0), 2L);
+                    Assert.assertEquals(t1.get(1), "daddy?");
                 }
             }
         }
@@ -800,10 +738,7 @@ public class ThriftColumnFamilyTest extends PigTestBase
         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