You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@bookkeeper.apache.org by yo...@apache.org on 2022/08/01 13:44:53 UTC

[bookkeeper] 07/17: Fix wrong ledger id parse radix for index relocation file in IndexPersistenceMgr (#2944)

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

yong pushed a commit to branch branch-4.15
in repository https://gitbox.apache.org/repos/asf/bookkeeper.git

commit d58fd7adcbb6d5b4e7e39d9208c60be977f7fbef
Author: Kezhu Wang <ke...@gmail.com>
AuthorDate: Thu Jul 28 10:07:03 2022 +0800

    Fix wrong ledger id parse radix for index relocation file in IndexPersistenceMgr (#2944)
    
    Descriptions of the changes in this PR:
    
    ### Motivation
    1. `IndexPersistenceMgr` fails to start after crashing index file movement.
    
    ### Changes
    1. Add test to assert index file relocation
    2. Fix wrong ledger id parse radix for index relocation file in IndexPersistenceMgr
    
    Master Issue: None
    
    (cherry picked from commit eef34477befc27e01f2e4531677236427ed859ef)
---
 .../bookkeeper/bookie/IndexPersistenceMgr.java     |   5 +-
 .../bookkeeper/bookie/IndexPersistenceMgrTest.java | 127 +++++++++++++++++++--
 2 files changed, 120 insertions(+), 12 deletions(-)

diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/IndexPersistenceMgr.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/IndexPersistenceMgr.java
index ec349f3123..71afd81f61 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/IndexPersistenceMgr.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/IndexPersistenceMgr.java
@@ -274,8 +274,9 @@ public class IndexPersistenceMgr {
                                 // name is the HexString representation of the
                                 // ledgerId.
                                 String ledgerIdInHex = index.getName().replace(RLOC, "").replace(IDX, "");
+                                long ledgerId = Long.parseLong(ledgerIdInHex, 16);
                                 if (index.getName().endsWith(RLOC)) {
-                                    if (findIndexFile(Long.parseLong(ledgerIdInHex)) != null) {
+                                    if (findIndexFile(ledgerId) != null) {
                                         if (!index.delete()) {
                                             LOG.warn("Deleting the rloc file " + index + " failed");
                                         }
@@ -288,7 +289,7 @@ public class IndexPersistenceMgr {
                                         }
                                     }
                                 }
-                                activeLedgers.put(Long.parseLong(ledgerIdInHex, 16), true);
+                                activeLedgers.put(ledgerId, true);
                             }
                         }
                     }
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/IndexPersistenceMgrTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/IndexPersistenceMgrTest.java
index 898ef535b4..105c9c8b65 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/IndexPersistenceMgrTest.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/IndexPersistenceMgrTest.java
@@ -22,10 +22,13 @@ package org.apache.bookkeeper.bookie;
 
 import static java.nio.charset.StandardCharsets.UTF_8;
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
+import static org.mockito.Mockito.doAnswer;
+import static org.mockito.Mockito.spy;
 
 import io.netty.buffer.ByteBuf;
 import io.netty.buffer.ByteBufUtil;
@@ -59,7 +62,7 @@ import org.junit.Test;
 public class IndexPersistenceMgrTest {
 
     ServerConfiguration conf;
-    File journalDir, ledgerDir;
+    File journalDir, ledgerDir1, ledgerDir2;
     LedgerDirsManager ledgerDirsManager;
     LedgerDirsMonitor ledgerMonitor;
 
@@ -68,17 +71,21 @@ public class IndexPersistenceMgrTest {
         journalDir = File.createTempFile("IndexPersistenceMgr", "Journal");
         journalDir.delete();
         journalDir.mkdir();
-        ledgerDir = File.createTempFile("IndexPersistenceMgr", "Ledger");
-        ledgerDir.delete();
-        ledgerDir.mkdir();
+        ledgerDir1 = File.createTempFile("IndexPersistenceMgr", "Ledger1");
+        ledgerDir1.delete();
+        ledgerDir1.mkdir();
+        ledgerDir2 = File.createTempFile("IndexPersistenceMgr", "Ledger2");
+        ledgerDir2.delete();
+        ledgerDir2.mkdir();
         // Create current directories
         BookieImpl.getCurrentDirectory(journalDir).mkdir();
-        BookieImpl.getCurrentDirectory(ledgerDir).mkdir();
+        BookieImpl.getCurrentDirectory(ledgerDir1).mkdir();
+        BookieImpl.getCurrentDirectory(ledgerDir2).mkdir();
 
         conf = new ServerConfiguration();
         conf.setMetadataServiceUri(null);
         conf.setJournalDirName(journalDir.getPath());
-        conf.setLedgerDirNames(new String[] { ledgerDir.getPath() });
+        conf.setLedgerDirNames(new String[] { ledgerDir1.getPath(), ledgerDir2.getPath() });
 
         ledgerDirsManager = new LedgerDirsManager(conf, conf.getLedgerDirs(),
                 new DiskChecker(conf.getDiskUsageThreshold(), conf.getDiskUsageWarnThreshold()));
@@ -92,7 +99,8 @@ public class IndexPersistenceMgrTest {
     public void tearDown() throws Exception {
         ledgerMonitor.shutdown();
         FileUtils.deleteDirectory(journalDir);
-        FileUtils.deleteDirectory(ledgerDir);
+        FileUtils.deleteDirectory(ledgerDir1);
+        FileUtils.deleteDirectory(ledgerDir2);
     }
 
     private IndexPersistenceMgr createIndexPersistenceManager(int openFileLimit) throws Exception {
@@ -338,12 +346,111 @@ public class IndexPersistenceMgrTest {
         }
     }
 
+    @Test
+    public void testIndexFileRelocation() throws Exception {
+        final long ledgerId = Integer.MAX_VALUE;
+        final String ledgerName = IndexPersistenceMgr.getLedgerName(ledgerId);
+
+        IndexPersistenceMgr indexPersistenceMgr = createIndexPersistenceManager(1);
+        preCreateFileInfoForLedgerInDir1(ledgerId, FileInfo.V1);
+
+        ledgerDirsManager.addToFilledDirs(BookieImpl.getCurrentDirectory(ledgerDir1));
+        indexPersistenceMgr.flushLedgerHeader(ledgerId);
+
+        File expectedIndexFile = new File(BookieImpl.getCurrentDirectory(ledgerDir2), ledgerName);
+        CachedFileInfo fileInfo = indexPersistenceMgr.getFileInfo(ledgerId, null);
+        assertTrue(fileInfo.isSameFile(expectedIndexFile));
+        assertFalse(fileInfo.isDeleted());
+
+        indexPersistenceMgr.close();
+
+        // Test startup after clean shutdown.
+        //
+        // Index file should stay in original location.
+        IndexPersistenceMgr indexPersistenceMgr2 = createIndexPersistenceManager(1);
+        CachedFileInfo fileInfo2 = indexPersistenceMgr2.getFileInfo(ledgerId, null);
+        assertTrue(fileInfo2.isSameFile(expectedIndexFile));
+        indexPersistenceMgr2.close();
+    }
+
+    @Test
+    public void testIndexFileRelocationCrashBeforeOriginalFileDeleted() throws Exception {
+        final long ledgerId = Integer.MAX_VALUE;
+        final String ledgerName = IndexPersistenceMgr.getLedgerName(ledgerId);
+        final String reason = "crash before original file deleted";
+
+        try {
+            IndexPersistenceMgr indexPersistenceMgr = createIndexPersistenceManager(1);
+            preCreateFileInfoForLedgerInDir1(ledgerId, FileInfo.V1);
+
+            CachedFileInfo fileInfo = spy(indexPersistenceMgr.getFileInfo(ledgerId, null));
+            doAnswer(invocation -> {
+                throw new RuntimeException(reason);
+            }).when(fileInfo).delete();
+            indexPersistenceMgr.readFileInfoCache.put(ledgerId, fileInfo);
+
+            ledgerDirsManager.addToFilledDirs(BookieImpl.getCurrentDirectory(ledgerDir1));
+            indexPersistenceMgr.flushLedgerHeader(ledgerId);
+            fail("should fail due to " + reason);
+        } catch (RuntimeException ex) {
+            assertEquals(reason, ex.getMessage());
+        }
+
+        // Test startup after:
+        //   1. relocation file created.
+        //   2. crashed with possible corrupted relocation file.
+        //
+        // Index file should stay in original location in this case.
+        IndexPersistenceMgr indexPersistenceMgr2 = createIndexPersistenceManager(1);
+        File expectedIndexFile = new File(BookieImpl.getCurrentDirectory(ledgerDir1), ledgerName);
+        CachedFileInfo fileInfo2 = indexPersistenceMgr2.getFileInfo(ledgerId, null);
+        assertTrue(fileInfo2.isSameFile(expectedIndexFile));
+        indexPersistenceMgr2.close();
+    }
+
+    @Test
+    public void testIndexFileRelocationCrashAfterOriginalFileDeleted() throws Exception {
+        final long ledgerId = Integer.MAX_VALUE;
+        final String ledgerName = IndexPersistenceMgr.getLedgerName(ledgerId);
+        final String reason = "crash after original file deleted";
+
+        try {
+            IndexPersistenceMgr indexPersistenceMgr = createIndexPersistenceManager(1);
+            preCreateFileInfoForLedgerInDir1(ledgerId, FileInfo.V1);
+
+            CachedFileInfo fileInfo = spy(indexPersistenceMgr.getFileInfo(ledgerId, null));
+            doAnswer(invocation -> {
+                invocation.callRealMethod();
+                throw new RuntimeException(reason);
+            }).when(fileInfo).delete();
+            indexPersistenceMgr.readFileInfoCache.put(ledgerId, fileInfo);
+
+            ledgerDirsManager.addToFilledDirs(BookieImpl.getCurrentDirectory(ledgerDir1));
+            indexPersistenceMgr.flushLedgerHeader(ledgerId);
+            fail("should fail due to " + reason);
+        } catch (RuntimeException ex) {
+            assertEquals(reason, ex.getMessage());
+        }
+
+        // Test startup after:
+        //   1. relocation file created, filled and synced.
+        //   2. original index file deleted.
+        //   3. crashed.
+        //
+        // Index file should stay in new location in this case.
+        IndexPersistenceMgr indexPersistenceMgr2 = createIndexPersistenceManager(1);
+        File expectedIndexFile = new File(BookieImpl.getCurrentDirectory(ledgerDir2), ledgerName);
+        CachedFileInfo fileInfo2 = indexPersistenceMgr2.getFileInfo(ledgerId, null);
+        assertTrue(fileInfo2.isSameFile(expectedIndexFile));
+        indexPersistenceMgr2.close();
+    }
+
     void validateFileInfo(IndexPersistenceMgr indexPersistenceMgr, long ledgerId, int headerVersion)
             throws IOException, GeneralSecurityException {
         BookKeeper.DigestType digestType = BookKeeper.DigestType.CRC32;
         boolean getUseV2WireProtocol = true;
 
-        preCreateFileInfoForLedger(ledgerId, headerVersion);
+        preCreateFileInfoForLedgerInDir1(ledgerId, headerVersion);
         DigestManager digestManager = DigestManager.instantiate(ledgerId, masterKey,
                 BookKeeper.DigestType.toProtoDigestType(digestType), UnpooledByteBufAllocator.DEFAULT,
                 getUseV2WireProtocol);
@@ -389,8 +496,8 @@ public class IndexPersistenceMgrTest {
         }
     }
 
-    void preCreateFileInfoForLedger(long ledgerId, int headerVersion) throws IOException {
-        File ledgerCurDir = BookieImpl.getCurrentDirectory(ledgerDir);
+    void preCreateFileInfoForLedgerInDir1(long ledgerId, int headerVersion) throws IOException {
+        File ledgerCurDir = BookieImpl.getCurrentDirectory(ledgerDir1);
         String ledgerName = IndexPersistenceMgr.getLedgerName(ledgerId);
         File indexFile = new File(ledgerCurDir, ledgerName);
         indexFile.getParentFile().mkdirs();