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 2020/11/02 15:58:55 UTC

[cassandra] branch trunk updated: Allow sstableloader to work with tables named "backups"

This is an automated email from the ASF dual-hosted git repository.

brandonwilliams pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/cassandra.git


The following commit(s) were added to refs/heads/trunk by this push:
     new d808412  Allow sstableloader to work with tables named "backups"
d808412 is described below

commit d808412ff9778fe72ea784a8eff7a693908681f9
Author: Ekaterina Dimitrova <ek...@datastax.com>
AuthorDate: Wed Oct 14 16:33:02 2020 -0400

    Allow sstableloader to work with tables named "backups"
    
    Patch by Ekaterina Dimitrova, reviewed by brandonwilliams for
    CASSANDRA-16235
---
 CHANGES.txt                                        |  1 +
 .../apache/cassandra/io/sstable/SSTableLoader.java | 23 ++++++
 src/java/org/apache/cassandra/schema/Schema.java   |  4 +-
 .../cassandra/io/sstable/SSTableLoaderTest.java    | 92 ++++++++++++++++------
 4 files changed, 92 insertions(+), 28 deletions(-)

diff --git a/CHANGES.txt b/CHANGES.txt
index cdc7114..41b23b6 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 4.0-beta4
+ * Fix SSTableloader issue when restoring a table named backups (CASSANDRA-16235)
  * Invalid serialized size for responses caused by increasing message time by 1ms which caused extra bytes in size calculation (CASSANDRA-16103)
  * Throw BufferOverflowException from DataOutputBuffer for better visibility (CASSANDRA-16214)
 
diff --git a/src/java/org/apache/cassandra/io/sstable/SSTableLoader.java b/src/java/org/apache/cassandra/io/sstable/SSTableLoader.java
index ec2a700..47de00c 100644
--- a/src/java/org/apache/cassandra/io/sstable/SSTableLoader.java
+++ b/src/java/org/apache/cassandra/io/sstable/SSTableLoader.java
@@ -97,6 +97,29 @@ public class SSTableLoader implements StreamEventHandler
                                           }
 
                                           TableMetadataRef metadata = client.getTableMetadata(desc.cfname);
