You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@bookkeeper.apache.org by iv...@apache.org on 2018/01/25 14:12:02 UTC

[bookkeeper] branch master updated: Servers formatted without instance id can't join a cluster with insta…

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

ivank pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/bookkeeper.git


The following commit(s) were added to refs/heads/master by this push:
     new 38f9295  Servers formatted without instance id can't join a cluster with insta…
38f9295 is described below

commit 38f9295c3e1ab2047a82a5181d7f8d5286d4297b
Author: Ivan Kelly <iv...@apache.org>
AuthorDate: Thu Jan 25 15:11:55 2018 +0100

    Servers formatted without instance id can't join a cluster with insta…
    
    Test that old client can continue to work with servers which are using their
    hostname as their bookie id.
    
    This replaces TestBackwardCompat#testCompatWrites and TestBackwardCompat#testCompatReads
    from the old BC tests.
    
    Master Issue: #903
    
    Author: Ivan Kelly <iv...@apache.org>
    
    Reviewers: Sijie Guo <si...@apache.org>
    
    This closes #1034 from ivankelly/bc-cookie
---
 .../backward-compat-old-cookie-new-cluster/pom.xml |  36 ++
 ...mpatUpgradeOldServerInClusterWithCookies.groovy |  83 ++++
 .../src/test/resources/arquillian.xml              |  28 ++
 .../tests/backward/TestBackwardCompat.java         | 440 ---------------------
 tests/pom.xml                                      |   1 +
 5 files changed, 148 insertions(+), 440 deletions(-)

