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 2018/01/24 18:56:54 UTC

[GitHub] sijie closed pull request #1033: Test compat with useHostNameAsBookieID

sijie closed pull request #1033: Test compat with useHostNameAsBookieID
URL: https://github.com/apache/bookkeeper/pull/1033
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

diff --git a/tests/backward-compat-hostname-bookieid/pom.xml b/tests/backward-compat-hostname-bookieid/pom.xml
new file mode 100644
index 000000000..3b09d7a6c
--- /dev/null
+++ b/tests/backward-compat-hostname-bookieid/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-hostname-bookieid</artifactId>
+  <packaging>jar</packaging>
+  <name>Apache BookKeeper :: Tests :: Test upgrade between 4.1.0 and current version (with hostname bookie ID)</name>
+
+</project>
diff --git a/tests/backward-compat-hostname-bookieid/src/test/groovy/org/apache/bookkeeper/tests/TestCompatUpgradeWithHostnameBookieId.groovy b/tests/backward-compat-hostname-bookieid/src/test/groovy/org/apache/bookkeeper/tests/TestCompatUpgradeWithHostnameBookieId.groovy
new file mode 100644
index 000000000..67100def3
--- /dev/null
+++ b/tests/backward-compat-hostname-bookieid/src/test/groovy/org/apache/bookkeeper/tests/TestCompatUpgradeWithHostnameBookieId.groovy
@@ -0,0 +1,131 @@
+/*
+* 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 TestCompatUpgradeWithHostnameBookieId {
+    private static final Logger LOG = LoggerFactory.getLogger(TestCompatUpgradeWithHostnameBookieId.class)
+    private static byte[] PASSWD = "foobar".getBytes()
+
+    private def oldClientVersions = ["4.1.0", "4.2.0", "4.2.1", "4.2.2", "4.2.3", "4.2.4",
+                                     "4.3.0", "4.3.1", "4.3.2", "4.4.0", "4.5.0", "4.5.1",
+                                     "4.6.0"]
+
+    @ArquillianResource
+    DockerClient docker
+
+
+    private void writeEntries(def ledger, int numEntries) throws Exception {
+        for (int i = 0; i < numEntries; i++) {
+            ledger.addEntry(("foobar" + i).getBytes())
+        }
+    }
+
+    private void assertHasEntries(def ledger, int numEntries) throws Exception {
+        Assert.assertEquals(numEntries, ledger.getLastAddConfirmed() + 1 /* counts from 0 */)
+        def entries = ledger.readEntries(0, ledger.getLastAddConfirmed())
+        int j = 0
+        while (entries.hasMoreElements()) {
+            def e = entries.nextElement()
+            Assert.assertEquals(new String(e.getEntry()), "foobar"+ j)
+            j++
+        }
+    }
+
+    /**
+     * Test compatability between version old version and the current version.
+     * - 4.1.0 server restarts with useHostNameAsBookieID=true.
+     * - Write ledgers with old and new clients
+     * - Read ledgers written by old clients.
+     */
+    @Test
+    public void testOldClientsWorkWithNewServerUsingHostnameAsBookieID() throws Exception {
+        int numEntries = 10
+        BookKeeperClusterUtils.legacyMetadataFormat(docker)
+        String zookeeper = BookKeeperClusterUtils.zookeeperConnectString(docker)
+
+        String currentVersion = System.getProperty("currentVersion")
+
+        Assert.assertTrue(BookKeeperClusterUtils.startAllBookiesWithVersion(docker, "4.1.0"))
+
+        // Write a ledger with v4.1.0 client
+        def v410CL = MavenClassLoader.forBookKeeperVersion("4.1.0")
+        def v410BK = v410CL.newBookKeeper(zookeeper)
+
+        def ledger410 = v410BK.createLedger(3, 2, v410CL.digestType("CRC32"), PASSWD)
+        writeEntries(ledger410, numEntries)
+        ledger410.close()
+
+        // Write a ledger with v4.2.0 client
+        def v420CL = MavenClassLoader.forBookKeeperVersion("4.2.0")
+        def v420BK = v420CL.newBookKeeper(zookeeper)
+
+        def ledger420 = v420BK.createLedger(3, 2, v420CL.digestType("CRC32"), PASSWD)
+        writeEntries(ledger420, numEntries)
+        ledger420.close()
+
+        // Stop bookies, change config to use hostname as id, restart with latest version
+        Assert.assertTrue(BookKeeperClusterUtils.stopAllBookies(docker))
+        BookKeeperClusterUtils.updateAllBookieConf(docker, currentVersion, "useHostNameAsBookieID", "true")
+        Assert.assertTrue(BookKeeperClusterUtils.startAllBookiesWithVersion(docker, currentVersion))
+
+        // Ensure we can read ledger with v4.1.0 client
+        def ledger410r = v410BK.openLedger(ledger410.getId(), v410CL.digestType("CRC32"), PASSWD)
+        assertHasEntries(ledger410r, numEntries)
+        ledger410r.close()
+
+        // Ensure we can read ledger with v4.2.0 client
+        def ledger420r = v420BK.openLedger(ledger420.getId(), v420CL.digestType("CRC32"), PASSWD)
+        assertHasEntries(ledger420r, numEntries)
+        ledger420r.close()
+
+        // Ensure we can write and read new ledgers with all client versions
+        def currentCL = MavenClassLoader.forBookKeeperVersion(currentVersion)
+        def currentBK = currentCL.newBookKeeper(zookeeper)
+        oldClientVersions.each{
+            LOG.info("Testing ledger creation for version {}", it)
+            def oldCL = MavenClassLoader.forBookKeeperVersion(it)
+            def oldBK = oldCL.newBookKeeper(zookeeper)
+
+            def ledger0 = oldBK.createLedger(3, 2, oldCL.digestType("CRC32"), PASSWD)
+            writeEntries(ledger0, numEntries)
+            ledger0.close()
+
+            def ledger1 = currentBK.openLedger(ledger0.getId(), currentCL.digestType("CRC32"), PASSWD)
+            assertHasEntries(ledger1, numEntries)
+            ledger1.close()
+
+            oldBK.close()
+        }
+        currentBK.close()
+    }
+
+
+}
diff --git a/tests/backward-compat-hostname-bookieid/src/test/resources/arquillian.xml b/tests/backward-compat-hostname-bookieid/src/test/resources/arquillian.xml
new file mode 100644
index 000000000..f914ff2c2
--- /dev/null
+++ b/tests/backward-compat-hostname-bookieid/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-compat-upgrade-direct/src/test/groovy/org/apache/bookkeeper/tests/TestCompatUpgradeDirect.groovy b/tests/backward-compat-upgrade-direct/src/test/groovy/org/apache/bookkeeper/tests/TestCompatUpgradeDirect.groovy
index cc32991de..2f7fb67e4 100644
--- a/tests/backward-compat-upgrade-direct/src/test/groovy/org/apache/bookkeeper/tests/TestCompatUpgradeDirect.groovy
+++ b/tests/backward-compat-upgrade-direct/src/test/groovy/org/apache/bookkeeper/tests/TestCompatUpgradeDirect.groovy
@@ -32,7 +32,6 @@ import org.slf4j.Logger
 import org.slf4j.LoggerFactory
 
 @RunWith(Arquillian.class)
