You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by mi...@apache.org on 2014/08/15 22:26:54 UTC

[1/6] git commit: Handle CQL row marker in SSTableImport

Repository: cassandra
Updated Branches:
  refs/heads/cassandra-2.1 ea686198f -> 141b9399d
  refs/heads/cassandra-2.1.0 23233b384 -> 8137fce52
  refs/heads/trunk 5233948d1 -> f0635da39


Handle CQL row marker in SSTableImport

patch by Mikhail Stepura; reviewed by Tyler Hobbs for CASSANDRA-7477


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

Branch: refs/heads/cassandra-2.1
Commit: 8137fce529fe56db571d288e1e179aff905368de
Parents: 23233b3
Author: Mikhail Stepura <mi...@apache.org>
Authored: Thu Aug 14 16:31:17 2014 -0700
Committer: Mikhail Stepura <mi...@apache.org>
Committed: Fri Aug 15 13:11:47 2014 -0700

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../apache/cassandra/tools/SSTableImport.java   | 12 +++-
 test/resources/CQLTable.json                    | 10 ++++
 .../cassandra/tools/SSTableImportTest.java      | 62 +++++++++++++++++++-
 4 files changed, 81 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/8137fce5/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 5b5283f..8714265 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 2.1.0-rc6
+ * json2sstable couldn't import JSON for CQL table (CASSANDRA-7477)
  * Invalidate all caches on table drop (CASSANDRA-7561)
  * Skip strict endpoint selection for ranges if RF == nodes (CASSANRA-7765)
  * Fix Thrift range filtering without 2ary index lookups (CASSANDRA-7741)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8137fce5/src/java/org/apache/cassandra/tools/SSTableImport.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/SSTableImport.java b/src/java/org/apache/cassandra/tools/SSTableImport.java
index 4e7bf06..6e1415f 100644
--- a/src/java/org/apache/cassandra/tools/SSTableImport.java
+++ b/src/java/org/apache/cassandra/tools/SSTableImport.java
@@ -142,7 +142,11 @@ public class SSTableImport
                 }
                 else
                 {
-                    value = stringAsType((String) fields.get(1), meta.getValueValidator(comparator.cellFromByteBuffer(name)));
+                    assert meta.isCQL3Table() || name.hasRemaining() : "Cell name should not be empty";
+                    value = stringAsType((String) fields.get(1), 
+                            meta.getValueValidator(name.hasRemaining() 
+                                    ? comparator.cellFromByteBuffer(name)
+                                    : meta.comparator.rowMarker(Composites.EMPTY)));
                 }
             }
         }
@@ -215,8 +219,10 @@ public class SSTableImport
                 cfamily.addAtom(new RangeTombstone(start, end, col.timestamp, col.localExpirationTime));
                 continue;
             }
-
-            CellName cname = cfm.comparator.cellFromByteBuffer(col.getName());
+            
+            assert cfm.isCQL3Table() || col.getName().hasRemaining() : "Cell name should not be empty";
+            CellName cname = col.getName().hasRemaining() ? cfm.comparator.cellFromByteBuffer(col.getName()) 
+                    : cfm.comparator.rowMarker(Composites.EMPTY);
 
             if (col.isExpiring())
             {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8137fce5/test/resources/CQLTable.json
----------------------------------------------------------------------
diff --git a/test/resources/CQLTable.json b/test/resources/CQLTable.json
new file mode 100644
index 0000000..af15f70
--- /dev/null
+++ b/test/resources/CQLTable.json
@@ -0,0 +1,10 @@
+[
+{"key": "00000001",
+ "cells": [["","",1408056347831000],
+           ["v1","NY",1408056347831000],
+           ["v2","1980",1408056347831000]]},
+{"key": "00000002",
+ "cells": [["","",1408056347812000],
+           ["v1","CA",1408056347812000],
+           ["v2","2014",1408056347812000]]}
+]

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8137fce5/test/unit/org/apache/cassandra/tools/SSTableImportTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/tools/SSTableImportTest.java b/test/unit/org/apache/cassandra/tools/SSTableImportTest.java
index 2fdeaf4..308a184 100644
--- a/test/unit/org/apache/cassandra/tools/SSTableImportTest.java
+++ b/test/unit/org/apache/cassandra/tools/SSTableImportTest.java
@@ -18,7 +18,11 @@
 */
 package org.apache.cassandra.tools;
 
+import static org.hamcrest.CoreMatchers.is;
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertThat;
+import static org.junit.matchers.JUnitMatchers.hasItem;
+
 import static org.apache.cassandra.io.sstable.SSTableUtils.tempSSTableFile;
 import static org.apache.cassandra.utils.ByteBufferUtil.hexToBytes;
 
@@ -27,16 +31,21 @@ import java.io.IOException;
 import java.net.URI;
 import java.net.URISyntaxException;
 
+import org.hamcrest.Description;
+import org.hamcrest.Matcher;
 import org.junit.Test;
+import org.junit.internal.matchers.TypeSafeMatcher;
 
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
+import org.apache.cassandra.cql3.QueryProcessor;
+import org.apache.cassandra.cql3.UntypedResultSet;
+import org.apache.cassandra.cql3.UntypedResultSet.Row;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.columniterator.OnDiskAtomIterator;
 import org.apache.cassandra.db.filter.QueryFilter;
 import org.apache.cassandra.io.sstable.Descriptor;
 import org.apache.cassandra.io.sstable.SSTableReader;
-import org.apache.cassandra.utils.ByteBufferUtil;
 
 public class SSTableImportTest extends SchemaLoader
 {
@@ -138,4 +147,55 @@ public class SSTableImportTest extends SchemaLoader
         assert c instanceof CounterCell : c;
         assert ((CounterCell) c).total() == 42;
     }
+    
+    @Test
+    /* 
+     *  The schema is 
+     *      CREATE TABLE cql_keyspace.table1 (k int PRIMARY KEY, v1 text, v2 int)
+     * */
+    public void shouldImportCqlTable() throws IOException, URISyntaxException
+    {
+        String cql_keyspace = "cql_keyspace";
+        String cql_table = "table1";
+        String jsonUrl = resourcePath("CQLTable.json");
+        File tempSS = tempSSTableFile(cql_keyspace, cql_table);
+        new SSTableImport(true).importJson(jsonUrl, cql_keyspace, cql_table, tempSS.getPath());
+        SSTableReader reader = SSTableReader.open(Descriptor.fromFilename(tempSS.getPath()));
+        Keyspace.open(cql_keyspace).getColumnFamilyStore(cql_table).addSSTable(reader);
+        
+        UntypedResultSet result = QueryProcessor.executeOnceInternal(String.format("SELECT * FROM %s.%s", cql_keyspace, cql_table));
+        assertThat(result.size(), is(2));
+        assertThat(result, hasItem(withElements(1, "NY", 1980)));
+        assertThat(result, hasItem(withElements(2, "CA", 2014)));
+    }
+
+    @Test(expected=AssertionError.class)
+    public void shouldRejectEmptyCellNamesForNonCqlTables() throws IOException, URISyntaxException
+    {
+        String jsonUrl = resourcePath("CQLTable.json");
+        File tempSS = tempSSTableFile("Keyspace1", "Counter1");
+        new SSTableImport(true).importJson(jsonUrl, "Keyspace1", "Counter1", tempSS.getPath());
+    }
+    
+    private static Matcher<UntypedResultSet.Row> withElements(final int key, final String v1, final int v2) {
+        return new TypeSafeMatcher<UntypedResultSet.Row>()
+        {
+            @Override
+            public boolean matchesSafely(Row input)
+            {
+                if (!input.has("k") || !input.has("v1") || !input.has("v2"))
+                    return false;
+                return input.getInt("k") == key
+                        && input.getString("v1").equals(v1)
+                        && input.getInt("v2") == v2;
+            }
+
+            @Override
+            public void describeTo(Description description)
+            {
+                description.appendText(String.format("a row containing: %s, %s, %s", key, v1, v2));
+            }
+        };
+        
+    }
 }


