You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by ma...@apache.org on 2021/03/18 08:46:13 UTC

[cassandra] branch cassandra-3.0 updated: Ignore trailing zeros in hint files

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

marcuse pushed a commit to branch cassandra-3.0
in repository https://gitbox.apache.org/repos/asf/cassandra.git


The following commit(s) were added to refs/heads/cassandra-3.0 by this push:
     new 32394e9  Ignore trailing zeros in hint files
32394e9 is described below

commit 32394e9d0ee86c66b7b21a9e9832ab6671b00f6e
Author: Marcus Eriksson <ma...@apache.org>
AuthorDate: Mon Mar 15 13:08:41 2021 +0100

    Ignore trailing zeros in hint files
    
    Patch by marcuse; reviewed by Aleksey Yeschenko for CASSANDRA-16523
---
 CHANGES.txt                                        |   1 +
 .../org/apache/cassandra/hints/HintsReader.java    |  27 ++++-
 .../apache/cassandra/hints/HintsReaderTest.java    | 117 +++++++++++++++++----
 3 files changed, 125 insertions(+), 20 deletions(-)

diff --git a/CHANGES.txt b/CHANGES.txt
index 8aa7708..8cf7f32 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 3.0.25:
+ * Ignore trailing zeros in hint files (CASSANDRA-16523)
  * Refuse DROP COMPACT STORAGE if some 2.x sstables are in use (CASSANDRA-15897)
  * Fix ColumnFilter::toString not returning a valid CQL fragment (CASSANDRA-16483)
  * Fix ColumnFilter behaviour to prevent digest mitmatches during upgrades (CASSANDRA-16415)
diff --git a/src/java/org/apache/cassandra/hints/HintsReader.java b/src/java/org/apache/cassandra/hints/HintsReader.java
index 7003e04..34113ec 100644
--- a/src/java/org/apache/cassandra/hints/HintsReader.java
+++ b/src/java/org/apache/cassandra/hints/HintsReader.java
@@ -36,7 +36,6 @@ import org.apache.cassandra.io.FSReadError;
 import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.utils.AbstractIterator;
 import org.apache.cassandra.utils.ByteBufferUtil;
