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

[2/3] git commit: SSTableExport uses correct validator to create string representation of partition keys patch by Alexey Zotov; reviewed by Benjamin Lerer for CASSANDRA-7498

SSTableExport uses correct validator to create string representation of partition keys
patch by Alexey Zotov; reviewed by Benjamin Lerer for CASSANDRA-7498


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

Branch: refs/heads/trunk
Commit: 91d61b3276ec5ffda85a99009a779f1ebcb4358a
Parents: bd191f1
Author: Jonathan Ellis <jb...@apache.org>
Authored: Tue Aug 5 16:41:01 2014 -0500
Committer: Jonathan Ellis <jb...@apache.org>
Committed: Tue Aug 5 16:41:09 2014 -0500

----------------------------------------------------------------------
 CHANGES.txt                                     |  2 +
 .../apache/cassandra/tools/SSTableExport.java   | 40 ++++++++-------
 .../apache/cassandra/tools/SSTableImport.java   | 23 +++++++--
 .../unit/org/apache/cassandra/SchemaLoader.java |  3 +-
 test/unit/org/apache/cassandra/Util.java        |  6 +++
 .../cassandra/tools/SSTableExportTest.java      | 52 +++++++++++++++++---
 .../cassandra/tools/SSTableImportTest.java      | 40 ++++++++++++++-
 7 files changed, 132 insertions(+), 34 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/91d61b32/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index d3e2f5a..79c84b2 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,6 @@
 2.1.1
+ * 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)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/91d61b32/src/java/org/apache/cassandra/tools/SSTableExport.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/SSTableExport.java b/src/java/org/apache/cassandra/tools/SSTableExport.java
index 41e9fdc..c4d693c 100644
--- a/src/java/org/apache/cassandra/tools/SSTableExport.java
+++ b/src/java/org/apache/cassandra/tools/SSTableExport.java
@@ -38,9 +38,6 @@ import org.apache.cassandra.utils.ByteBufferUtil;
 import org.codehaus.jackson.JsonGenerator;
 import org.codehaus.jackson.map.ObjectMapper;
 
-import static org.apache.cassandra.utils.ByteBufferUtil.bytesToHex;
-import static org.apache.cassandra.utils.ByteBufferUtil.hexToBytes;
-
 /**
  * Export SSTables to JSON format.
  */