[2/6] git commit: Handle CQL row marker in SSTableImport

Posted by mi...@apache.org.
Handle CQL row marker in SSTableImport

patch by Mikhail Stepura; reviewed by Tyler Hobbs for CASSANDRA-7477


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

Branch: refs/heads/cassandra-2.1.0
Commit: 8137fce529fe56db571d288e1e179aff905368de
Parents: 23233b3
Author: Mikhail Stepura <mi...@apache.org>
Authored: Thu Aug 14 16:31:17 2014 -0700
Committer: Mikhail Stepura <mi...@apache.org>
Committed: Fri Aug 15 13:11:47 2014 -0700

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../apache/cassandra/tools/SSTableImport.java   | 12 +++-
 test/resources/CQLTable.json                    | 10 ++++
 .../cassandra/tools/SSTableImportTest.java      | 62 +++++++++++++++++++-
 4 files changed, 81 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/8137fce5/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 5b5283f..8714265 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 2.1.0-rc6
+ * json2sstable couldn't import JSON for CQL table (CASSANDRA-7477)
  * Invalidate all caches on table drop (CASSANDRA-7561)
  * Skip strict endpoint selection for ranges if RF == nodes (CASSANRA-7765)
  * Fix Thrift range filtering without 2ary index lookups (CASSANDRA-7741)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8137fce5/src/java/org/apache/cassandra/tools/SSTableImport.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/SSTableImport.java b/src/java/org/apache/cassandra/tools/SSTableImport.java
index 4e7bf06..6e1415f 100644
--- a/src/java/org/apache/cassandra/tools/SSTableImport.java
+++ b/src/java/org/apache/cassandra/tools/SSTableImport.java
@@ -142,7 +142,11 @@ public class SSTableImport
                 }
                 else
                 {
-                    value = stringAsType((String) fields.get(1), meta.getValueValidator(comparator.cellFromByteBuffer(name)));
+                    assert meta.isCQL3Table() || name.hasRemaining() : "Cell name should not be empty";
+                    value = stringAsType((String) fields.get(1), 
+                            meta.getValueValidator(name.hasRemaining() 
+                                    ? comparator.cellFromByteBuffer(name)
+                                    : meta.comparator.rowMarker(Composites.EMPTY)));
                 }
             }
         }
@@ -215,8 +219,10 @@ public class SSTableImport
                 cfamily.addAtom(new RangeTombstone(start, end, col.timestamp, col.localExpirationTime));
                 continue;
             }