-import org.apache.cassandra.utils.NativeLibrary;
 
 /**
  * A paged non-compressed hints reader that provides two iterators:
@@ -210,6 +209,14 @@ class HintsReader implements AutoCloseable, Iterable<HintsReader.Page>
             input.resetLimit();
 
             int size = input.readInt();
+            if (size == 0)
+            {
+                // Avoid throwing IOException when a hint file ends with a run of zeros - this
+                // can happen when hard-rebooting unresponsive machines.
+                if (!verifyAllZeros(input))
+                    throw new IOException("Corrupt hint file found");
+                throw new EOFException("Unexpected end of file (size == 0)");
+            }
 
             // if we cannot corroborate the size via crc, then we cannot safely skip this hint
             if (!input.checkCrc())
@@ -307,6 +314,14 @@ class HintsReader implements AutoCloseable, Iterable<HintsReader.Page>
             input.resetLimit();
 
             int size = input.readInt();
+            if (size == 0)
+            {
+                // Avoid throwing IOException when a hint file ends with a run of zeros - this
+                // can happen when hard-rebooting unresponsive machines.
+                if (!verifyAllZeros(input))
+                    throw new IOException("Corrupt hint file found");
+                throw new EOFException("Unexpected end of file (size == 0)");
+            }
 
             // if we cannot corroborate the size via crc, then we cannot safely skip this hint
             if (!input.checkCrc())
@@ -333,4 +348,14 @@ class HintsReader implements AutoCloseable, Iterable<HintsReader.Page>
             return null;
         }
     }
+
+    private static boolean verifyAllZeros(ChecksummedDataInput input) throws IOException
+    {
+        while (!input.isEOF())
+        {
+            if (input.readByte() != 0)
+                return false;
+        }
+        return true;
+    }
 }
diff --git a/test/unit/org/apache/cassandra/hints/HintsReaderTest.java b/test/unit/org/apache/cassandra/hints/HintsReaderTest.java
index 70cf6e7..84b5981 100644
--- a/test/unit/org/apache/cassandra/hints/HintsReaderTest.java
+++ b/test/unit/org/apache/cassandra/hints/HintsReaderTest.java
@@ -22,9 +22,11 @@ import java.io.File;
 import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.nio.file.Files;
+import java.nio.file.StandardOpenOption;
 import java.util.Iterator;
 import java.util.UUID;
 import java.util.concurrent.TimeUnit;
+import java.util.function.Function;
 
 import com.google.common.collect.Iterables;
 import org.junit.BeforeClass;
@@ -35,8 +37,11 @@ import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.db.Mutation;
 import org.apache.cassandra.db.RowUpdateBuilder;
+import org.apache.cassandra.db.UnknownColumnFamilyException;
 import org.apache.cassandra.db.rows.Cell;
 import org.apache.cassandra.db.rows.Row;
+import org.apache.cassandra.io.FSReadError;
+import org.apache.cassandra.io.util.DataInputBuffer;
 import org.apache.cassandra.io.util.FileUtils;
 import org.apache.cassandra.schema.KeyspaceParams;
 
@@ -91,41 +96,115 @@ public class HintsReaderTest
         }
     }
 
-    private void readHints(int num, int numTable) throws IOException
+    private void readHints(int num, int numTable)
+    {
+        readAndVerify(num, numTable, HintsReader.Page::hintsIterator);
+        readAndVerify(num, numTable, this::deserializePageBuffers);
+    }
+
+    private void readAndVerify(int num, int numTable, Function<HintsReader.Page, Iterator<Hint>> getHints)
     {
         long baseTimestamp = descriptor.timestamp;
         int index = 0;
-
         try (HintsReader reader = HintsReader.open(new File(directory, descriptor.fileName())))
         {
             for (HintsReader.Page page : reader)
             {
-                Iterator<Hint> hints = page.hintsIterator();
+                Iterator<Hint> hints = getHints.apply(page);
                 while (hints.hasNext())
                 {
                     int i = index / numTable;
                     Hint hint = hints.next();
+                    if (hint != null)
+                    {
+                        verifyHint(hint, baseTimestamp, i);
+                        index++;
+                    }
+                }
+            }
+        }
+        assertEquals(index, num);
+    }
 
-                    long timestamp = baseTimestamp + i;
-                    Mutation mutation = hint.mutation;
-
-                    assertEquals(timestamp, hint.creationTime);
-                    assertEquals(dk(bytes(i)), mutation.key());
+    private void verifyHint(Hint hint, long baseTimestamp, int i)
+    {
+        long timestamp = baseTimestamp + i;
+        Mutation mutation = hint.mutation;
+
+        assertEquals(timestamp, hint.creationTime);
+        assertEquals(dk(bytes(i)), mutation.key());
+
+        Row row = mutation.getPartitionUpdates().iterator().next().iterator().next();
+        assertEquals(1, Iterables.size(row.cells()));
+        assertEquals(bytes(i), row.clustering().get(0));
+        Cell cell = row.cells().iterator().next();
+        assertNotNull(cell);
+        assertEquals(bytes(i), cell.value());
+        assertEquals(timestamp * 1000, cell.timestamp());
+    }
 
-                    Row row = mutation.getPartitionUpdates().iterator().next().iterator().next();
-                    assertEquals(1, Iterables.size(row.cells()));
-                    assertEquals(bytes(i), row.clustering().get(0));
-                    Cell cell = row.cells().iterator().next();
-                    assertNotNull(cell);
-                    assertEquals(bytes(i), cell.value());
-                    assertEquals(timestamp * 1000, cell.timestamp());
+    private Iterator<Hint> deserializePageBuffers(HintsReader.Page page)
+    {
+        final Iterator<ByteBuffer> buffers = page.buffersIterator();
+        return new Iterator<Hint>()
+        {
+            public boolean hasNext()
+            {
+                return buffers.hasNext();
+            }
 
-                    index++;
+            public Hint next()
+            {
+                try
+                {
+                    return Hint.serializer.deserialize(new DataInputBuffer(buffers.next(), false),
+                                                       descriptor.messagingVersion());
                 }
-            }
-        }
+                catch (UnknownColumnFamilyException e)
+                {
+                    return null;  // ignore
+                }
+                catch (IOException e)
+                {
+                    throw new RuntimeException("Unexpected error deserializing hint", e);
+                }
+            };
+        };
+    }
 
-        assertEquals(index, num);
+    @Test
+    public void corruptFile() throws IOException
+    {
+        corruptFileHelper(new byte[100], "corruptFile");
+    }
+
+    @Test(expected = FSReadError.class)
+    public void corruptFileNotAllZeros() throws IOException
+    {
+        byte [] bs = new byte[100];
+        bs[50] = 1;
+        corruptFileHelper(bs, "corruptFileNotAllZeros");
+    }
+
+    private void corruptFileHelper(byte[] toAppend, String ks) throws IOException
+    {
+        SchemaLoader.createKeyspace(ks,
+                                    KeyspaceParams.simple(1),
+                                    SchemaLoader.standardCFMD(ks, CF_STANDARD1),
+                                    SchemaLoader.standardCFMD(ks, CF_STANDARD2));
+        int numTable = 2;
+        directory = Files.createTempDirectory(null).toFile();
+        try
+        {
+            generateHints(3, ks);
+            File hintFile = new File(directory, descriptor.fileName());
+            Files.write(hintFile.toPath(), toAppend, StandardOpenOption.APPEND);
+            readHints(3 * numTable, numTable);
+        }
+        finally
+        {
+            directory.delete();
+        }
     }
 
     @Test


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