+
+                                          if (metadata == null && // we did not find metadata
+                                              directory.getName().equals(Directories.BACKUPS_SUBDIR)) // and it's likely we hit CASSANDRA-16235
+                                          {
+                                              File parentDirectory = directory.getParentFile();
+                                              File parentParentDirectory = parentDirectory != null ? parentDirectory.getParentFile() : null;
+                                              // check that descriptor's cfname and ksname are 1 directory closer to root than they should be
+                                              if (parentDirectory != null &&
+                                                  parentParentDirectory != null &&
+                                                  desc.cfname.equals(parentDirectory.getName()) &&
+                                                  desc.ksname.equals(parentParentDirectory.getName()))
+                                              {
+                                                  Descriptor newDesc = new Descriptor(desc.directory,
+                                                                                      desc.ksname,
+                                                                                      Directories.BACKUPS_SUBDIR,
+                                                                                      desc.generation,
+                                                                                      desc.formatType);
+                                                  metadata = client.getTableMetadata(newDesc.cfname);
+                                                  if (metadata != null)
+                                                      desc = newDesc;
+                                              }
+                                          }
+
                                           if (metadata == null)
                                           {
                                               outputHandler.output(String.format("Skipping file %s: table %s.%s doesn't exist", name, keyspace, desc.cfname));
diff --git a/src/java/org/apache/cassandra/schema/Schema.java b/src/java/org/apache/cassandra/schema/Schema.java
index 4b70f67..9c0c590 100644
--- a/src/java/org/apache/cassandra/schema/Schema.java
+++ b/src/java/org/apache/cassandra/schema/Schema.java
@@ -221,10 +221,8 @@ public final class Schema implements SchemaProvider
     @Override
     public void storeKeyspaceInstance(Keyspace keyspace)
     {
-        if (keyspaceInstances.containsKey(keyspace.getName()))
+        if (keyspaceInstances.putIfAbsent(keyspace.getName(), keyspace) != null)
             throw new IllegalArgumentException(String.format("Keyspace %s was already initialized.", keyspace.getName()));
-
-        keyspaceInstances.put(keyspace.getName(), keyspace);
     }
 
     /**
diff --git a/test/unit/org/apache/cassandra/io/sstable/SSTableLoaderTest.java b/test/unit/org/apache/cassandra/io/sstable/SSTableLoaderTest.java
index 334e476..ac0cda1 100644
--- a/test/unit/org/apache/cassandra/io/sstable/SSTableLoaderTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/SSTableLoaderTest.java
@@ -20,6 +20,7 @@ package org.apache.cassandra.io.sstable;
 import java.io.File;
 import java.util.Collections;
 import java.util.List;
+import java.util.Objects;
 import java.util.concurrent.CountDownLatch;
 
 import com.google.common.io.Files;
@@ -58,17 +59,22 @@ public class SSTableLoaderTest
     public static final String KEYSPACE2 = "SSTableLoaderTest1";
     public static final String CF_STANDARD1 = "Standard1";
     public static final String CF_STANDARD2 = "Standard2";
+    public static final String CF_BACKUPS = Directories.BACKUPS_SUBDIR;
+
+    private static final String schema = "CREATE TABLE %s.%s (key ascii, name ascii, val ascii, val1 ascii, PRIMARY KEY (key, name))";
+    private static final String query = "INSERT INTO %s.%s (key, name, val) VALUES (?, ?, ?)";
 
     private File tmpdir;
 
     @BeforeClass
-    public static void defineSchema() throws Exception
+    public static void defineSchema()
     {
         SchemaLoader.prepareServer();
         SchemaLoader.createKeyspace(KEYSPACE1,
                                     KeyspaceParams.simple(1),
                                     SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD1),
-                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD2));
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD2),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_BACKUPS));
 
         SchemaLoader.createKeyspace(KEYSPACE2,
                 KeyspaceParams.simple(1),
@@ -90,11 +96,11 @@ public class SSTableLoaderTest
         try {
             FileUtils.deleteRecursive(tmpdir);
         } catch (FSWriteError e) {
-            /**
-             * Windows does not allow a mapped file to be deleted, so we probably forgot to clean the buffers somewhere.
-             * We force a GC here to force buffer deallocation, and then try deleting the directory again.
-             * For more information, see: http://bugs.java.com/bugdatabase/view_bug.do?bug_id=4715154
-             * If this is not the problem, the exception will be rethrown anyway.
+            /*
+              Windows does not allow a mapped file to be deleted, so we probably forgot to clean the buffers somewhere.
+              We force a GC here to force buffer deallocation, and then try deleting the directory again.
+              For more information, see: http://bugs.java.com/bugdatabase/view_bug.do?bug_id=4715154
+              If this is not the problem, the exception will be rethrown anyway.
              */
             System.gc();
             FileUtils.deleteRecursive(tmpdir);
@@ -121,13 +127,9 @@ public class SSTableLoaderTest
     @Test
     public void testLoadingSSTable() throws Exception
     {
-        File dataDir = new File(tmpdir.getAbsolutePath() + File.separator + KEYSPACE1 + File.separator + CF_STANDARD1);
-        assert dataDir.mkdirs();
+        File dataDir = dataDir(CF_STANDARD1);
         TableMetadata metadata = Schema.instance.getTableMetadata(KEYSPACE1, CF_STANDARD1);
 
-        String schema = "CREATE TABLE %s.%s (key ascii, name ascii, val ascii, val1 ascii, PRIMARY KEY (key, name))";
-        String query = "INSERT INTO %s.%s (key, name, val) VALUES (?, ?, ?)";
-
         try (CQLSSTableWriter writer = CQLSSTableWriter.builder()
                                                        .inDirectory(dataDir)
                                                        .forTable(String.format(schema, KEYSPACE1, CF_STANDARD1))
@@ -148,9 +150,10 @@ public class SSTableLoaderTest
 
         assertEquals(1, partitions.size());
         assertEquals("key1", AsciiType.instance.getString(partitions.get(0).partitionKey().getKey()));
+        assert metadata != null;
         assertEquals(ByteBufferUtil.bytes("100"), partitions.get(0).getRow(Clustering.make(ByteBufferUtil.bytes("col1")))
-                                                                   .getCell(metadata.getColumn(ByteBufferUtil.bytes("val")))
-                                                                   .buffer());
+                                                            .getCell(metadata.getColumn(ByteBufferUtil.bytes("val")))
+                                                            .buffer());
 
         // The stream future is signalled when the work is complete but before releasing references. Wait for release
         // before cleanup (CASSANDRA-10118).
@@ -160,14 +163,7 @@ public class SSTableLoaderTest
     @Test
     public void testLoadingIncompleteSSTable() throws Exception
     {
-        File dataDir = new File(tmpdir.getAbsolutePath() + File.separator + KEYSPACE1 + File.separator + CF_STANDARD2);
-        assert dataDir.mkdirs();
-
-        //make sure we have no tables...
-        assertTrue(dataDir.listFiles().length == 0);
-
-        String schema = "CREATE TABLE %s.%s (key ascii, name ascii, val ascii, val1 ascii, PRIMARY KEY (key, name))";
-        String query = "INSERT INTO %s.%s (key, name, val) VALUES (?, ?, ?)";
+        File dataDir = dataDir(CF_STANDARD2);
 
         CQLSSTableWriter writer = CQLSSTableWriter.builder()
                                                   .inDirectory(dataDir)
@@ -188,7 +184,7 @@ public class SSTableLoaderTest
         cfs.forceBlockingFlush(); // wait for sstables to be on disk else we won't be able to stream them
 
         //make sure we have some tables...
-        assertTrue(dataDir.listFiles().length > 0);
+        assertTrue(Objects.requireNonNull(dataDir.listFiles()).length > 0);
 
         final CountDownLatch latch = new CountDownLatch(2);
         //writer is still open so loader should not load anything
@@ -246,15 +242,61 @@ public class SSTableLoaderTest
 
         assertEquals(1, partitions.size());
         assertEquals("key1", AsciiType.instance.getString(partitions.get(0).partitionKey().getKey()));
+        assert metadata != null;
+        assertEquals(ByteBufferUtil.bytes("100"), partitions.get(0).getRow(Clustering.make(ByteBufferUtil.bytes("col1")))
+                                                            .getCell(metadata.getColumn(ByteBufferUtil.bytes("val")))
+                                                            .buffer());
+
+        // The stream future is signalled when the work is complete but before releasing references. Wait for release
+        // before cleanup (CASSANDRA-10118).
+        latch.await();
+    }
+
+    @Test
+    public void testLoadingBackupsTable() throws Exception
+    {
+        File dataDir = dataDir(CF_BACKUPS);
+        TableMetadata metadata = Schema.instance.getTableMetadata(KEYSPACE1, CF_BACKUPS);
+
+        try (CQLSSTableWriter writer = CQLSSTableWriter.builder()
+                                                       .inDirectory(dataDir)
+                                                       .forTable(String.format(schema, KEYSPACE1, CF_BACKUPS))
+                                                       .using(String.format(query, KEYSPACE1, CF_BACKUPS))
+                                                       .build())
+        {
+            writer.addRow("key", "col1", "100");
+        }
+
+        ColumnFamilyStore cfs = Keyspace.open(KEYSPACE1).getColumnFamilyStore(CF_BACKUPS);
+        cfs.forceBlockingFlush(); // wait for sstables to be on disk else we won't be able to stream them
+
+        final CountDownLatch latch = new CountDownLatch(1);
+        SSTableLoader loader = new SSTableLoader(dataDir, new TestClient(), new OutputHandler.SystemOutput(false, false));
+        loader.stream(Collections.emptySet(), completionStreamListener(latch)).get();
+
+        List<FilteredPartition> partitions = Util.getAll(Util.cmd(cfs).build());
+
+        assertEquals(1, partitions.size());
+        assertEquals("key", AsciiType.instance.getString(partitions.get(0).partitionKey().getKey()));
+        assert metadata != null;
         assertEquals(ByteBufferUtil.bytes("100"), partitions.get(0).getRow(Clustering.make(ByteBufferUtil.bytes("col1")))
-                .getCell(metadata.getColumn(ByteBufferUtil.bytes("val")))
-                .buffer());
+                                                            .getCell(metadata.getColumn(ByteBufferUtil.bytes("val")))
+                                                            .buffer());
 
         // The stream future is signalled when the work is complete but before releasing references. Wait for release
         // before cleanup (CASSANDRA-10118).
         latch.await();
     }
 
+    private File dataDir(String cf)
+    {
+        File dataDir = new File(tmpdir.getAbsolutePath() + File.separator + SSTableLoaderTest.KEYSPACE1 + File.separator + cf);
+        assert dataDir.mkdirs();
+        //make sure we have no tables...
+        assertEquals(Objects.requireNonNull(dataDir.listFiles()).length, 0);
+        return dataDir;
+    }
+
     StreamEventHandler completionStreamListener(final CountDownLatch latch)
     {
         return new StreamEventHandler() {


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@cassandra.apache.org
For additional commands, e-mail: commits-help@cassandra.apache.org