-
-            CellName cname = cfm.comparator.cellFromByteBuffer(col.getName());
+            
+            assert cfm.isCQL3Table() || col.getName().hasRemaining() : "Cell name should not be empty";
+            CellName cname = col.getName().hasRemaining() ? cfm.comparator.cellFromByteBuffer(col.getName()) 
+                    : cfm.comparator.rowMarker(Composites.EMPTY);
 
             if (col.isExpiring())
             {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8137fce5/test/resources/CQLTable.json
----------------------------------------------------------------------
diff --git a/test/resources/CQLTable.json b/test/resources/CQLTable.json
new file mode 100644
index 0000000..af15f70
--- /dev/null
+++ b/test/resources/CQLTable.json
@@ -0,0 +1,10 @@
+[
+{"key": "00000001",
+ "cells": [["","",1408056347831000],
+           ["v1","NY",1408056347831000],
+           ["v2","1980",1408056347831000]]},
+{"key": "00000002",
+ "cells": [["","",1408056347812000],
+           ["v1","CA",1408056347812000],
+           ["v2","2014",1408056347812000]]}
+]

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8137fce5/test/unit/org/apache/cassandra/tools/SSTableImportTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/tools/SSTableImportTest.java b/test/unit/org/apache/cassandra/tools/SSTableImportTest.java
index 2fdeaf4..308a184 100644
--- a/test/unit/org/apache/cassandra/tools/SSTableImportTest.java
+++ b/test/unit/org/apache/cassandra/tools/SSTableImportTest.java
@@ -18,7 +18,11 @@
 */
 package org.apache.cassandra.tools;
 
+import static org.hamcrest.CoreMatchers.is;
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertThat;
+import static org.junit.matchers.JUnitMatchers.hasItem;
+
 import static org.apache.cassandra.io.sstable.SSTableUtils.tempSSTableFile;
 import static org.apache.cassandra.utils.ByteBufferUtil.hexToBytes;
 
@@ -27,16 +31,21 @@ import java.io.IOException;
 import java.net.URI;
 import java.net.URISyntaxException;
 
+import org.hamcrest.Description;
+import org.hamcrest.Matcher;
 import org.junit.Test;
+import org.junit.internal.matchers.TypeSafeMatcher;
 
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
+import org.apache.cassandra.cql3.QueryProcessor;
+import org.apache.cassandra.cql3.UntypedResultSet;
+import org.apache.cassandra.cql3.UntypedResultSet.Row;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.columniterator.OnDiskAtomIterator;
 import org.apache.cassandra.db.filter.QueryFilter;
 import org.apache.cassandra.io.sstable.Descriptor;
 import org.apache.cassandra.io.sstable.SSTableReader;
-import org.apache.cassandra.utils.ByteBufferUtil;
 
 public class SSTableImportTest extends SchemaLoader
 {
@@ -138,4 +147,55 @@ public class SSTableImportTest extends SchemaLoader
         assert c instanceof CounterCell : c;
         assert ((CounterCell) c).total() == 42;
     }
+    
+    @Test
+    /* 
+     *  The schema is 
+     *      CREATE TABLE cql_keyspace.table1 (k int PRIMARY KEY, v1 text, v2 int)
+     * */
+    public void shouldImportCqlTable() throws IOException, URISyntaxException
+    {
+        String cql_keyspace = "cql_keyspace";
+        String cql_table = "table1";
+        String jsonUrl = resourcePath("CQLTable.json");
+        File tempSS = tempSSTableFile(cql_keyspace, cql_table);
+        new SSTableImport(true).importJson(jsonUrl, cql_keyspace, cql_table, tempSS.getPath());
+        SSTableReader reader = SSTableReader.open(Descriptor.fromFilename(tempSS.getPath()));
+        Keyspace.open(cql_keyspace).getColumnFamilyStore(cql_table).addSSTable(reader);
+        
+        UntypedResultSet result = QueryProcessor.executeOnceInternal(String.format("SELECT * FROM %s.%s", cql_keyspace, cql_table));
+        assertThat(result.size(), is(2));
+        assertThat(result, hasItem(withElements(1, "NY", 1980)));
+        assertThat(result, hasItem(withElements(2, "CA", 2014)));
+    }
+
+    @Test(expected=AssertionError.class)
+    public void shouldRejectEmptyCellNamesForNonCqlTables() throws IOException, URISyntaxException
+    {
+        String jsonUrl = resourcePath("CQLTable.json");
+        File tempSS = tempSSTableFile("Keyspace1", "Counter1");
+        new SSTableImport(true).importJson(jsonUrl, "Keyspace1", "Counter1", tempSS.getPath());
+    }
+    
+    private static Matcher<UntypedResultSet.Row> withElements(final int key, final String v1, final int v2) {
+        return new TypeSafeMatcher<UntypedResultSet.Row>()
+        {
+            @Override
+            public boolean matchesSafely(Row input)
+            {
+                if (!input.has("k") || !input.has("v1") || !input.has("v2"))
+                    return false;
+                return input.getInt("k") == key
+                        && input.getString("v1").equals(v1)
+                        && input.getInt("v2") == v2;
+            }
+
+            @Override
+            public void describeTo(Description description)
+            {
+                description.appendText(String.format("a row containing: %s, %s, %s", key, v1, v2));
+            }
+        };
+        
+    }
 }


[5/6] git commit: Merge branch 'cassandra-2.1.0' into cassandra-2.1

Posted by mi...@apache.org.
Merge branch 'cassandra-2.1.0' into cassandra-2.1

Conflicts:
	test/unit/org/apache/cassandra/tools/SSTableImportTest.java


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

Branch: refs/heads/cassandra-2.1
Commit: 141b9399d3ae64dc9eaab82a058b6c5afd4d16cf
Parents: ea68619 8137fce
Author: Mikhail Stepura <mi...@apache.org>
Authored: Fri Aug 15 13:20:10 2014 -0700
Committer: Mikhail Stepura <mi...@apache.org>
Committed: Fri Aug 15 13:20:10 2014 -0700

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../apache/cassandra/tools/SSTableImport.java   | 12 +++-
 test/resources/CQLTable.json                    | 10 +++
 .../cassandra/tools/SSTableImportTest.java      | 65 +++++++++++++++++++-
 4 files changed, 83 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/141b9399/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index 9e9b805,8714265..5e184a2
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,32 -1,5 +1,33 @@@
 +2.1.1
 + * Validate IPv6 wildcard addresses properly (CASSANDRA-7680)
 + * (cqlsh) Error when tracing query (CASSANDRA-7613)
 + * Avoid IOOBE when building SyntaxError message snippet (CASSANDRA-7569)
 + * SSTableExport uses correct validator to create string representation of partition
 +   keys (CASSANDRA-7498)
 + * Avoid NPEs when receiving type changes for an unknown keyspace (CASSANDRA-7689)
 + * Add support for custom 2i validation (CASSANDRA-7575)
 + * Pig support for hadoop CqlInputFormat (CASSANDRA-6454)
 + * Add listen_interface and rpc_interface options (CASSANDRA-7417)
 + * Improve schema merge performance (CASSANDRA-7444)
 + * Adjust MT depth based on # of partition validating (CASSANDRA-5263)
 + * Optimise NativeCell comparisons (CASSANDRA-6755)
 + * Configurable client timeout for cqlsh (CASSANDRA-7516)
 + * Include snippet of CQL query near syntax error in messages (CASSANDRA-7111)
 +Merged from 2.0:
 + * (cqlsh) cqlsh should automatically disable tracing when selecting
 +   from system_traces (CASSANDRA-7641)
 + * (Hadoop) Add CqlOutputFormat (CASSANDRA-6927)
 + * Don't depend on cassandra config for nodetool ring (CASSANDRA-7508)
 + * (cqlsh) Fix failing cqlsh formatting tests (CASSANDRA-7703)
 + * Fix IncompatibleClassChangeError from hadoop2 (CASSANDRA-7229)
 + * Add 'nodetool sethintedhandoffthrottlekb' (CASSANDRA-7635)
 + * (cqlsh) Add tab-completion for CREATE/DROP USER IF [NOT] EXISTS (CASSANDRA-7611)
 + * Catch errors when the JVM pulls the rug out from GCInspector (CASSANDRA-5345)
 + * cqlsh fails when version number parts are not int (CASSANDRA-7524)
 +
 +
  2.1.0-rc6