@@ -188,7 +185,7 @@ public class SSTableExport
     {
         out.print("{");
         writeKey(out, "key");
-        writeJSON(out, bytesToHex(key.getKey()));
+        writeJSON(out, metadata.getKeyValidator().getString(key.getKey()));
         out.print(",\n");
 
         if (!deletionInfo.isLive())
@@ -222,9 +219,10 @@ public class SSTableExport
      *
      * @param desc the descriptor of the file to export the rows from
      * @param outs PrintStream to write the output to
+     * @param metadata Metadata to print keys in a proper format
      * @throws IOException on failure to read/write input/output
      */
-    public static void enumeratekeys(Descriptor desc, PrintStream outs)
+    public static void enumeratekeys(Descriptor desc, PrintStream outs, CFMetaData metadata)
     throws IOException
     {
         KeyIterator iter = new KeyIterator(desc);
@@ -238,7 +236,7 @@ public class SSTableExport
                 throw new IOException("Key out of order! " + lastKey + " > " + key);
             lastKey = key;
 
-            outs.println(bytesToHex(key.getKey()));
+            outs.println(metadata.getKeyValidator().getString(key.getKey()));
             checkStream(outs); // flushes
         }
         iter.close();
@@ -251,9 +249,10 @@ public class SSTableExport
      * @param outs     PrintStream to write the output to
      * @param toExport the keys corresponding to the rows to export
      * @param excludes keys to exclude from export
+     * @param metadata Metadata to print keys in a proper format
      * @throws IOException on failure to read/write input/output
      */
-    public static void export(Descriptor desc, PrintStream outs, Collection<String> toExport, String[] excludes) throws IOException
+    public static void export(Descriptor desc, PrintStream outs, Collection<String> toExport, String[] excludes, CFMetaData metadata) throws IOException
     {
         SSTableReader sstable = SSTableReader.open(desc);
         RandomAccessReader dfile = sstable.openDataReader();
@@ -272,7 +271,7 @@ public class SSTableExport
 
         for (String key : toExport)
         {
-            DecoratedKey decoratedKey = partitioner.decorateKey(hexToBytes(key));
+            DecoratedKey decoratedKey = partitioner.decorateKey(metadata.getKeyValidator().fromString(key));
 
             if (lastKey != null && lastKey.compareTo(decoratedKey) > 0)
                 throw new IOException("Key out of order! " + lastKey + " > " + decoratedKey);
@@ -302,7 +301,7 @@ public class SSTableExport
 
     // This is necessary to accommodate the test suite since you cannot open a Reader more
     // than once from within the same process.
-    static void export(SSTableReader reader, PrintStream outs, String[] excludes) throws IOException
+    static void export(SSTableReader reader, PrintStream outs, String[] excludes, CFMetaData metadata) throws IOException
     {
         Set<String> excludeSet = new HashSet<String>();
 
@@ -322,7 +321,7 @@ public class SSTableExport
         {
             row = (SSTableIdentityIterator) scanner.next();
 
-            String currentKey = bytesToHex(row.getKey().getKey());
+            String currentKey = row.getColumnFamily().metadata().getKeyValidator().getString(row.getKey().getKey());
 
             if (excludeSet.contains(currentKey))
                 continue;
@@ -347,11 +346,12 @@ public class SSTableExport
      * @param desc     the descriptor of the sstable to read from
      * @param outs     PrintStream to write the output to
      * @param excludes keys to exclude from export
+     * @param metadata Metadata to print keys in a proper format
      * @throws IOException on failure to read/write input/output
      */
-    public static void export(Descriptor desc, PrintStream outs, String[] excludes) throws IOException
+    public static void export(Descriptor desc, PrintStream outs, String[] excludes, CFMetaData metadata) throws IOException
     {
-        export(SSTableReader.open(desc), outs, excludes);
+        export(SSTableReader.open(desc), outs, excludes, metadata);
     }
 
     /**
@@ -359,11 +359,12 @@ public class SSTableExport
      *
      * @param desc     the descriptor of the sstable to read from
      * @param excludes keys to exclude from export
+     * @param metadata Metadata to print keys in a proper format
      * @throws IOException on failure to read/write SSTable/standard out
      */
-    public static void export(Descriptor desc, String[] excludes) throws IOException
+    public static void export(Descriptor desc, String[] excludes, CFMetaData metadata) throws IOException
     {
-        export(desc, System.out, excludes);
+        export(desc, System.out, excludes, metadata);
     }
 
     /**
@@ -415,7 +416,7 @@ public class SSTableExport
         }
         Keyspace keyspace = Keyspace.open(descriptor.ksname);
 
-        // Make it work for indexes too - find parent cf if necessary
+        // Make it works for indexes too - find parent cf if necessary
         String baseName = descriptor.cfname;
         if (descriptor.cfname.contains("."))
         {
@@ -424,9 +425,10 @@ public class SSTableExport
         }
 
         // IllegalArgumentException will be thrown here if ks/cf pair does not exist
+        ColumnFamilyStore cfStore = null;
         try
         {
-            keyspace.getColumnFamilyStore(baseName);
+            cfStore = keyspace.getColumnFamilyStore(baseName);
         }
         catch (IllegalArgumentException e)
         {
@@ -439,14 +441,14 @@ public class SSTableExport
         {
             if (cmd.hasOption(ENUMERATEKEYS_OPTION))
             {
-                enumeratekeys(descriptor, System.out);
+                enumeratekeys(descriptor, System.out, cfStore.metadata);
             }
             else
             {
                 if ((keys != null) && (keys.length > 0))
-                    export(descriptor, System.out, Arrays.asList(keys), excludes);
+                    export(descriptor, System.out, Arrays.asList(keys), excludes, cfStore.metadata);
                 else
-                    export(descriptor, excludes);
+                    export(descriptor, excludes, cfStore.metadata);
             }
         }
         catch (IOException e)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/91d61b32/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..5352eb4 100644
--- a/src/java/org/apache/cassandra/tools/SSTableImport.java
+++ b/src/java/org/apache/cassandra/tools/SSTableImport.java
@@ -17,8 +17,6 @@
  */
 package org.apache.cassandra.tools;
 
-import static org.apache.cassandra.utils.ByteBufferUtil.hexToBytes;
-
 import java.io.File;
 import java.io.IOException;
 import java.nio.ByteBuffer;
@@ -308,7 +306,7 @@ public class SSTableImport
         for (Object row : data)
         {
             Map<?,?> rowAsMap = (Map<?, ?>)row;
-            decoratedKeys.put(partitioner.decorateKey(hexToBytes((String)rowAsMap.get("key"))), rowAsMap);
+            decoratedKeys.put(partitioner.decorateKey(getKeyValidator(columnFamily).fromString((String) rowAsMap.get("key"))), rowAsMap);
         }
 
         for (Map.Entry<DecoratedKey, Map<?, ?>> row : decoratedKeys.entrySet())
@@ -381,7 +379,7 @@ public class SSTableImport
         {
             String key = parser.getCurrentName();
             Map<?, ?> row = parser.readValueAs(new TypeReference<Map<?, ?>>(){});
-            DecoratedKey currentKey = partitioner.decorateKey(hexToBytes((String) row.get("key")));
+            DecoratedKey currentKey = partitioner.decorateKey(getKeyValidator(columnFamily).fromString((String) row.get("key")));
 
             if (row.containsKey("metadata"))
                 parseMeta((Map<?, ?>) row.get("metadata"), columnFamily, null);
@@ -423,6 +421,21 @@ public class SSTableImport
     }
 
     /**
+     * Get key validator for column family
+     * @param columnFamily column family instance
+     * @return key validator for given column family
+     */
+    private AbstractType<?> getKeyValidator(ColumnFamily columnFamily) {
+        // this is a fix to support backward compatibility
+        // which allows to skip the current key validator
+        // please, take a look onto CASSANDRA-7498 for more details
+        if ("true".equals(System.getProperty("skip.key.validator", "false"))) {
+            return BytesType.instance;
+        }
+        return columnFamily.metadata().getKeyValidator();
+    }
+
+    /**
      * Get JsonParser object for file
      * @param fileName name of the file
      * @return json parser instance for given file
@@ -526,7 +539,7 @@ public class SSTableImport
     {
         try
         {
-            return (type == BytesType.instance) ? hexToBytes(content) : type.fromString(content);
+            return type.fromString(content);
         }
         catch (MarshalException e)
         {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/91d61b32/test/unit/org/apache/cassandra/SchemaLoader.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/SchemaLoader.java b/test/unit/org/apache/cassandra/SchemaLoader.java
index a9d69fd..0334117 100644
--- a/test/unit/org/apache/cassandra/SchemaLoader.java
+++ b/test/unit/org/apache/cassandra/SchemaLoader.java
@@ -180,7 +180,8 @@ public class SchemaLoader
 
                                            standardCFMD(ks1, "UUIDKeys").keyValidator(UUIDType.instance),
                                            CFMetaData.denseCFMetaData(ks1, "MixedTypes", LongType.instance).keyValidator(UUIDType.instance).defaultValidator(BooleanType.instance),
-                                           CFMetaData.denseCFMetaData(ks1, "MixedTypesComposite", composite).keyValidator(composite).defaultValidator(BooleanType.instance)
+                                           CFMetaData.denseCFMetaData(ks1, "MixedTypesComposite", composite).keyValidator(composite).defaultValidator(BooleanType.instance),
+                                           standardCFMD(ks1, "AsciiKeys").keyValidator(AsciiType.instance)
         ));
 
         // Keyspace 2

http://git-wip-us.apache.org/repos/asf/cassandra/blob/91d61b32/test/unit/org/apache/cassandra/Util.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/Util.java b/test/unit/org/apache/cassandra/Util.java
index 88031a4..1015be6 100644
--- a/test/unit/org/apache/cassandra/Util.java
+++ b/test/unit/org/apache/cassandra/Util.java
@@ -43,6 +43,7 @@ import org.apache.cassandra.db.filter.IDiskAtomFilter;
 import org.apache.cassandra.db.filter.QueryFilter;
 import org.apache.cassandra.db.filter.SliceQueryFilter;
 import org.apache.cassandra.db.filter.NamesQueryFilter;
+import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.dht.*;
 import org.apache.cassandra.gms.ApplicationState;
 import org.apache.cassandra.gms.Gossiper;
@@ -65,6 +66,11 @@ public class Util
         return StorageService.getPartitioner().decorateKey(ByteBufferUtil.bytes(key));
     }
 
+    public static DecoratedKey dk(String key, AbstractType type)
+    {
+        return StorageService.getPartitioner().decorateKey(type.fromString(key));
+    }
+
     public static DecoratedKey dk(ByteBuffer key)
     {
         return StorageService.getPartitioner().decorateKey(key);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/91d61b32/test/unit/org/apache/cassandra/tools/SSTableExportTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/tools/SSTableExportTest.java b/test/unit/org/apache/cassandra/tools/SSTableExportTest.java
index d9133f6..c3f3419 100644
--- a/test/unit/org/apache/cassandra/tools/SSTableExportTest.java
+++ b/test/unit/org/apache/cassandra/tools/SSTableExportTest.java
@@ -28,7 +28,6 @@ import static org.apache.cassandra.utils.ByteBufferUtil.hexToBytes;
 import java.io.File;
 import java.io.FileReader;
 import java.io.IOException;
-import java.io.OutputStream;
 import java.io.PrintStream;
 
 import org.apache.cassandra.SchemaLoader;
@@ -36,6 +35,8 @@ import org.apache.cassandra.Util;
 import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.db.*;
 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.UTF8Type;
 import org.apache.cassandra.io.sstable.Descriptor;
 import org.apache.cassandra.io.sstable.SSTableReader;
@@ -77,7 +78,9 @@ public class SSTableExportTest extends SchemaLoader
 
         // Enumerate and verify
         File temp = File.createTempFile("Standard1", ".txt");
-        SSTableExport.enumeratekeys(Descriptor.fromFilename(writer.getFilename()), new PrintStream(temp.getPath()));
+        final Descriptor descriptor = Descriptor.fromFilename(writer.getFilename());
+        SSTableExport.enumeratekeys(descriptor, new PrintStream(temp.getPath()),
+                CFMetaData.sparseCFMetaData(descriptor.ksname, descriptor.cfname, BytesType.instance));
 
 
         try (FileReader file = new FileReader(temp))
@@ -119,7 +122,8 @@ public class SSTableExportTest extends SchemaLoader
 
         // Export to JSON and verify
         File tempJson = File.createTempFile("Standard1", ".json");
-        SSTableExport.export(reader, new PrintStream(tempJson.getPath()), new String[]{asHex("rowExclude")});
+        SSTableExport.export(reader, new PrintStream(tempJson.getPath()), new String[]{asHex("rowExclude")},
+                CFMetaData.sparseCFMetaData("Keyspace1", "Standard1", BytesType.instance));
 
         JSONArray json = (JSONArray)JSONValue.parseWithException(new FileReader(tempJson));
         assertEquals("unexpected number of rows", 2, json.size());
@@ -168,7 +172,8 @@ public class SSTableExportTest extends SchemaLoader
 
         // Export to JSON and verify
         File tempJson = File.createTempFile("Standard1", ".json");
-        SSTableExport.export(reader, new PrintStream(tempJson.getPath()), new String[]{asHex("rowExclude")});
+        SSTableExport.export(reader, new PrintStream(tempJson.getPath()), new String[]{asHex("rowExclude")},
+                CFMetaData.sparseCFMetaData("Keyspace1", "Standard1", BytesType.instance));
 
         // Import JSON to another SSTable file
         File tempSS2 = tempSSTableFile("Keyspace1", "Standard1");
@@ -202,7 +207,8 @@ public class SSTableExportTest extends SchemaLoader
 
         // Export to JSON and verify
         File tempJson = File.createTempFile("Counter1", ".json");
-        SSTableExport.export(reader, new PrintStream(tempJson.getPath()), new String[0]);
+        SSTableExport.export(reader, new PrintStream(tempJson.getPath()), new String[0],
+                CFMetaData.sparseCFMetaData("Keyspace1", "Counter1", BytesType.instance));
         JSONArray json = (JSONArray)JSONValue.parseWithException(new FileReader(tempJson));
         assertEquals("unexpected number of rows", 1, json.size());
 
@@ -233,7 +239,8 @@ public class SSTableExportTest extends SchemaLoader
 
         // Export to JSON and verify
         File tempJson = File.createTempFile("ValuesWithQuotes", ".json");
-        SSTableExport.export(reader, new PrintStream(tempJson.getPath()), new String[0]);
+        SSTableExport.export(reader, new PrintStream(tempJson.getPath()), new String[0],
+                CFMetaData.sparseCFMetaData("Keyspace1", "ValuesWithQuotes", BytesType.instance));
 
         JSONArray json = (JSONArray)JSONValue.parseWithException(new FileReader(tempJson));
         assertEquals("unexpected number of rows", 1, json.size());
@@ -264,7 +271,8 @@ public class SSTableExportTest extends SchemaLoader
         SSTableReader reader = writer.closeAndOpenReader();
         // Export to JSON and verify
         File tempJson = File.createTempFile("CFWithDeletionInfo", ".json");
-        SSTableExport.export(reader, new PrintStream(tempJson.getPath()), new String[0]);
+        SSTableExport.export(reader, new PrintStream(tempJson.getPath()), new String[0],
+                CFMetaData.sparseCFMetaData("Keyspace1", "Counter1", BytesType.instance));
 
         JSONArray json = (JSONArray)JSONValue.parseWithException(new FileReader(tempJson));
         assertEquals("unexpected number of rows", 1, json.size());
@@ -323,7 +331,8 @@ public class SSTableExportTest extends SchemaLoader
         SSTableReader reader = writer.closeAndOpenReader();
         // Export to JSON and verify
         File tempJson = File.createTempFile("CFWithColumnNameEqualToDefaultKeyAlias", ".json");
-        SSTableExport.export(reader, new PrintStream(tempJson.getPath()), new String[0]);
+        SSTableExport.export(reader, new PrintStream(tempJson.getPath()), new String[0],
+                CFMetaData.sparseCFMetaData("Keyspace1", "UUIDKeys", BytesType.instance));
 
         JSONArray json = (JSONArray)JSONValue.parseWithException(new FileReader(tempJson));
         assertEquals(1, json.size());
@@ -337,4 +346,31 @@ public class SSTableExportTest extends SchemaLoader
         assertEquals(CFMetaData.DEFAULT_KEY_ALIAS, ByteBufferUtil.string(hexToBytes((String) col.get(0))));
         assertEquals("not a uuid", ByteBufferUtil.string(hexToBytes((String) col.get(1))));
     }
+
+    @Test
+    public void testAsciiKeyValidator() throws IOException, ParseException
+    {
+        File tempSS = tempSSTableFile("Keyspace1", "AsciiKeys");
+        ColumnFamily cfamily = ArrayBackedSortedColumns.factory.create("Keyspace1", "AsciiKeys");
+        SSTableWriter writer = new SSTableWriter(tempSS.getPath(), 2, ActiveRepairService.UNREPAIRED_SSTABLE);
+
+        // Add a row
+        cfamily.addColumn(column("column", "value", 1L));
+        writer.append(Util.dk("key", AsciiType.instance), cfamily);
+
+        SSTableReader reader = writer.closeAndOpenReader();
+        // Export to JSON and verify
+        File tempJson = File.createTempFile("CFWithAsciiKeys", ".json");
+        SSTableExport.export(reader,
+                             new PrintStream(tempJson.getPath()),
+                             new String[0],
+                             CFMetaData.sparseCFMetaData("Keyspace1", "AsciiKeys", BytesType.instance));
+
+        JSONArray json = (JSONArray)JSONValue.parseWithException(new FileReader(tempJson));
+        assertEquals(1, json.size());
+
+        JSONObject row = (JSONObject)json.get(0);
+        // check row key
+        assertEquals("key", row.get("key"));
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/91d61b32/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..edaf0fd 100644
--- a/test/unit/org/apache/cassandra/tools/SSTableImportTest.java
+++ b/test/unit/org/apache/cassandra/tools/SSTableImportTest.java
@@ -27,6 +27,8 @@ import java.io.IOException;
 import java.net.URI;
 import java.net.URISyntaxException;
 
+import org.apache.cassandra.db.marshal.AsciiType;
+import org.apache.cassandra.db.marshal.BytesType;
 import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
@@ -36,7 +38,6 @@ 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 +139,41 @@ public class SSTableImportTest extends SchemaLoader
         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
+    }
 }