diff --git a/tests/backward-compat-old-cookie-new-cluster/pom.xml b/tests/backward-compat-old-cookie-new-cluster/pom.xml
new file mode 100644
index 0000000..7048c1e
--- /dev/null
+++ b/tests/backward-compat-old-cookie-new-cluster/pom.xml
@@ -0,0 +1,36 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+   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.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="
+  http://maven.apache.org/POM/4.0.0
+  http://maven.apache.org/xsd/maven-4.0.0.xsd">
+  <modelVersion>4.0.0</modelVersion>
+  <parent>
+    <groupId>org.apache.bookkeeper.tests</groupId>
+    <artifactId>integration-tests-base-groovy</artifactId>
+    <version>4.7.0-SNAPSHOT</version>
+    <relativePath>../integration-tests-base-groovy</relativePath>
+  </parent>
+
+  <groupId>org.apache.bookkeeper.tests</groupId>
+  <artifactId>backward-compat-old-cookie-new-cluster</artifactId>
+  <packaging>jar</packaging>
+  <name>Apache BookKeeper :: Tests :: Test upgrade 4.1.0 to current in cluster with cookies</name>
+
+</project>
diff --git a/tests/backward-compat-old-cookie-new-cluster/src/test/groovy/org/apache/bookkeeper/tests/TestCompatUpgradeOldServerInClusterWithCookies.groovy b/tests/backward-compat-old-cookie-new-cluster/src/test/groovy/org/apache/bookkeeper/tests/TestCompatUpgradeOldServerInClusterWithCookies.groovy
new file mode 100644
index 0000000..45cdfd1
--- /dev/null
+++ b/tests/backward-compat-old-cookie-new-cluster/src/test/groovy/org/apache/bookkeeper/tests/TestCompatUpgradeOldServerInClusterWithCookies.groovy
@@ -0,0 +1,83 @@
+/*
+* 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.tests
+
+import com.github.dockerjava.api.DockerClient
+
+import org.jboss.arquillian.junit.Arquillian
+import org.jboss.arquillian.test.api.ArquillianResource
+
+import org.junit.Assert
+import org.junit.Test
+import org.junit.runner.RunWith
+
+import org.slf4j.Logger
+import org.slf4j.LoggerFactory
+
+@RunWith(Arquillian.class)
+class TestCompatUpgradeOldServerInClusterWithCookies {
+    private static final Logger LOG = LoggerFactory.getLogger(TestCompatUpgradeOldServerInClusterWithCookies.class)
+    private static byte[] PASSWD = "foobar".getBytes()
+
+    @ArquillianResource
+    DockerClient docker
+
+    @Test
+    public void testUpgradeServerToClusterWithCookies() throws Exception {
+        BookKeeperClusterUtils.legacyMetadataFormat(docker)
+        String zookeeper = BookKeeperClusterUtils.zookeeperConnectString(docker)
+
+        String currentVersion = System.getProperty("currentVersion")
+        int numEntries = 10
+
+        Assert.assertTrue(BookKeeperClusterUtils.startAllBookiesWithVersion(docker, "4.1.0"))
+        def v410CL = MavenClassLoader.forBookKeeperVersion("4.1.0")
+        def v410BK = v410CL.newBookKeeper(zookeeper)
+
+        def ledger0 = v410BK.createLedger(3, 2, v410CL.digestType("CRC32"), PASSWD)
+        for (int i = 0; i < numEntries; i++) {
+            ledger0.addEntry(("foobar" + i).getBytes())
+        }
+        ledger0.close()
+
+        Assert.assertTrue(BookKeeperClusterUtils.stopAllBookies(docker))
+
+        // format metadata
+        String bookieScript = "/opt/bookkeeper/" + currentVersion + "/bin/bookkeeper"
+        Assert.assertTrue(
+            BookKeeperClusterUtils.runOnAnyBookie(docker, bookieScript,
+                                                  "shell", "metaformat", "-nonInteractive", "-force"))
+
+        // bookies shouldn't come up because the cookie doesn't have instance id
+        Assert.assertFalse(BookKeeperClusterUtils.startAllBookiesWithVersion(docker, currentVersion))
+
+        // format bookie
+        BookKeeperClusterUtils.runOnAllBookies(docker, bookieScript,
+                                               "shell", "bookieformat", "-nonInteractive", "-force")
+
+        // bookies should come up
+        Assert.assertTrue(BookKeeperClusterUtils.startAllBookiesWithVersion(docker, currentVersion))
+
+        // but data has been lost of course, we formatted everything
+        try {
+            v410BK.openLedger(ledger0.getId(), v410CL.digestType("CRC32"), PASSWD)
+        } catch (Exception e) {
+            // correct behaviour
+        }
+    }
+}
diff --git a/tests/backward-compat-old-cookie-new-cluster/src/test/resources/arquillian.xml b/tests/backward-compat-old-cookie-new-cluster/src/test/resources/arquillian.xml
new file mode 100644
index 0000000..f914ff2
--- /dev/null
+++ b/tests/backward-compat-old-cookie-new-cluster/src/test/resources/arquillian.xml
@@ -0,0 +1,28 @@
+<?xml version="1.0"?>
+<!--
+   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.
+-->
+<arquillian xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+            xmlns="http://jboss.org/schema/arquillian"
+            xsi:schemaLocation="http://jboss.org/schema/arquillian
+                                http://jboss.org/schema/arquillian/arquillian_1_0.xsd">
+
+  <extension qualifier="docker">
+    <property name="definitionFormat">CUBE</property>
+    <property name="dockerContainersResource">cube-definitions/3-node-all-version-unstarted.yaml</property>
+  </extension>
+
+</arquillian>
diff --git a/tests/backward/src/test/java/org/apache/bookkeeper/tests/backward/TestBackwardCompat.java b/tests/backward/src/test/java/org/apache/bookkeeper/tests/backward/TestBackwardCompat.java
deleted file mode 100644
index 7d86eb7..0000000
--- a/tests/backward/src/test/java/org/apache/bookkeeper/tests/backward/TestBackwardCompat.java
+++ /dev/null
@@ -1,440 +0,0 @@
-/**
- *
- * 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.tests.backward;
-
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-import java.io.File;
-import java.io.IOException;
-import java.net.InetAddress;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Enumeration;
-import java.util.List;
-import org.apache.bookkeeper.bookie.Bookie;
-import org.apache.bookkeeper.bookie.BookieException;
-import org.apache.bookkeeper.client.BookKeeperAdmin;
-import org.apache.bookkeeper.conf.ClientConfiguration;
-import org.apache.bookkeeper.conf.TestBKConfiguration;
-import org.apache.bookkeeper.test.PortManager;
-import org.apache.bookkeeper.test.ZooKeeperUtil;
-import org.apache.bookkeeper.util.IOUtils;
-import org.apache.commons.io.FileUtils;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-
-/**
- * Test backward compat between versions.
- */
-public class TestBackwardCompat {
-
-    private static final ZooKeeperUtil zkUtil = new ZooKeeperUtil();
-    private static final byte[] ENTRYDATA = "ThisIsAnEntry".getBytes();
-
-    static void waitUp(int port) throws Exception {
-        while (zkUtil.getZooKeeperClient().exists(
-                      "/ledgers/available/" + InetAddress.getLocalHost().getHostAddress() + ":" + port,
-                      false) == null) {
-            Thread.sleep(500);
-        }
-    }
-
-    final List<File> tempDirs = new ArrayList<File>();
-
-    File createTempDir(String prefix, String suffix) throws IOException {
-        File dir = IOUtils.createTempDir(prefix, suffix);
-        tempDirs.add(dir);
-        return dir;
-    }
-
-    @Before
-    public void startZooKeeperServer() throws Exception {
-        zkUtil.startServer();
-    }
-
-    @After
-    public void stopZooKeeperServer() throws Exception {
-        zkUtil.killServer();
-        for (File dir : tempDirs) {
-            FileUtils.deleteDirectory(dir);
-        }
-        tempDirs.clear();
-    }
-
-    /**
-     * Version 4.1.0 classes.
-     */
-    static class Server410 {
-        org.apache.bk_v4_1_0.bookkeeper.conf.ServerConfiguration conf;
-        org.apache.bk_v4_1_0.bookkeeper.proto.BookieServer server = null;
-
-        Server410(File journalDir, File ledgerDir, int port) throws Exception {
-            conf = new org.apache.bk_v4_1_0.bookkeeper.conf.ServerConfiguration();
-            conf.setBookiePort(port);
-            conf.setZkServers(zkUtil.getZooKeeperConnectString());
-            conf.setJournalDirName(journalDir.getPath());
-            conf.setLedgerDirNames(new String[] { ledgerDir.getPath() });
-        }
-
-        void start() throws Exception {
-            server = new org.apache.bk_v4_1_0.bookkeeper.proto.BookieServer(conf);
-            server.start();
-            waitUp(conf.getBookiePort());
-        }
-
-        org.apache.bk_v4_1_0.bookkeeper.conf.ServerConfiguration getConf() {
-            return conf;
-        }
-
-        void stop() throws Exception {
-            if (server != null) {
-                server.shutdown();
-            }
-        }
-    }
-
-    static class Ledger410 {
-        org.apache.bk_v4_1_0.bookkeeper.client.BookKeeper bk;
-        org.apache.bk_v4_1_0.bookkeeper.client.LedgerHandle lh;
-
-        private Ledger410(org.apache.bk_v4_1_0.bookkeeper.client.BookKeeper bk,
-                          org.apache.bk_v4_1_0.bookkeeper.client.LedgerHandle lh) {
-            this.bk = bk;
-            this.lh = lh;
-        }
-
-        static Ledger410 newLedger() throws Exception {
-            org.apache.bk_v4_1_0.bookkeeper.client.BookKeeper newbk =
-                    new org.apache.bk_v4_1_0.bookkeeper.client.BookKeeper(zkUtil.getZooKeeperConnectString());
-            org.apache.bk_v4_1_0.bookkeeper.client.LedgerHandle newlh =
-                    newbk.createLedger(1, 1,
-                                  org.apache.bk_v4_1_0.bookkeeper.client.BookKeeper.DigestType.CRC32,
-                                  "foobar".getBytes());
-            return new Ledger410(newbk, newlh);
-        }
-
-        static Ledger410 openLedger(long id) throws Exception {
-            org.apache.bk_v4_1_0.bookkeeper.client.BookKeeper newbk =
-                    new org.apache.bk_v4_1_0.bookkeeper.client.BookKeeper(zkUtil.getZooKeeperConnectString());
-            org.apache.bk_v4_1_0.bookkeeper.client.LedgerHandle newlh =
-                    newbk.openLedger(id,
-                                org.apache.bk_v4_1_0.bookkeeper.client.BookKeeper.DigestType.CRC32,
-                                "foobar".getBytes());
-            return new Ledger410(newbk, newlh);
-        }
-
-        long getId() {
-            return lh.getId();
-        }
-
-        void write100() throws Exception {
-            for (int i = 0; i < 100; i++) {
-                lh.addEntry(ENTRYDATA);
-            }
-        }
-
-        long readAll() throws Exception {
-            long count = 0;
-            Enumeration<org.apache.bk_v4_1_0.bookkeeper.client.LedgerEntry> entries =
-                    lh.readEntries(0, lh.getLastAddConfirmed());
-            while (entries.hasMoreElements()) {
-                assertTrue("entry data doesn't match",
-                           Arrays.equals(entries.nextElement().getEntry(), ENTRYDATA));
-                count++;
-            }
-            return count;
-        }
-
-        void close() throws Exception {
-            try {
-                if (lh != null) {
-                    lh.close();
-                }
-            } finally {
-                if (bk != null) {
-                    bk.close();
-                }
-            }
-        }
-    }
-
-    /**
-     * Version 4.2.0 classes.
-     */
-    static class Server420 {
-        org.apache.bk_v4_2_0.bookkeeper.conf.ServerConfiguration conf;
-        org.apache.bk_v4_2_0.bookkeeper.proto.BookieServer server = null;
-
-        Server420(File journalDir, File ledgerDir, int port) throws Exception {
-            conf = new org.apache.bk_v4_2_0.bookkeeper.conf.ServerConfiguration();
-            conf.setBookiePort(port);
-            conf.setZkServers(zkUtil.getZooKeeperConnectString());
-            conf.setJournalDirName(journalDir.getPath());
-            conf.setLedgerDirNames(new String[] { ledgerDir.getPath() });
-            conf.setDiskUsageThreshold(0.999f);
-        }
-
-        void start() throws Exception {
-            server = new org.apache.bk_v4_2_0.bookkeeper.proto.BookieServer(conf);
-            server.start();
-            waitUp(conf.getBookiePort());
-        }
-
-        org.apache.bk_v4_2_0.bookkeeper.conf.ServerConfiguration getConf() {
-            return conf;
-        }
-
-        void stop() throws Exception {
-            if (server != null) {
-                server.shutdown();
-            }
-        }
-    }
-
-    static class Ledger420 {
-        org.apache.bk_v4_2_0.bookkeeper.client.BookKeeper bk;
-        org.apache.bk_v4_2_0.bookkeeper.client.LedgerHandle lh;
-
-        private Ledger420(org.apache.bk_v4_2_0.bookkeeper.client.BookKeeper bk,
-                          org.apache.bk_v4_2_0.bookkeeper.client.LedgerHandle lh) {
-            this.bk = bk;
-            this.lh = lh;
-        }
-
-        static Ledger420 newLedger() throws Exception {
-            org.apache.bk_v4_2_0.bookkeeper.client.BookKeeper newbk =
-                    new org.apache.bk_v4_2_0.bookkeeper.client.BookKeeper(zkUtil.getZooKeeperConnectString());
-            org.apache.bk_v4_2_0.bookkeeper.client.LedgerHandle newlh =
-                    newbk.createLedger(1, 1,
-                                  org.apache.bk_v4_2_0.bookkeeper.client.BookKeeper.DigestType.CRC32,
-                                  "foobar".getBytes());
-            return new Ledger420(newbk, newlh);
-        }
-
-        static Ledger420 openLedger(long id) throws Exception {
-            org.apache.bk_v4_2_0.bookkeeper.client.BookKeeper newbk =
-                    new org.apache.bk_v4_2_0.bookkeeper.client.BookKeeper(zkUtil.getZooKeeperConnectString());
-            org.apache.bk_v4_2_0.bookkeeper.client.LedgerHandle newlh =
-                    newbk.openLedger(id,
-                                org.apache.bk_v4_2_0.bookkeeper.client.BookKeeper.DigestType.CRC32,
-                                "foobar".getBytes());
-            return new Ledger420(newbk, newlh);
-        }
-
-        long getId() {
-            return lh.getId();
-        }
-
-        void write100() throws Exception {
-            for (int i = 0; i < 100; i++) {
-                lh.addEntry(ENTRYDATA);
-            }
-        }
-
-        long readAll() throws Exception {
-            long count = 0;
-            Enumeration<org.apache.bk_v4_2_0.bookkeeper.client.LedgerEntry> entries =
-                    lh.readEntries(0, lh.getLastAddConfirmed());
-            while (entries.hasMoreElements()) {
-                assertTrue("entry data doesn't match",
-                           Arrays.equals(entries.nextElement().getEntry(), ENTRYDATA));
-                count++;
-            }
-            return count;
-        }
-
-        void close() throws Exception {
-            try {
-                if (lh != null) {
-                    lh.close();
-                }
-            } finally {
-                if (bk != null) {
-                    bk.close();
-                }
-            }
-        }
-    }
-
-    /**
-     * Current verion classes.
-     */
-    static class ServerCurrent {
-        org.apache.bookkeeper.conf.ServerConfiguration conf;
-        org.apache.bookkeeper.proto.BookieServer server = null;
-
-        ServerCurrent(File journalDir, File ledgerDir, int port,
-                boolean useHostNameAsBookieID) throws Exception {
-            conf = TestBKConfiguration.newServerConfiguration();
-            conf.setAllowEphemeralPorts(false);
-            conf.setBookiePort(port);
-            conf.setZkServers(zkUtil.getZooKeeperConnectString());
-            conf.setJournalDirName(journalDir.getPath());
-            conf.setLedgerDirNames(new String[] { ledgerDir.getPath() });
-            conf.setUseHostNameAsBookieID(useHostNameAsBookieID);
-        }
-
-        void start() throws Exception {
-            server = new org.apache.bookkeeper.proto.BookieServer(conf);
-            server.start();
-            waitUp(conf.getBookiePort());
-        }
-
-        org.apache.bookkeeper.conf.ServerConfiguration getConf() {
-            return conf;
-        }
-
-        void stop() throws Exception {
-            if (server != null) {
-                server.shutdown();
-            }
-        }
-    }
-
-    static class LedgerCurrent {
-        org.apache.bookkeeper.client.BookKeeper bk;
-        org.apache.bookkeeper.client.LedgerHandle lh;
-
-        private LedgerCurrent(org.apache.bookkeeper.client.BookKeeper bk,
-                              org.apache.bookkeeper.client.LedgerHandle lh) {
-            this.bk = bk;
-            this.lh = lh;
-        }
-
-        static LedgerCurrent newLedger() throws Exception {
-            org.apache.bookkeeper.client.BookKeeper newbk =
-                    new org.apache.bookkeeper.client.BookKeeper(zkUtil.getZooKeeperConnectString());
-            org.apache.bookkeeper.client.LedgerHandle newlh =
-                    newbk.createLedger(1, 1,
-                                     org.apache.bookkeeper.client.BookKeeper.DigestType.CRC32,
-                                     "foobar".getBytes());
-            return new LedgerCurrent(newbk, newlh);
-        }
-
-        static LedgerCurrent openLedger(long id) throws Exception {
-            org.apache.bookkeeper.client.BookKeeper newbk =
-                    new org.apache.bookkeeper.client.BookKeeper(zkUtil.getZooKeeperConnectString());
-            org.apache.bookkeeper.client.LedgerHandle newlh =
-                    newbk.openLedger(id,
-                                org.apache.bookkeeper.client.BookKeeper.DigestType.CRC32,
-                                "foobar".getBytes());
-            return new LedgerCurrent(newbk, newlh);
-        }
-
-        static LedgerCurrent openLedger(long id, ClientConfiguration conf) throws Exception {
-            conf.setZkServers(zkUtil.getZooKeeperConnectString());
-            org.apache.bookkeeper.client.BookKeeper newbk =
-                    new org.apache.bookkeeper.client.BookKeeper(conf);
-            org.apache.bookkeeper.client.LedgerHandle newlh =
-                    newbk.openLedger(id,
-                                   org.apache.bookkeeper.client.BookKeeper.DigestType.CRC32,
-                                "foobar".getBytes());
-            return new LedgerCurrent(newbk, newlh);
-        }
-
-        long getId() {
-            return lh.getId();
-        }
-
-        void write100() throws Exception {
-            for (int i = 0; i < 100; i++) {
-                lh.addEntry(ENTRYDATA);
-            }
-        }
-
-        long readAll() throws Exception {
-            long count = 0;
-            Enumeration<org.apache.bookkeeper.client.LedgerEntry> entries =
-                    lh.readEntries(0, lh.getLastAddConfirmed());
-            while (entries.hasMoreElements()) {
-                assertTrue("entry data doesn't match",
-                           Arrays.equals(entries.nextElement().getEntry(), ENTRYDATA));
-                count++;
-            }
-            return count;
-        }
-
-        void close() throws Exception {
-            try {
-                if (lh != null) {
-                    lh.close();
-                }
-            } finally {
-                if (bk != null) {
-                    bk.close();
-                }
-            }
-        }
-    }
-
-    /*
-     * Test old cookie accessing the new version formatted cluster.
-     */
-    @Test
-    public void testOldCookieAccessingNewCluster() throws Exception {
-        File journalDir = createTempDir("bookie", "journal");
-        File ledgerDir = createTempDir("bookie", "ledger");
-
-        int port = PortManager.nextFreePort();
-
-        // start old server
-        Server410 s410 = new Server410(journalDir, ledgerDir, port);
-        s410.start();
-
-        Ledger410 l410 = Ledger410.newLedger();
-        l410.write100();
-        l410.getId();
-        l410.close();
-        s410.stop();
-
-        // Format the metadata using current version
-        ServerCurrent currentServer = new ServerCurrent(journalDir, ledgerDir,
-                port, false);
-        BookKeeperAdmin.format(currentServer.conf,
-                false, true);
-        // start the current version server with old version cookie
-        try {
-            currentServer.start();
-            fail("Bookie should not start with old cookie");
-        } catch (BookieException e) {
-            assertTrue("Old Cookie should not be able to access", e
-                    .getMessage().contains("instanceId"));
-        } finally {
-            currentServer.stop();
-        }
-
-        // Format the bookie also and restart
-        assertTrue("Format should be success",
-                Bookie.format(currentServer.conf, false, true));
-        try {
-            currentServer = null;
-            currentServer = new ServerCurrent(journalDir, ledgerDir, port, false);
-            currentServer.start();
-        } finally {
-            if (null != currentServer) {
-                currentServer.stop();
-            }
-        }
-    }
-
-}
diff --git a/tests/pom.xml b/tests/pom.xml
index f3df8d4..9139f0b 100644
--- a/tests/pom.xml
+++ b/tests/pom.xml
@@ -45,6 +45,7 @@
     <module>backward-compat-hierarchical-ledger-manager</module>
     <module>backward-compat-hostname-bookieid</module>
     <module>backward-compat-recovery-no-password</module>
+    <module>backward-compat-old-cookie-new-cluster</module>
   </modules>
   <build>
     <plugins>

-- 
To stop receiving notification emails like this one, please contact
ivank@apache.org.