+  * json2sstable couldn't import JSON for CQL table (CASSANDRA-7477)
   * Invalidate all caches on table drop (CASSANDRA-7561)
   * Skip strict endpoint selection for ranges if RF == nodes (CASSANRA-7765)
   * Fix Thrift range filtering without 2ary index lookups (CASSANDRA-7741)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/141b9399/src/java/org/apache/cassandra/tools/SSTableImport.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/141b9399/test/unit/org/apache/cassandra/tools/SSTableImportTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/tools/SSTableImportTest.java
index edaf0fd,308a184..38e5914
--- a/test/unit/org/apache/cassandra/tools/SSTableImportTest.java
+++ b/test/unit/org/apache/cassandra/tools/SSTableImportTest.java
@@@ -36,6 -44,6 +44,8 @@@ import org.apache.cassandra.cql3.Untype
  import org.apache.cassandra.db.*;
  import org.apache.cassandra.db.columniterator.OnDiskAtomIterator;
  import org.apache.cassandra.db.filter.QueryFilter;
++import org.apache.cassandra.db.marshal.AsciiType;
++import org.apache.cassandra.db.marshal.BytesType;
  import org.apache.cassandra.io.sstable.Descriptor;
  import org.apache.cassandra.io.sstable.SSTableReader;
  
@@@ -139,41 -147,55 +149,92 @@@ public class SSTableImportTest extends 
          assert c instanceof CounterCell : c;
          assert ((CounterCell) c).total() == 42;
      }
 +
 +    @Test
 +    public void testImportWithAsciiKeyValidator() throws IOException, URISyntaxException
 +    {
 +        // Import JSON to temp SSTable file
 +        String jsonUrl = resourcePath("SimpleCF.json");
 +        File tempSS = tempSSTableFile("Keyspace1", "AsciiKeys");
 +        new SSTableImport(true).importJson(jsonUrl, "Keyspace1", "AsciiKeys", tempSS.getPath());
 +
 +        // Verify results
 +        SSTableReader reader = SSTableReader.open(Descriptor.fromFilename(tempSS.getPath()));
 +        // check that keys are treated as ascii
 +        QueryFilter qf = QueryFilter.getIdentityFilter(Util.dk("726f7741", AsciiType.instance), "AsciiKeys", System.currentTimeMillis());
 +        OnDiskAtomIterator iter = qf.getSSTableColumnIterator(reader);
 +        assert iter.hasNext(); // "ascii" key exists
 +        QueryFilter qf2 = QueryFilter.getIdentityFilter(Util.dk("726f7741", BytesType.instance), "AsciiKeys", System.currentTimeMillis());
 +        OnDiskAtomIterator iter2 = qf2.getSSTableColumnIterator(reader);
 +        assert !iter2.hasNext(); // "bytes" key does not exist
 +    }
 +
 +    @Test
 +    public void testBackwardCompatibilityOfImportWithAsciiKeyValidator() throws IOException, URISyntaxException
 +    {
 +        // Import JSON to temp SSTable file
 +        String jsonUrl = resourcePath("SimpleCF.json");
 +        File tempSS = tempSSTableFile("Keyspace1", "AsciiKeys");
 +        // To ignore current key validator
 +        System.setProperty("skip.key.validator", "true");
 +        new SSTableImport(true).importJson(jsonUrl, "Keyspace1", "AsciiKeys", tempSS.getPath());
 +
 +        // Verify results
 +        SSTableReader reader = SSTableReader.open(Descriptor.fromFilename(tempSS.getPath()));
 +        // check that keys are treated as bytes
 +        QueryFilter qf = QueryFilter.getIdentityFilter(Util.dk("rowA"), "AsciiKeys", System.currentTimeMillis());
 +        OnDiskAtomIterator iter = qf.getSSTableColumnIterator(reader);
 +        assert iter.hasNext(); // "bytes" key exists
 +    }
+     
+     @Test
+     /* 
+      *  The schema is 
+      *      CREATE TABLE cql_keyspace.table1 (k int PRIMARY KEY, v1 text, v2 int)
+      * */
+     public void shouldImportCqlTable() throws IOException, URISyntaxException
+     {
+         String cql_keyspace = "cql_keyspace";
+         String cql_table = "table1";
+         String jsonUrl = resourcePath("CQLTable.json");
+         File tempSS = tempSSTableFile(cql_keyspace, cql_table);
+         new SSTableImport(true).importJson(jsonUrl, cql_keyspace, cql_table, tempSS.getPath());
+         SSTableReader reader = SSTableReader.open(Descriptor.fromFilename(tempSS.getPath()));
+         Keyspace.open(cql_keyspace).getColumnFamilyStore(cql_table).addSSTable(reader);
+         
+         UntypedResultSet result = QueryProcessor.executeOnceInternal(String.format("SELECT * FROM %s.%s", cql_keyspace, cql_table));
+         assertThat(result.size(), is(2));
+         assertThat(result, hasItem(withElements(1, "NY", 1980)));
+         assertThat(result, hasItem(withElements(2, "CA", 2014)));
+     }
+ 
+     @Test(expected=AssertionError.class)
+     public void shouldRejectEmptyCellNamesForNonCqlTables() throws IOException, URISyntaxException
+     {
+         String jsonUrl = resourcePath("CQLTable.json");
+         File tempSS = tempSSTableFile("Keyspace1", "Counter1");
+         new SSTableImport(true).importJson(jsonUrl, "Keyspace1", "Counter1", tempSS.getPath());
+     }
+     
+     private static Matcher<UntypedResultSet.Row> withElements(final int key, final String v1, final int v2) {
+         return new TypeSafeMatcher<UntypedResultSet.Row>()
+         {
+             @Override
+             public boolean matchesSafely(Row input)
+             {
+                 if (!input.has("k") || !input.has("v1") || !input.has("v2"))
+                     return false;
+                 return input.getInt("k") == key
+                         && input.getString("v1").equals(v1)
+                         && input.getInt("v2") == v2;
+             }
+ 
+             @Override
+             public void describeTo(Description description)
+             {
+                 description.appendText(String.format("a row containing: %s, %s, %s", key, v1, v2));
+             }
+         };
+         
+     }
  }


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

Posted by mi...@apache.org.
Merge branch 'cassandra-2.1' into trunk

Conflicts:
	test/unit/org/apache/cassandra/tools/SSTableImportTest.java


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

