You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@bookkeeper.apache.org by GitBox <gi...@apache.org> on 2022/06/29 12:39:56 UTC

[GitHub] [bookkeeper] eolivelli commented on a diff in pull request #3372: add and check indexDirs in Cookie meta

eolivelli commented on code in PR #3372:
URL: https://github.com/apache/bookkeeper/pull/3372#discussion_r909578415


##########
bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/CookieIndexDirTest.java:
##########
@@ -0,0 +1,973 @@
+/*
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ *
+ */
+
+package org.apache.bookkeeper.bookie;
+
+import static org.apache.bookkeeper.bookie.UpgradeTest.initV1JournalDirectory;
+import static org.apache.bookkeeper.bookie.UpgradeTest.initV1LedgerDirectory;
+import static org.apache.bookkeeper.bookie.UpgradeTest.initV2JournalDirectory;
+import static org.apache.bookkeeper.bookie.UpgradeTest.initV2LedgerDirectory;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+import com.google.common.collect.Sets;
+import java.io.File;
+import java.io.IOException;
+import java.net.URI;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Random;
+import java.util.Set;
+import org.apache.bookkeeper.bookie.BookieException.InvalidCookieException;
+import org.apache.bookkeeper.client.BookKeeperAdmin;
+import org.apache.bookkeeper.conf.ServerConfiguration;
+import org.apache.bookkeeper.conf.TestBKConfiguration;
+import org.apache.bookkeeper.discover.RegistrationManager;
+import org.apache.bookkeeper.meta.MetadataBookieDriver;
+import org.apache.bookkeeper.meta.MetadataDrivers;
+import org.apache.bookkeeper.stats.NullStatsLogger;
+import org.apache.bookkeeper.test.BookKeeperClusterTestCase;
+import org.apache.bookkeeper.util.BookKeeperConstants;
+import org.apache.bookkeeper.util.PortManager;
+import org.apache.bookkeeper.versioning.LongVersion;
+import org.apache.bookkeeper.versioning.Version;
+import org.apache.bookkeeper.versioning.Versioned;
+import org.apache.commons.io.FileUtils;
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * Test cookies.
+ */
+public class CookieIndexDirTest extends BookKeeperClusterTestCase {
+
+    final int bookiePort = PortManager.nextFreePort();
+
+    public CookieIndexDirTest() {
+        super(0);
+    }
+
+    private String newDirectory() throws Exception {
+        return newDirectory(true);
+    }
+
+    private String newDirectory(boolean createCurDir) throws Exception {
+        File d = tmpDirs.createNew("cookie", "tmpdir");
+        if (createCurDir) {
+            new File(d, "current").mkdirs();
+        }
+        return d.getPath();
+    }
+
+    MetadataBookieDriver metadataBookieDriver;
+    RegistrationManager rm;
+
+    @Override
+    public void setUp() throws Exception {
+        super.setUp();
+        baseConf.setMetadataServiceUri(zkUtil.getMetadataServiceUri());
+        this.metadataBookieDriver = MetadataDrivers.getBookieDriver(
+            URI.create(baseConf.getMetadataServiceUri()));
+        this.metadataBookieDriver.initialize(baseConf, NullStatsLogger.INSTANCE);
+        this.rm = metadataBookieDriver.createRegistrationManager();
+    }
+
+    @Override
+    public void tearDown() throws Exception {
+        super.tearDown();
+        if (rm != null) {
+            rm.close();
+        }
+        if (metadataBookieDriver != null) {
+            metadataBookieDriver.close();
+        }
+    }
+
+    private static List<File> currentDirectoryList(File[] dirs) {
+        return Arrays.asList(BookieImpl.getCurrentDirectories(dirs));
+    }
+
+    private void validateConfig(ServerConfiguration conf) throws Exception {
+        List<File> dirs = new ArrayList<>();
+        for (File f : conf.getJournalDirs()) {
+            File cur = BookieImpl.getCurrentDirectory(f);
+            dirs.add(cur);
+            BookieImpl.checkDirectoryStructure(cur);
+        }
+        for (File f : conf.getLedgerDirs()) {
+            File cur = BookieImpl.getCurrentDirectory(f);
+            dirs.add(cur);
+            BookieImpl.checkDirectoryStructure(cur);
+        }
+        if (conf.getIndexDirs() != null) {
+            for (File f : conf.getIndexDirs()) {
+                File cur = BookieImpl.getCurrentDirectory(f);
+                dirs.add(cur);
+                BookieImpl.checkDirectoryStructure(cur);
+            }
+        }
+        LegacyCookieValidation cookieValidation = new LegacyCookieValidation(conf, rm);
+        cookieValidation.checkCookies(dirs);
+
+    }
+
+    /**
+     * Test starting bookie with clean state.
+     */
+    @Test
+    public void testCleanStart() throws Exception {
+        ServerConfiguration conf = TestBKConfiguration.newServerConfiguration();
+        conf.setJournalDirName(newDirectory(true))
+            .setLedgerDirNames(new String[] { newDirectory(true) })
+            .setIndexDirName(new String[] { newDirectory(true) })
+            .setBookiePort(bookiePort)
+            .setMetadataServiceUri(zkUtil.getMetadataServiceUri());
+
+        validateConfig(conf);
+    }
+
+    /**
+     * Test that if a zookeeper cookie
+     * is different to a local cookie, the bookie
+     * will fail to start.
+     */
+    @Test
+    public void testBadJournalCookie() throws Exception {
+        ServerConfiguration conf1 = TestBKConfiguration.newServerConfiguration()
+            .setJournalDirName(newDirectory())
+            .setLedgerDirNames(new String[] { newDirectory() })
+            .setIndexDirName(new String[] { newDirectory() })
+            .setBookiePort(bookiePort);
+        Cookie.Builder cookieBuilder = Cookie.generateCookie(conf1);
+        Cookie c = cookieBuilder.build();
+        c.writeToRegistrationManager(rm, conf1, Version.NEW);
+
+        String journalDir = newDirectory();
+        String ledgerDir = newDirectory();
+        String indexDir = newDirectory();
+        ServerConfiguration conf2 = TestBKConfiguration.newServerConfiguration();
+        conf2.setJournalDirName(journalDir)
+            .setLedgerDirNames(new String[] { ledgerDir })
+            .setIndexDirName(new String[] { indexDir })
+            .setBookiePort(bookiePort)
+            .setMetadataServiceUri(zkUtil.getMetadataServiceUri());
+        Cookie.Builder cookieBuilder2 = Cookie.generateCookie(conf2);
+        Cookie c2 = cookieBuilder2.build();
+        c2.writeToDirectory(new File(journalDir, "current"));
+        c2.writeToDirectory(new File(ledgerDir, "current"));
+        c2.writeToDirectory(new File(indexDir, "current"));
+
+        try {
+            validateConfig(conf2);
+
+            fail("Shouldn't have been able to start");
+        } catch (InvalidCookieException ice) {
+            // correct behaviour
+        }
+    }
+
+    /**
+     * Test that if a directory is removed from
+     * the configuration, the bookie will fail to
+     * start.
+     */
+    @Test
+    public void testDirectoryMissing() throws Exception {
+        String[] ledgerDirs = new String[] {
+            newDirectory(), newDirectory(), newDirectory() };
+        String[] indexDirs = new String[] {
+            newDirectory(), newDirectory(), newDirectory() };
+        String journalDir = newDirectory();
+        ServerConfiguration conf = TestBKConfiguration.newServerConfiguration();
+        conf.setJournalDirName(journalDir)
+            .setLedgerDirNames(ledgerDirs)
+            .setIndexDirName(indexDirs)
+            .setBookiePort(bookiePort)
+            .setMetadataServiceUri(zkUtil.getMetadataServiceUri());
+
+        validateConfig(conf);
+
+        conf.setLedgerDirNames(new String[] { ledgerDirs[0], ledgerDirs[1] });
+        try {
+            validateConfig(conf);
+            fail("Shouldn't have been able to start");
+        } catch (InvalidCookieException ice) {
+            // correct behaviour
+        }
+
+        conf.setIndexDirName(new String[] { indexDirs[0], indexDirs[1] }).setLedgerDirNames(ledgerDirs);
+        try {
+            validateConfig(conf);
+            fail("Shouldn't have been able to start");
+        } catch (InvalidCookieException ice) {
+            // correct behaviour
+        }
+
+        conf.setJournalDirName(newDirectory()).setLedgerDirNames(ledgerDirs).setIndexDirName(indexDirs);
+        try {
+            validateConfig(conf);
+            fail("Shouldn't have been able to start");
+        } catch (InvalidCookieException ice) {
+            // correct behaviour
+        }
+
+        conf.setJournalDirName(journalDir);
+        validateConfig(conf);
+    }
+
+    /**
+     * Test that if a cookie is missing from a journal directory
+     * the bookie will fail to start.
+     */
+    @Test
+    public void testCookieMissingOnJournalDir() throws Exception {
+        String[] ledgerDirs = new String[] {
+            newDirectory(), newDirectory(), newDirectory() };
+        String[] indexDirs = new String[] {
+                newDirectory(), newDirectory(), newDirectory() };
+        String journalDir = newDirectory();
+        ServerConfiguration conf = TestBKConfiguration.newServerConfiguration();
+        conf.setJournalDirName(journalDir)
+            .setLedgerDirNames(ledgerDirs)
+            .setIndexDirName(indexDirs)
+            .setBookiePort(bookiePort)
+            .setMetadataServiceUri(zkUtil.getMetadataServiceUri());
+
+        validateConfig(conf);
+
+        File cookieFile =
+            new File(BookieImpl.getCurrentDirectory(new File(journalDir)), BookKeeperConstants.VERSION_FILENAME);
+        assertTrue(cookieFile.delete());
+        try {
+            validateConfig(conf);
+            fail("Shouldn't have been able to start");
+        } catch (InvalidCookieException ice) {
+            // correct behaviour
+        }
+    }
+
+    /**
+     * Test that if a cookie is missing from a ledger directory
+     * the bookie will fail to start.
+     */
+    @Test
+    public void testCookieMissingOnLedgerDir() throws Exception {
+        String[] ledgerDirs = new String[] {
+            newDirectory(), newDirectory(), newDirectory() };
+        String journalDir = newDirectory();
+        ServerConfiguration conf = TestBKConfiguration.newServerConfiguration();
+        conf.setJournalDirName(journalDir)
+            .setLedgerDirNames(ledgerDirs)
+            .setBookiePort(bookiePort)
+            .setMetadataServiceUri(zkUtil.getMetadataServiceUri());
+
+        validateConfig(conf);
+
+        File cookieFile =
+            new File(BookieImpl.getCurrentDirectory(new File(ledgerDirs[0])), BookKeeperConstants.VERSION_FILENAME);
+        assertTrue(cookieFile.delete());
+        try {
+            validateConfig(conf);
+            fail("Shouldn't have been able to start");
+        } catch (InvalidCookieException ice) {
+            // correct behaviour
+        }
+    }
+
+    /**
+     * Test that if a cookie is missing from a index directory
+     * the bookie will fail to start.
+     */
+    @Test
+    public void testCookieMissingOnIndexDir() throws Exception {
+        String[] ledgerDirs = new String[] {
+                newDirectory(), newDirectory(), newDirectory() };
+        String[] indexDirs = new String[] {
+                newDirectory(), newDirectory(), newDirectory() };
+        String journalDir = newDirectory();
+        ServerConfiguration conf = TestBKConfiguration.newServerConfiguration();
+        conf.setJournalDirName(journalDir)
+                .setLedgerDirNames(ledgerDirs)
+                .setIndexDirName(indexDirs)
+                .setBookiePort(bookiePort)
+                .setMetadataServiceUri(zkUtil.getMetadataServiceUri());
+
+        validateConfig(conf);
+
+        File cookieFile =
+                new File(BookieImpl.getCurrentDirectory(new File(indexDirs[0])), BookKeeperConstants.VERSION_FILENAME);
+        assertTrue(cookieFile.delete());
+        try {
+            validateConfig(conf);
+            fail("Shouldn't have been able to start");
+        } catch (InvalidCookieException ice) {
+            // correct behaviour
+        }
+    }
+
+    /**
+     * Test that if a ledger directory is added to a
+     * preexisting bookie, the bookie will fail
+     * to start.
+     */
+    @Test
+    public void testLedgerDirectoryAdded() throws Exception {
+        String ledgerDir0 = newDirectory();
+        String indexDir0 = newDirectory();
+        String journalDir = newDirectory();
+        ServerConfiguration conf = TestBKConfiguration.newServerConfiguration();
+        conf.setJournalDirName(journalDir)
+            .setLedgerDirNames(new String[] { ledgerDir0 })
+            .setIndexDirName(new String[] { indexDir0 })
+            .setBookiePort(bookiePort)
+            .setMetadataServiceUri(zkUtil.getMetadataServiceUri());
+
+        validateConfig(conf);
+
+        conf.setLedgerDirNames(new String[] { ledgerDir0, newDirectory() });
+        try {
+            validateConfig(conf);
+            fail("Shouldn't have been able to start");
+        } catch (InvalidCookieException ice) {
+            // correct behaviour
+        }
+
+        conf.setLedgerDirNames(new String[] { ledgerDir0 });
+        validateConfig(conf);
+    }
+
+    /**
+     * Test that if a index directory is added to a
+     * preexisting bookie, the bookie will fail
+     * to start.
+     */
+    @Test
+    public void testIndexDirectoryAdded() throws Exception {
+        String ledgerDir0 = newDirectory();
+        String indexDir0 = newDirectory();
+        String journalDir = newDirectory();
+        ServerConfiguration conf = TestBKConfiguration.newServerConfiguration();
+        conf.setJournalDirName(journalDir)
+                .setLedgerDirNames(new String[] { ledgerDir0 })
+                .setIndexDirName(new String[] { indexDir0 })
+                .setBookiePort(bookiePort)
+                .setMetadataServiceUri(zkUtil.getMetadataServiceUri());
+
+        validateConfig(conf);
+
+        conf.setIndexDirName(new String[] { indexDir0, newDirectory() });
+        try {
+            validateConfig(conf);
+            fail("Shouldn't have been able to start");
+        } catch (InvalidCookieException ice) {
+            // correct behaviour

Review Comment:
   is there a way to add a better assertion here (and in other similar points) ? it may have failed for other reasons



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org