-@FixMethodOrder(MethodSorters.NAME_ASCENDING)
 class TestCompatUpgradeDirect {
     private static final Logger LOG = LoggerFactory.getLogger(TestCompatUpgradeDirect.class)
     private static byte[] PASSWD = "foobar".getBytes()
@@ -91,7 +90,7 @@ class TestCompatUpgradeDirect {
     }
 
     @Test
-    public void test1_v410ClientCantFenceLedgerFromCurrent() throws Exception {
+    public void test9_v410ClientCantFenceLedgerFromCurrent() throws Exception {
         String currentVersion = System.getProperty("currentVersion")
         String zookeeper = BookKeeperClusterUtils.zookeeperConnectString(docker)
 
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
index 7a42fc2d0..7d86eb746 100644
--- 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
@@ -20,7 +20,6 @@
  */
 package org.apache.bookkeeper.tests.backward;
 
-import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
@@ -438,105 +437,4 @@ public void testOldCookieAccessingNewCluster() throws Exception {
         }
     }
 
-    /**
-     * Test compatability between old versions and the current version.
-     * - old server restarts with useHostNameAsBookieID=true.
-     * - Read ledgers with old and new clients
-     */
-    @Test
-    public void testCompatReads() throws Exception {
-        File journalDir = createTempDir("bookie", "journal");
-        File ledgerDir = createTempDir("bookie", "ledger");
-
-        int port = PortManager.nextFreePort();
-        // start server, upgrade
-        Server410 s410 = new Server410(journalDir, ledgerDir, port);
-        s410.start();
-
-        Ledger410 l410 = Ledger410.newLedger();
-        l410.write100();
-        long oldLedgerId = l410.getId();
-        l410.close();
-
-        // Check that 420 client can to write to 410 server
-        Ledger420 l420 = Ledger420.newLedger();
-        l420.write100();
-        long lid420 = l420.getId();
-        l420.close();
-
-        s410.stop();
-
-        // Start the current server, will not require a filesystem upgrade
-        ServerCurrent scur = new ServerCurrent(journalDir, ledgerDir, port,
-                false);
-        scur.start();
-
-        // check that old client can read its old ledgers on new server
-        l410 = Ledger410.openLedger(oldLedgerId);
-        assertEquals(100, l410.readAll());
-        l410.close();
-
-        // Check that 420 client can read old ledgers on new server
-        l420 = Ledger420.openLedger(lid420);
-        assertEquals("Failed to read entries!", 100, l420.readAll());
-        l420.close();
-
-        // Check that current client can read old ledgers on new server
-        final LedgerCurrent curledger = LedgerCurrent.openLedger(lid420);
-        assertEquals("Failed to read entries!", 100, curledger.readAll());
-        curledger.close();
-    }
-
-    /**
-     * Test compatability between version old version and the current version.
-     * - 4.1.0 server restarts with useHostNameAsBookieID=true.
-     * - Write ledgers with old and new clients
-     * - Read ledgers written by old clients.
-     */
-    @Test
-    public void testCompatWrites() throws Exception {
-        File journalDir = createTempDir("bookie", "journal");
-        File ledgerDir = createTempDir("bookie", "ledger");
-
-        int port = PortManager.nextFreePort();
-        // start server, upgrade
-        Server410 s410 = new Server410(journalDir, ledgerDir, port);
-        s410.start();
-        s410.stop();
-
-        // Start the current server, will not require a filesystem upgrade
-        ServerCurrent scur = new ServerCurrent(journalDir, ledgerDir, port,
-                false);
-        scur.start();
-
-        // Check that current client can to write to server
-        LedgerCurrent lcur = LedgerCurrent.newLedger();
-        lcur.write100();
-        lcur.close();
-        final LedgerCurrent curledger = LedgerCurrent.openLedger(lcur.getId());
-        assertEquals("Failed to read entries!", 100, curledger.readAll());
-
-        // Check that 410 client can write to server
-        Ledger410 l410 = Ledger410.newLedger();
-        l410.write100();
-        long oldLedgerId = l410.getId();
-        l410.close();
-
-        // Check that 420 client can write to server
-        Ledger410 l420 = Ledger410.newLedger();
-        l420.write100();
-        long lid420 = l420.getId();
-        l420.close();
-
-        // check that new client can read old ledgers on new server
-        LedgerCurrent oldledger = LedgerCurrent.openLedger(oldLedgerId);
-        assertEquals("Failed to read entries!", 100, oldledger.readAll());
-        oldledger.close();
-
-        // check that new client can read old ledgers on new server
-        oldledger = LedgerCurrent.openLedger(lid420);
-        assertEquals("Failed to read entries!", 100, oldledger.readAll());
-        oldledger.close();
-        scur.stop();
-    }
 }
diff --git a/tests/pom.xml b/tests/pom.xml
index 8382eb25a..f3df8d425 100644
--- a/tests/pom.xml
+++ b/tests/pom.xml
@@ -43,6 +43,7 @@
     <module>backward-compat-upgrade</module>
     <module>backward-compat-upgrade-direct</module>
     <module>backward-compat-hierarchical-ledger-manager</module>
+    <module>backward-compat-hostname-bookieid</module>
     <module>backward-compat-recovery-no-password</module>
   </modules>
   <build>


 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services