Branch: refs/heads/trunk
Commit: f0635da39ddfb68c856db58b698855b9f44e698e
Parents: 5233948 141b939
Author: Mikhail Stepura <mi...@apache.org>
Authored: Fri Aug 15 13:26:40 2014 -0700
Committer: Mikhail Stepura <mi...@apache.org>
Committed: Fri Aug 15 13:26:40 2014 -0700

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../apache/cassandra/tools/SSTableImport.java   | 12 +++-
 test/resources/CQLTable.json                    | 10 +++
 .../cassandra/tools/SSTableImportTest.java      | 72 ++++++++++++++++++--
 4 files changed, 87 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


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

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f0635da3/src/java/org/apache/cassandra/tools/SSTableImport.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f0635da3/test/unit/org/apache/cassandra/tools/SSTableImportTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/tools/SSTableImportTest.java
index 5c4318a,38e5914..01becfe
--- a/test/unit/org/apache/cassandra/tools/SSTableImportTest.java
+++ b/test/unit/org/apache/cassandra/tools/SSTableImportTest.java
@@@ -18,58 -18,39 +18,71 @@@
  */
  package org.apache.cassandra.tools;
  
 -import static org.hamcrest.CoreMatchers.is;
+ import static org.junit.Assert.assertEquals;
++import static org.hamcrest.CoreMatchers.is;
+ import static org.junit.Assert.assertThat;
+ import static org.junit.matchers.JUnitMatchers.hasItem;
+ 
+ import static org.apache.cassandra.io.sstable.SSTableUtils.tempSSTableFile;
+ import static org.apache.cassandra.utils.ByteBufferUtil.hexToBytes;
+ 
  import java.io.File;
  import java.io.IOException;
  import java.net.URI;
  import java.net.URISyntaxException;
  
 +import org.junit.BeforeClass;
+ import org.hamcrest.Description;
+ import org.hamcrest.Matcher;
  import org.junit.Test;
+ import org.junit.internal.matchers.TypeSafeMatcher;
  
  import org.apache.cassandra.SchemaLoader;
  import org.apache.cassandra.Util;
++import org.apache.cassandra.db.*;
 +import org.apache.cassandra.config.CFMetaData;
 +import org.apache.cassandra.config.KSMetaData;
 +import org.apache.cassandra.db.ArrayBackedSortedColumns;
 +import org.apache.cassandra.db.BufferDeletedCell;
 +import org.apache.cassandra.db.Cell;
 +import org.apache.cassandra.db.ColumnFamily;
 +import org.apache.cassandra.db.CounterCell;
 +import org.apache.cassandra.db.DeletionInfo;
 +import org.apache.cassandra.db.ExpiringCell;
+ import org.apache.cassandra.cql3.QueryProcessor;
+ import org.apache.cassandra.cql3.UntypedResultSet;
+ import org.apache.cassandra.cql3.UntypedResultSet.Row;
 -import org.apache.cassandra.db.*;
  import org.apache.cassandra.db.columniterator.OnDiskAtomIterator;
  import org.apache.cassandra.db.filter.QueryFilter;
  import org.apache.cassandra.db.marshal.AsciiType;
  import org.apache.cassandra.db.marshal.BytesType;
 +import org.apache.cassandra.db.marshal.CounterColumnType;
 +import org.apache.cassandra.exceptions.ConfigurationException;
  import org.apache.cassandra.io.sstable.Descriptor;
  import org.apache.cassandra.io.sstable.SSTableReader;
 +import org.apache.cassandra.locator.SimpleStrategy;
 +import org.apache.thrift.TException;
  
- import static org.apache.cassandra.io.sstable.SSTableUtils.tempSSTableFile;
- import static org.apache.cassandra.utils.ByteBufferUtil.hexToBytes;
- import static org.junit.Assert.assertEquals;
- 
 -public class SSTableImportTest extends SchemaLoader
 +public class SSTableImportTest
  {
 +    public static final String KEYSPACE1 = "SSTableImportTest";
 +    public static final String CF_STANDARD = "Standard1";
 +    public static final String CF_COUNTER = "Counter1";
++    public static final String CQL_TABLE = "table1";
 +
 +    @BeforeClass
 +    public static void defineSchema() throws ConfigurationException, IOException, TException
 +    {
 +        SchemaLoader.prepareServer();
 +        SchemaLoader.createKeyspace(KEYSPACE1,
 +                                    SimpleStrategy.class,
 +                                    KSMetaData.optsWithRF(1),
 +                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD),
 +                                    CFMetaData.denseCFMetaData(KEYSPACE1, CF_COUNTER, BytesType.instance).defaultValidator(CounterColumnType.instance),
-                                     SchemaLoader.standardCFMD(KEYSPACE1, "AsciiKeys").keyValidator(AsciiType.instance));
++                                    SchemaLoader.standardCFMD(KEYSPACE1, "AsciiKeys").keyValidator(AsciiType.instance),
++                                    CFMetaData.compile("CREATE TABLE table1 (k int PRIMARY KEY, v1 text, v2 int)", KEYSPACE1));
 +    }
 +
      @Test
      public void testImportSimpleCf() throws IOException, URISyntaxException
      {
@@@ -205,4 -186,55 +218,53 @@@
          OnDiskAtomIterator iter = qf.getSSTableColumnIterator(reader);
          assert iter.hasNext(); // "bytes" key exists
      }
