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/28 17:21:56 UTC

[GitHub] [bookkeeper] StevenLuMT opened a new pull request, #3372: add and check indexDirs in Cookie meta

StevenLuMT opened a new pull request, #3372:
URL: https://github.com/apache/bookkeeper/pull/3372

   Descriptions of the changes in this PR:
   
   ### Motivation
   
   planning for index dir: [mail talking](https://lists.apache.org/thread/r657jf55khl59bbqltj2s95107lbkr0w)
   stage 2: add and check indexDirs in cookie meta
   
   ### Changes
   
   1. add indexDirs in cookie meta
   2. check indexDirs in cookie meta
   


-- 
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


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

Posted by GitBox <gi...@apache.org>.
eolivelli commented on PR #3372:
URL: https://github.com/apache/bookkeeper/pull/3372#issuecomment-1170782414

   there should be already a bookkeeper shell command to rewrite the cookie


-- 
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


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

Posted by GitBox <gi...@apache.org>.
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


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

Posted by GitBox <gi...@apache.org>.
StevenLuMT commented on code in PR #3372:
URL: https://github.com/apache/bookkeeper/pull/3372#discussion_r910051793


##########
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
   
   here is expected to throw InvalidCookieException, if not, fail this testcase.
   so InvalidCookieException is enough, I think @eolivelli 



-- 
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


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

Posted by GitBox <gi...@apache.org>.
StevenLuMT commented on PR #3372:
URL: https://github.com/apache/bookkeeper/pull/3372#issuecomment-1169841659

   rerun failure checks


-- 
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


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

Posted by GitBox <gi...@apache.org>.
StevenLuMT commented on PR #3372:
URL: https://github.com/apache/bookkeeper/pull/3372#issuecomment-1169456393

   rerun failure checks


-- 
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


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

Posted by GitBox <gi...@apache.org>.
StevenLuMT commented on PR #3372:
URL: https://github.com/apache/bookkeeper/pull/3372#issuecomment-1169739207

   rerun failure checks


-- 
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


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

Posted by GitBox <gi...@apache.org>.
StevenLuMT commented on PR #3372:
URL: https://github.com/apache/bookkeeper/pull/3372#issuecomment-1169491878

   rerun failure checks


-- 
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


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

Posted by GitBox <gi...@apache.org>.
StevenLuMT commented on PR #3372:
URL: https://github.com/apache/bookkeeper/pull/3372#issuecomment-1169437042

   rerun failure checks


-- 
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


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

Posted by GitBox <gi...@apache.org>.
StevenLuMT commented on PR #3372:
URL: https://github.com/apache/bookkeeper/pull/3372#issuecomment-1169503774

   rerun failure checks


-- 
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


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

Posted by GitBox <gi...@apache.org>.
StevenLuMT commented on PR #3372:
URL: https://github.com/apache/bookkeeper/pull/3372#issuecomment-1169791677

   rerun failure checks


-- 
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


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

Posted by GitBox <gi...@apache.org>.
StevenLuMT commented on PR #3372:
URL: https://github.com/apache/bookkeeper/pull/3372#issuecomment-1170118869

   > great work some questions
   > 
   > 1. how do we deal with rollbacks of versions ?
   > 2. is this change backward compatible ?
   
   @eolivelli 
   good question,the code is backward compatible :
   1. when set the existed config ServerConfiguration#setIndexDirName, the new cookie will work
   2. when not set this config, the old testcase(CookieTest) do the regression testing
   3. if old bookeeper node want to upgrade this cookie, later I will add a new pr for adding a funtion in **cookie_generate**


-- 
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


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

Posted by GitBox <gi...@apache.org>.
StevenLuMT commented on PR #3372:
URL: https://github.com/apache/bookkeeper/pull/3372#issuecomment-1169446244

   rerun failure checks


-- 
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


[GitHub] [bookkeeper] eolivelli merged pull request #3372: add and check indexDirs in Cookie meta

Posted by GitBox <gi...@apache.org>.
eolivelli merged PR #3372:
URL: https://github.com/apache/bookkeeper/pull/3372


-- 
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


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

Posted by GitBox <gi...@apache.org>.
StevenLuMT commented on PR #3372:
URL: https://github.com/apache/bookkeeper/pull/3372#issuecomment-1169652876

   rerun failure checks


-- 
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


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

Posted by GitBox <gi...@apache.org>.
StevenLuMT commented on PR #3372:
URL: https://github.com/apache/bookkeeper/pull/3372#issuecomment-1170898155

   > 3\. adding a funtion in **cookie_generate**
   
   yes , it's cookie_generate(class name is GenerateCookieCommand.java), I will add a param for --index-dirs 
   another new pr @eolivelli 


-- 
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