cassandra-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From mish...@apache.org
Subject [4/6] git commit: Merge branch 'cassandra-2.1.0' into cassandra-2.1
Date Fri, 15 Aug 2014 20:26:57 GMT
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 <mishail@apache.org>
Authored: Fri Aug 15 13:20:10 2014 -0700
Committer: Mikhail Stepura <mishail@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));
+             }
+         };
+         
+     }
  }


Mime
View raw message