+     
+     @Test
+     /* 
+      *  The schema is 
+      *      CREATE TABLE cql_keyspace.table1 (k int PRIMARY KEY, v1 text, v2 int)
+      * */
+     public void shouldImportCqlTable() throws IOException, URISyntaxException
+     {
 -        String cql_keyspace = "cql_keyspace";
 -        String cql_table = "table1";
+         String jsonUrl = resourcePath("CQLTable.json");
 -        File tempSS = tempSSTableFile(cql_keyspace, cql_table);
 -        new SSTableImport(true).importJson(jsonUrl, cql_keyspace, cql_table, tempSS.getPath());
++        File tempSS = tempSSTableFile(KEYSPACE1, CQL_TABLE);
++        new SSTableImport(true).importJson(jsonUrl, KEYSPACE1, CQL_TABLE, tempSS.getPath());
+         SSTableReader reader = SSTableReader.open(Descriptor.fromFilename(tempSS.getPath()));
 -        Keyspace.open(cql_keyspace).getColumnFamilyStore(cql_table).addSSTable(reader);
++        Keyspace.open(KEYSPACE1).getColumnFamilyStore(CQL_TABLE).addSSTable(reader);
+         
 -        UntypedResultSet result = QueryProcessor.executeOnceInternal(String.format("SELECT * FROM %s.%s", cql_keyspace, cql_table));
++        UntypedResultSet result = QueryProcessor.executeOnceInternal(String.format("SELECT * FROM \"%s\".%s", KEYSPACE1, CQL_TABLE));
+         assertThat(result.size(), is(2));
+         assertThat(result, hasItem(withElements(1, "NY", 1980)));
+         assertThat(result, hasItem(withElements(2, "CA", 2014)));
+     }
+ 
+     @Test(expected=AssertionError.class)
+     public void shouldRejectEmptyCellNamesForNonCqlTables() throws IOException, URISyntaxException
+     {
+         String jsonUrl = resourcePath("CQLTable.json");
+         File tempSS = tempSSTableFile("Keyspace1", "Counter1");
+         new SSTableImport(true).importJson(jsonUrl, "Keyspace1", "Counter1", tempSS.getPath());
+     }
+     
+     private static Matcher<UntypedResultSet.Row> withElements(final int key, final String v1, final int v2) {
+         return new TypeSafeMatcher<UntypedResultSet.Row>()
+         {
+             @Override
+             public boolean matchesSafely(Row input)
+             {
+                 if (!input.has("k") || !input.has("v1") || !input.has("v2"))
+                     return false;
+                 return input.getInt("k") == key
+                         && input.getString("v1").equals(v1)
+                         && input.getInt("v2") == v2;
+             }
+ 
+             @Override
+             public void describeTo(Description description)
+             {
+                 description.appendText(String.format("a row containing: %s, %s, %s", key, v1, v2));
+             }
+         };
+         
+     }
  }


[3/6] git commit: Handle CQL row marker in SSTableImport

Posted by mi...@apache.org.
Handle CQL row marker in SSTableImport

patch by Mikhail Stepura; reviewed by Tyler Hobbs for CASSANDRA-7477


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

Branch: refs/heads/trunk
Commit: 8137fce529fe56db571d288e1e179aff905368de
Parents: 23233b3
Author: Mikhail Stepura <mi...@apache.org>
Authored: Thu Aug 14 16:31:17 2014 -0700
Committer: Mikhail Stepura <mi...@apache.org>
Committed: Fri Aug 15 13:11:47 2014 -0700

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../apache/cassandra/tools/SSTableImport.java   | 12 +++-
 test/resources/CQLTable.json                    | 10 ++++
 .../cassandra/tools/SSTableImportTest.java      | 62 +++++++++++++++++++-
 4 files changed, 81 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/8137fce5/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 5b5283f..8714265 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 2.1.0-rc6
+ * json2sstable couldn't import JSON for CQL table (CASSANDRA-7477)
  * Invalidate all caches on table drop (CASSANDRA-7561)
  * Skip strict endpoint selection for ranges if RF == nodes (CASSANRA-7765)
  * Fix Thrift range filtering without 2ary index lookups (CASSANDRA-7741)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8137fce5/src/java/org/apache/cassandra/tools/SSTableImport.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/SSTableImport.java b/src/java/org/apache/cassandra/tools/SSTableImport.java
index 4e7bf06..6e1415f 100644
--- a/src/java/org/apache/cassandra/tools/SSTableImport.java
+++ b/src/java/org/apache/cassandra/tools/SSTableImport.java
@@ -142,7 +142,11 @@ public class SSTableImport
                 }
                 else
                 {
-                    value = stringAsType((String) fields.get(1), meta.getValueValidator(comparator.cellFromByteBuffer(name)));
+                    assert meta.isCQL3Table() || name.hasRemaining() : "Cell name should not be empty";
+                    value = stringAsType((String) fields.get(1), 
+                            meta.getValueValidator(name.hasRemaining() 
+                                    ? comparator.cellFromByteBuffer(name)
+                                    : meta.comparator.rowMarker(Composites.EMPTY)));
                 }
             }
         }
@@ -215,8 +219,10 @@ public class SSTableImport
                 cfamily.addAtom(new RangeTombstone(start, end, col.timestamp, col.localExpirationTime));
                 continue;
             }
-
-            CellName cname = cfm.comparator.cellFromByteBuffer(col.getName());
+            
+            assert cfm.isCQL3Table() || col.getName().hasRemaining() : "Cell name should not be empty";
+            CellName cname = col.getName().hasRemaining() ? cfm.comparator.cellFromByteBuffer(col.getName()) 
+                    : cfm.comparator.rowMarker(Composites.EMPTY);
 
             if (col.isExpiring())
             {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8137fce5/test/resources/CQLTable.json
----------------------------------------------------------------------
diff --git a/test/resources/CQLTable.json b/test/resources/CQLTable.json
new file mode 100644
index 0000000..af15f70
--- /dev/null
+++ b/test/resources/CQLTable.json
@@ -0,0 +1,10 @@
+[
+{"key": "00000001",
+ "cells": [["","",1408056347831000],
+           ["v1","NY",1408056347831000],
+           ["v2","1980",1408056347831000]]},
+{"key": "00000002",
+ "cells": [["","",1408056347812000],
+           ["v1","CA",1408056347812000],
+           ["v2","2014",1408056347812000]]}
+]

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8137fce5/test/unit/org/apache/cassandra/tools/SSTableImportTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/tools/SSTableImportTest.java b/test/unit/org/apache/cassandra/tools/SSTableImportTest.java
index 2fdeaf4..308a184 100644
--- a/test/unit/org/apache/cassandra/tools/SSTableImportTest.java
+++ b/test/unit/org/apache/cassandra/tools/SSTableImportTest.java
@@ -18,7 +18,11 @@
 */
 package org.apache.cassandra.tools;
 
+import static org.hamcrest.CoreMatchers.is;
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertThat;
+import static org.junit.matchers.JUnitMatchers.hasItem;
+
 import static org.apache.cassandra.io.sstable.SSTableUtils.tempSSTableFile;
 import static org.apache.cassandra.utils.ByteBufferUtil.hexToBytes;
 
@@ -27,16 +31,21 @@ import java.io.IOException;
 import java.net.URI;
 import java.net.URISyntaxException;
 
+import org.hamcrest.Description;
+import org.hamcrest.Matcher;
 import org.junit.Test;
+import org.junit.internal.matchers.TypeSafeMatcher;
 
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
+import org.apache.cassandra.cql3.QueryProcessor;
+import org.apache.cassandra.cql3.UntypedResultSet;
+import org.apache.cassandra.cql3.UntypedResultSet.Row;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.columniterator.OnDiskAtomIterator;
 import org.apache.cassandra.db.filter.QueryFilter;
 import org.apache.cassandra.io.sstable.Descriptor;
 import org.apache.cassandra.io.sstable.SSTableReader;
-import org.apache.cassandra.utils.ByteBufferUtil;
 
 public class SSTableImportTest extends SchemaLoader
 {
@@ -138,4 +147,55 @@ public class SSTableImportTest extends SchemaLoader
         assert c instanceof CounterCell : c;
         assert ((CounterCell) c).total() == 42;
     }
+    
+    @Test
+    /* 
+     *  The schema is 
+     *      CREATE TABLE cql_keyspace.table1 (k int PRIMARY KEY, v1 text, v2 int)
+     * */
+    public void shouldImportCqlTable() throws IOException, URISyntaxException
+    {
+        String cql_keyspace = "cql_keyspace";
+        String cql_table = "table1";
+        String jsonUrl = resourcePath("CQLTable.json");
+        File tempSS = tempSSTableFile(cql_keyspace, cql_table);
+        new SSTableImport(true).importJson(jsonUrl, cql_keyspace, cql_table, tempSS.getPath());
+        SSTableReader reader = SSTableReader.open(Descriptor.fromFilename(tempSS.getPath()));
+        Keyspace.open(cql_keyspace).getColumnFamilyStore(cql_table).addSSTable(reader);
+        
+        UntypedResultSet result = QueryProcessor.executeOnceInternal(String.format("SELECT * FROM %s.%s", cql_keyspace, cql_table));
+        assertThat(result.size(), is(2));
+        assertThat(result, hasItem(withElements(1, "NY", 1980)));
+        assertThat(result, hasItem(withElements(2, "CA", 2014)));
+    }
+
+    @Test(expected=AssertionError.class)
+    public void shouldRejectEmptyCellNamesForNonCqlTables() throws IOException, URISyntaxException
+    {
+        String jsonUrl = resourcePath("CQLTable.json");
+        File tempSS = tempSSTableFile("Keyspace1", "Counter1");
+        new SSTableImport(true).importJson(jsonUrl, "Keyspace1", "Counter1", tempSS.getPath());
+    }
+    
+    private static Matcher<UntypedResultSet.Row> withElements(final int key, final String v1, final int v2) {
+        return new TypeSafeMatcher<UntypedResultSet.Row>()
+        {
+            @Override
+            public boolean matchesSafely(Row input)
+            {
+                if (!input.has("k") || !input.has("v1") || !input.has("v2"))
+                    return false;
+                return input.getInt("k") == key
+                        && input.getString("v1").equals(v1)
+                        && input.getInt("v2") == v2;
+            }
+
+            @Override
+            public void describeTo(Description description)
+            {
+                description.appendText(String.format("a row containing: %s, %s, %s", key, v1, v2));
+            }
+        };
+        
+    }
 }


[4/6] git commit: Merge branch 'cassandra-2.1.0' into cassandra-2.1

Posted by mi...@apache.org.
Merge branch 'cassandra-2.1.0' into cassandra-2.1

Conflicts:
	test/unit/org/apache/cassandra/tools/SSTableImportTest.java


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

Branch: refs/heads/trunk
Commit: 141b9399d3ae64dc9eaab82a058b6c5afd4d16cf
Parents: ea68619 8137fce
Author: Mikhail Stepura <mi...@apache.org>
Authored: Fri Aug 15 13:20:10 2014 -0700
Committer: Mikhail Stepura <mi...@apache.org>
Committed: Fri Aug 15 13:20:10 2014 -0700

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../apache/cassandra/tools/SSTableImport.java   | 12 +++-
 test/resources/CQLTable.json                    | 10 +++
 .../cassandra/tools/SSTableImportTest.java      | 65 +++++++++++++++++++-
 4 files changed, 83 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/141b9399/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index 9e9b805,8714265..5e184a2
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,32 -1,5 +1,33 @@@
 +2.1.1
 + * Validate IPv6 wildcard addresses properly (CASSANDRA-7680)
 + * (cqlsh) Error when tracing query (CASSANDRA-7613)
 + * Avoid IOOBE when building SyntaxError message snippet (CASSANDRA-7569)
 + * SSTableExport uses correct validator to create string representation of partition
 +   keys (CASSANDRA-7498)
 + * Avoid NPEs when receiving type changes for an unknown keyspace (CASSANDRA-7689)
 + * Add support for custom 2i validation (CASSANDRA-7575)
 + * Pig support for hadoop CqlInputFormat (CASSANDRA-6454)
 + * Add listen_interface and rpc_interface options (CASSANDRA-7417)
 + * Improve schema merge performance (CASSANDRA-7444)
 + * Adjust MT depth based on # of partition validating (CASSANDRA-5263)
 + * Optimise NativeCell comparisons (CASSANDRA-6755)
 + * Configurable client timeout for cqlsh (CASSANDRA-7516)
 + * Include snippet of CQL query near syntax error in messages (CASSANDRA-7111)
 +Merged from 2.0:
 + * (cqlsh) cqlsh should automatically disable tracing when selecting
 +   from system_traces (CASSANDRA-7641)
 + * (Hadoop) Add CqlOutputFormat (CASSANDRA-6927)
 + * Don't depend on cassandra config for nodetool ring (CASSANDRA-7508)
 + * (cqlsh) Fix failing cqlsh formatting tests (CASSANDRA-7703)
 + * Fix IncompatibleClassChangeError from hadoop2 (CASSANDRA-7229)
 + * Add 'nodetool sethintedhandoffthrottlekb' (CASSANDRA-7635)
 + * (cqlsh) Add tab-completion for CREATE/DROP USER IF [NOT] EXISTS (CASSANDRA-7611)
 + * Catch errors when the JVM pulls the rug out from GCInspector (CASSANDRA-5345)
 + * cqlsh fails when version number parts are not int (CASSANDRA-7524)
 +
 +
  2.1.0-rc6
+  * json2sstable couldn't import JSON for CQL table (CASSANDRA-7477)
   * Invalidate all caches on table drop (CASSANDRA-7561)
   * Skip strict endpoint selection for ranges if RF == nodes (CASSANRA-7765)
   * Fix Thrift range filtering without 2ary index lookups (CASSANDRA-7741)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/141b9399/src/java/org/apache/cassandra/tools/SSTableImport.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/141b9399/test/unit/org/apache/cassandra/tools/SSTableImportTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/tools/SSTableImportTest.java
index edaf0fd,308a184..38e5914
--- a/test/unit/org/apache/cassandra/tools/SSTableImportTest.java
+++ b/test/unit/org/apache/cassandra/tools/SSTableImportTest.java
@@@ -36,6 -44,6 +44,8 @@@ import org.apache.cassandra.cql3.Untype
  import org.apache.cassandra.db.*;
  import org.apache.cassandra.db.columniterator.OnDiskAtomIterator;
  import org.apache.cassandra.db.filter.QueryFilter;
++import org.apache.cassandra.db.marshal.AsciiType;
++import org.apache.cassandra.db.marshal.BytesType;
  import org.apache.cassandra.io.sstable.Descriptor;
  import org.apache.cassandra.io.sstable.SSTableReader;
  
@@@ -139,41 -147,55 +149,92 @@@ public class SSTableImportTest extends 
          assert c instanceof CounterCell : c;
          assert ((CounterCell) c).total() == 42;
      }
 +
 +    @Test
 +    public void testImportWithAsciiKeyValidator() throws IOException, URISyntaxException
 +    {
 +        // Import JSON to temp SSTable file
 +        String jsonUrl = resourcePath("SimpleCF.json");
 +        File tempSS = tempSSTableFile("Keyspace1", "AsciiKeys");
 +        new SSTableImport(true).importJson(jsonUrl, "Keyspace1", "AsciiKeys", tempSS.getPath());
 +
 +        // Verify results
 +        SSTableReader reader = SSTableReader.open(Descriptor.fromFilename(tempSS.getPath()));
 +        // check that keys are treated as ascii
 +        QueryFilter qf = QueryFilter.getIdentityFilter(Util.dk("726f7741", AsciiType.instance), "AsciiKeys", System.currentTimeMillis());
 +        OnDiskAtomIterator iter = qf.getSSTableColumnIterator(reader);
 +        assert iter.hasNext(); // "ascii" key exists
 +        QueryFilter qf2 = QueryFilter.getIdentityFilter(Util.dk("726f7741", BytesType.instance), "AsciiKeys", System.currentTimeMillis());
 +        OnDiskAtomIterator iter2 = qf2.getSSTableColumnIterator(reader);
 +        assert !iter2.hasNext(); // "bytes" key does not exist
 +    }
 +
 +    @Test
 +    public void testBackwardCompatibilityOfImportWithAsciiKeyValidator() throws IOException, URISyntaxException
 +    {
 +        // Import JSON to temp SSTable file
 +        String jsonUrl = resourcePath("SimpleCF.json");
 +        File tempSS = tempSSTableFile("Keyspace1", "AsciiKeys");
 +        // To ignore current key validator
 +        System.setProperty("skip.key.validator", "true");
 +        new SSTableImport(true).importJson(jsonUrl, "Keyspace1", "AsciiKeys", tempSS.getPath());
 +
 +        // Verify results
 +        SSTableReader reader = SSTableReader.open(Descriptor.fromFilename(tempSS.getPath()));
 +        // check that keys are treated as bytes
 +        QueryFilter qf = QueryFilter.getIdentityFilter(Util.dk("rowA"), "AsciiKeys", System.currentTimeMillis());
 +        OnDiskAtomIterator iter = qf.getSSTableColumnIterator(reader);
 +        assert iter.hasNext(); // "bytes" key exists
 +    }
+     
+     @Test
+     /* 
+      *  The schema is 
+      *      CREATE TABLE cql_keyspace.table1 (k int PRIMARY KEY, v1 text, v2 int)
+      * */
+     public void shouldImportCqlTable() throws IOException, URISyntaxException
+     {
+         String cql_keyspace = "cql_keyspace";
+         String cql_table = "table1";
+         String jsonUrl = resourcePath("CQLTable.json");
+         File tempSS = tempSSTableFile(cql_keyspace, cql_table);
+         new SSTableImport(true).importJson(jsonUrl, cql_keyspace, cql_table, tempSS.getPath());
+         SSTableReader reader = SSTableReader.open(Descriptor.fromFilename(tempSS.getPath()));
+         Keyspace.open(cql_keyspace).getColumnFamilyStore(cql_table).addSSTable(reader);
+         
+         UntypedResultSet result = QueryProcessor.executeOnceInternal(String.format("SELECT * FROM %s.%s", cql_keyspace, cql_table));
+         assertThat(result.size(), is(2));
+         assertThat(result, hasItem(withElements(1, "NY", 1980)));
+         assertThat(result, hasItem(withElements(2, "CA", 2014)));
+     }
+ 
+     @Test(expected=AssertionError.class)
+     public void shouldRejectEmptyCellNamesForNonCqlTables() throws IOException, URISyntaxException
+     {
+         String jsonUrl = resourcePath("CQLTable.json");
+         File tempSS = tempSSTableFile("Keyspace1", "Counter1");
+         new SSTableImport(true).importJson(jsonUrl, "Keyspace1", "Counter1", tempSS.getPath());
+     }
+     
+     private static Matcher<UntypedResultSet.Row> withElements(final int key, final String v1, final int v2) {
+         return new TypeSafeMatcher<UntypedResultSet.Row>()
+         {
+             @Override
+             public boolean matchesSafely(Row input)
+             {
+                 if (!input.has("k") || !input.has("v1") || !input.has("v2"))
+                     return false;
+                 return input.getInt("k") == key
+                         && input.getString("v1").equals(v1)
+                         && input.getInt("v2") == v2;
+             }
+ 
+             @Override
+             public void describeTo(Description description)
+             {
+                 description.appendText(String.format("a row containing: %s, %s, %s", key, v1, v2));
+             }
+         };
+         
+     }
  }