You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@bookkeeper.apache.org by eo...@apache.org on 2017/09/07 12:16:41 UTC

[bookkeeper] branch master updated: ISSUE #494: Rename ZkVersion to LongVersion

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

eolivelli 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 d338add  ISSUE #494: Rename ZkVersion to LongVersion
d338add is described below

commit d338addc369cbe3b96af4ee4232d0cce7d72d089
Author: Jia Zhai <zh...@apache.org>
AuthorDate: Thu Sep 7 14:16:27 2017 +0200

    ISSUE #494: Rename ZkVersion to LongVersion
    
    Descriptions of the changes in this PR:
    
    - rename `ZkVersion` to `LongVersion`
    - move it from `meta` package to `versioning` package
    
    Author: Jia Zhai <zh...@apache.org>
    
    Reviewers: Enrico Olivelli <eo...@apache.org>
    
    This closes #495 from zhaijack/metadata/rename_zkversion_to_longversion, closes #494
---
 .../java/org/apache/bookkeeper/bookie/Cookie.java  | 12 ++--
 .../bookkeeper/meta/AbstractZkLedgerManager.java   | 23 ++++----
 .../ZkVersion.java => versioning/LongVersion.java} | 47 ++++++++--------
 .../org/apache/bookkeeper/bookie/CookieTest.java   | 29 +++++-----
 .../org/apache/bookkeeper/meta/TestZkVersion.java  | 65 ----------------------
 .../bookkeeper/versioning/TestLongVersion.java     | 55 ++++++++++++++++++
 .../resources/bookkeeper/server-suppressions.xml   |  1 -
 7 files changed, 108 insertions(+), 124 deletions(-)

diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Cookie.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Cookie.java
index f057391..be475be 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Cookie.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Cookie.java
@@ -41,11 +41,11 @@ import java.util.List;
 import java.util.Set;
 import org.apache.bookkeeper.conf.AbstractConfiguration;
 import org.apache.bookkeeper.conf.ServerConfiguration;
-import org.apache.bookkeeper.meta.ZkVersion;
 import org.apache.bookkeeper.net.BookieSocketAddress;
 import org.apache.bookkeeper.proto.DataFormats.CookieFormat;
 import org.apache.bookkeeper.util.BookKeeperConstants;
 import org.apache.bookkeeper.util.ZkUtils;
+import org.apache.bookkeeper.versioning.LongVersion;
 import org.apache.bookkeeper.versioning.Version;
 import org.apache.bookkeeper.versioning.Versioned;
 import org.apache.zookeeper.CreateMode;
@@ -262,10 +262,10 @@ class Cookie {
             zk.create(zkPath, data,
                     zkAcls, CreateMode.PERSISTENT);
         } else {
-            if (!(version instanceof ZkVersion)) {
+            if (!(version instanceof LongVersion)) {
                 throw new IllegalArgumentException("Invalid version type, expected ZkVersion type");
             }
-            zk.setData(zkPath, data, ((ZkVersion) version).getZnodeVersion());
+            zk.setData(zkPath, data, (int) ((LongVersion) version).getLongVersion());
         }
     }
 
@@ -299,12 +299,12 @@ class Cookie {
     public void deleteFromZooKeeper(ZooKeeper zk, AbstractConfiguration conf,
                                     BookieSocketAddress address, Version version)
             throws KeeperException, InterruptedException, UnknownHostException {
-        if (!(version instanceof ZkVersion)) {
+        if (!(version instanceof LongVersion)) {
             throw new IllegalArgumentException("Invalid version type, expected ZkVersion type");
         }
 
         String zkPath = getZkPath(conf, address);
-        zk.delete(zkPath, ((ZkVersion) version).getZnodeVersion());
+        zk.delete(zkPath, (int) ((LongVersion) version).getLongVersion());
         LOG.info("Removed cookie from {} for bookie {}.", conf.getZkLedgersRootPath(), address);
     }
 
@@ -364,7 +364,7 @@ class Cookie {
             Builder builder = parse(reader);
             Cookie cookie = builder.build();
             // sets stat version from ZooKeeper
-            ZkVersion version = new ZkVersion(stat.getVersion());
+            LongVersion version = new LongVersion(stat.getVersion());
             return new Versioned<Cookie>(cookie, version);
         } finally {
             reader.close();
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/AbstractZkLedgerManager.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/AbstractZkLedgerManager.java
index 3965eb6..c93a8a9 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/AbstractZkLedgerManager.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/AbstractZkLedgerManager.java
@@ -18,6 +18,7 @@
 package org.apache.bookkeeper.meta;
 
 import com.google.common.base.Optional;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
 import java.io.IOException;
 import java.util.HashSet;
 import java.util.List;
@@ -29,7 +30,6 @@ import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.Executors;
 import java.util.concurrent.ScheduledExecutorService;
 import java.util.concurrent.TimeUnit;
-
 import org.apache.bookkeeper.client.BKException;
 import org.apache.bookkeeper.client.LedgerMetadata;
 import org.apache.bookkeeper.conf.AbstractConfiguration;
@@ -39,6 +39,7 @@ import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.MultiCallback;
 import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.Processor;
 import org.apache.bookkeeper.util.BookKeeperConstants;
 import org.apache.bookkeeper.util.ZkUtils;
+import org.apache.bookkeeper.versioning.LongVersion;
 import org.apache.bookkeeper.versioning.Version;
 import org.apache.zookeeper.AsyncCallback;
 import org.apache.zookeeper.AsyncCallback.DataCallback;
@@ -52,12 +53,10 @@ import org.apache.zookeeper.WatchedEvent;
 import org.apache.zookeeper.Watcher;
 import org.apache.zookeeper.ZooKeeper;
 import org.apache.zookeeper.data.Stat;
+import org.apache.zookeeper.data.ACL;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import com.google.common.util.concurrent.ThreadFactoryBuilder;
-import org.apache.zookeeper.data.ACL;
-
 /**
  * Abstract ledger manager based on zookeeper, which provides common methods such as query zk nodes.
  */
@@ -231,7 +230,7 @@ abstract class AbstractZkLedgerManager implements LedgerManager, Watcher {
             public void processResult(int rc, String path, Object ctx, String name) {
                 if (rc == Code.OK.intValue()) {
                     // update version
-                    metadata.setVersion(new ZkVersion(0));
+                    metadata.setVersion(new LongVersion(0));
                     ledgerCb.operationComplete(BKException.Code.OK, null);
                 } else if (rc == Code.NODEEXISTS.intValue()) {
                     LOG.warn("Failed to create ledger metadata for {} which already exist", ledgerId);
@@ -268,12 +267,12 @@ abstract class AbstractZkLedgerManager implements LedgerManager, Watcher {
             cb.operationComplete(BKException.Code.MetadataVersionException, (Void)null);
             return;
         } else if (Version.ANY != version) {
-            if (!(version instanceof ZkVersion)) {
+            if (!(version instanceof LongVersion)) {
                 LOG.info("Not an instance of ZKVersion: {}", ledgerId);
                 cb.operationComplete(BKException.Code.MetadataVersionException, (Void)null);
                 return;
             } else {
-                znodeVersion = ((ZkVersion)version).getZnodeVersion();
+                znodeVersion = (int) ((LongVersion) version).getLongVersion();
             }
         }
         
@@ -387,7 +386,7 @@ abstract class AbstractZkLedgerManager implements LedgerManager, Watcher {
                 }
                 LedgerMetadata metadata;
                 try {
-                    metadata = LedgerMetadata.parseConfig(data, new ZkVersion(stat.getVersion()), Optional.of(stat.getCtime()));
+                    metadata = LedgerMetadata.parseConfig(data, new LongVersion(stat.getVersion()), Optional.of(stat.getCtime()));
                 } catch (IOException e) {
                     LOG.error("Could not parse ledger metadata for ledger: " + ledgerId, e);
                     readCb.operationComplete(BKException.Code.ZKException, null);
@@ -402,13 +401,13 @@ abstract class AbstractZkLedgerManager implements LedgerManager, Watcher {
     public void writeLedgerMetadata(final long ledgerId, final LedgerMetadata metadata,
                                     final GenericCallback<Void> cb) {
         Version v = metadata.getVersion();
-        if (Version.NEW == v || !(v instanceof ZkVersion)) {
+        if (Version.NEW == v || !(v instanceof LongVersion)) {
             cb.operationComplete(BKException.Code.MetadataVersionException, null);
             return;
         }
-        final ZkVersion zv = (ZkVersion) v;
+        final LongVersion zv = (LongVersion) v;
         zk.setData(getLedgerPath(ledgerId),
-                   metadata.serialize(), zv.getZnodeVersion(),
+                   metadata.serialize(), (int) zv.getLongVersion(),
                    new StatCallback() {
             @Override
             public void processResult(int rc, String path, Object ctx, Stat stat) {
@@ -416,7 +415,7 @@ abstract class AbstractZkLedgerManager implements LedgerManager, Watcher {
                     cb.operationComplete(BKException.Code.MetadataVersionException, null);
                 } else if (KeeperException.Code.OK.intValue() == rc) {
                     // update metadata version
-                    metadata.setVersion(zv.setZnodeVersion(stat.getVersion()));
+                    metadata.setVersion(zv.setLongVersion(stat.getVersion()));
                     cb.operationComplete(BKException.Code.OK, null);
                 } else {
                     LOG.warn("Conditional update ledger metadata failed: {}", KeeperException.Code.get(rc));
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/ZkVersion.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/versioning/LongVersion.java
similarity index 53%
rename from bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/ZkVersion.java
rename to bookkeeper-server/src/main/java/org/apache/bookkeeper/versioning/LongVersion.java
index 7fdb8bd..11f9b19 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/ZkVersion.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/versioning/LongVersion.java
@@ -1,5 +1,4 @@
 /*
- *
  * 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
@@ -8,26 +7,24 @@
  * "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.
+ *     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.meta;
+package org.apache.bookkeeper.versioning;
 
-import org.apache.bookkeeper.versioning.Version;
-import org.apache.bookkeeper.versioning.Version.Occurred;
-
-public class ZkVersion implements Version {
-    int znodeVersion;
+/**
+ * A version object holds integer version.
+ */
+public class LongVersion implements Version {
+    protected long version;
 
-    public ZkVersion(int version) {
-        znodeVersion = version;
+    public LongVersion(long v) {
+        this.version = v;
     }
 
     @Override
@@ -39,11 +36,11 @@ public class ZkVersion implements Version {
             return Occurred.AFTER;
         } else if (v == Version.ANY) {
             return Occurred.CONCURRENTLY;
-        } else if (!(v instanceof ZkVersion)) {
+        } else if (!(v instanceof LongVersion)) {
             throw new IllegalArgumentException("Invalid version type");
         }
-        ZkVersion zv = (ZkVersion)v;
-        int res = znodeVersion - zv.znodeVersion;
+        LongVersion zv = (LongVersion)v;
+        int res = Long.compare(version, zv.version);
         if (res == 0) {
             return Occurred.CONCURRENTLY;
         } else if (res < 0) {
@@ -53,17 +50,17 @@ public class ZkVersion implements Version {
         }
     }
 
-    public int getZnodeVersion() {
-        return znodeVersion;
+    public long getLongVersion() {
+        return version;
     }
 
-    public ZkVersion setZnodeVersion(int znodeVersion) {
-        this.znodeVersion = znodeVersion;
+    public LongVersion setLongVersion(long v) {
+        this.version = v;
         return this;
     }
 
     @Override
     public String toString() {
-        return Integer.toString(znodeVersion, 10);
+        return Long.toString(version, 10);
     }
 }
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/CookieTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/CookieTest.java
index 014c194..f622448 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/CookieTest.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/CookieTest.java
@@ -21,10 +21,6 @@
 
 package org.apache.bookkeeper.bookie;
 
-import org.apache.bookkeeper.test.BookKeeperClusterTestCase;
-import org.apache.bookkeeper.test.PortManager;
-import org.apache.bookkeeper.util.IOUtils;
-
 import static org.apache.bookkeeper.bookie.UpgradeTest.newV1JournalDirectory;
 import static org.apache.bookkeeper.bookie.UpgradeTest.newV1LedgerDirectory;
 import static org.apache.bookkeeper.bookie.UpgradeTest.newV2JournalDirectory;
@@ -32,26 +28,26 @@ import static org.apache.bookkeeper.bookie.UpgradeTest.newV2LedgerDirectory;
 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.util.HashSet;
 import java.util.List;
 import java.util.Set;
-
-import org.junit.Assert;
 import org.apache.bookkeeper.client.BookKeeperAdmin;
 import org.apache.bookkeeper.conf.ClientConfiguration;
 import org.apache.bookkeeper.conf.ServerConfiguration;
 import org.apache.bookkeeper.conf.TestBKConfiguration;
-import org.apache.bookkeeper.meta.ZkVersion;
+import org.apache.bookkeeper.test.BookKeeperClusterTestCase;
+import org.apache.bookkeeper.test.PortManager;
+import org.apache.bookkeeper.util.IOUtils;
+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;
 
-
-import com.google.common.collect.Sets;
-
 public class CookieTest extends BookKeeperClusterTestCase {
     final int bookiePort = PortManager.nextFreePort();
 
@@ -596,16 +592,19 @@ public class CookieTest extends BookKeeperClusterTestCase {
         b.shutdown();
         Versioned<Cookie> zkCookie = Cookie.readFromZooKeeper(zkc, conf);
         Version version1 = zkCookie.getVersion();
-        Assert.assertTrue("Invalid type expected ZkVersion type", version1 instanceof ZkVersion);
-        ZkVersion zkVersion1 = (ZkVersion) version1;
+        Assert.assertTrue("Invalid type expected ZkVersion type",
+            version1 instanceof LongVersion);
+        LongVersion zkVersion1 = (LongVersion) version1;
         Cookie cookie = zkCookie.getValue();
         cookie.writeToZooKeeper(zkc, conf, version1);
 
         zkCookie = Cookie.readFromZooKeeper(zkc, conf);
         Version version2 = zkCookie.getVersion();
-        Assert.assertTrue("Invalid type expected ZkVersion type", version2 instanceof ZkVersion);
-        ZkVersion zkVersion2 = (ZkVersion) version2;
-        Assert.assertEquals("Version mismatches!", zkVersion1.getZnodeVersion() + 1, zkVersion2.getZnodeVersion());
+        Assert.assertTrue("Invalid type expected ZkVersion type",
+            version2 instanceof LongVersion);
+        LongVersion zkVersion2 = (LongVersion) version2;
+        Assert.assertEquals("Version mismatches!",
+            zkVersion1.getLongVersion() + 1, zkVersion2.getLongVersion());
     }
 
     /**
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/TestZkVersion.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/TestZkVersion.java
deleted file mode 100644
index 5409246..0000000
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/TestZkVersion.java
+++ /dev/null
@@ -1,65 +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.meta;
-
-import org.junit.Test;
-import org.junit.Assert;
-
-import org.apache.bookkeeper.versioning.Version;
-import org.apache.bookkeeper.versioning.Version.Occurred;
-
-public class TestZkVersion {
-
-    @Test
-    public void testNullZkVersion() {
-        ZkVersion zkVersion = new ZkVersion(99);
-        try {
-            zkVersion.compare(null);
-            Assert.fail("Should fail comparing with null version.");
-        } catch (NullPointerException npe) {
-        }
-    }
-
-    @Test
-    public void testInvalidVersion() {
-        ZkVersion zkVersion = new ZkVersion(99);
-        try {
-            zkVersion.compare(new Version() {
-                @Override
-                public Occurred compare(Version v) {
-                    return Occurred.AFTER;
-                }
-            });
-            Assert.fail("Should not reach here!");
-        } catch (IllegalArgumentException iae) {
-        }
-    }
-
-    @Test
-    public void testCompare() {
-        ZkVersion zv = new ZkVersion(99);
-        Assert.assertEquals(Occurred.AFTER, zv.compare(new ZkVersion(98)));
-        Assert.assertEquals(Occurred.BEFORE, zv.compare(new ZkVersion(100)));
-        Assert.assertEquals(Occurred.CONCURRENTLY, zv.compare(new ZkVersion(99)));
-        Assert.assertEquals(Occurred.CONCURRENTLY, zv.compare(Version.ANY));
-        Assert.assertEquals(Occurred.AFTER, zv.compare(Version.NEW));
-    }
-}
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/versioning/TestLongVersion.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/versioning/TestLongVersion.java
new file mode 100644
index 0000000..51f98e6
--- /dev/null
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/versioning/TestLongVersion.java
@@ -0,0 +1,55 @@
+/*
+ * 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.versioning;
+
+import org.apache.bookkeeper.versioning.Version.Occurred;
+import org.junit.Test;
+import org.junit.Assert;
+
+public class TestLongVersion {
+
+    @Test(timeout=60000)
+    public void testNullIntVersion() {
+        LongVersion longVersion = new LongVersion(99);
+        try {
+            longVersion.compare(null);
+            Assert.fail("Should fail comparing with null version.");
+        } catch (NullPointerException npe) {
+        }
+    }
+
+    @Test(timeout=60000)
+    public void testInvalidVersion() {
+        LongVersion longVersion = new LongVersion(99);
+        try {
+            longVersion.compare(v -> Occurred.AFTER);
+            Assert.fail("Should not reach here!");
+        } catch (IllegalArgumentException iae) {
+        }
+    }
+
+    @Test(timeout=60000)
+    public void testCompare() {
+        LongVersion iv = new LongVersion(99);
+        Assert.assertEquals(Occurred.AFTER, iv.compare(new LongVersion(98)));
+        Assert.assertEquals(Occurred.BEFORE, iv.compare(new LongVersion(100)));
+        Assert.assertEquals(Occurred.CONCURRENTLY, iv.compare(new LongVersion(99)));
+        Assert.assertEquals(Occurred.CONCURRENTLY, iv.compare(Version.ANY));
+        Assert.assertEquals(Occurred.AFTER, iv.compare(Version.NEW));
+    }
+}
diff --git a/buildtools/src/main/resources/bookkeeper/server-suppressions.xml b/buildtools/src/main/resources/bookkeeper/server-suppressions.xml
index dfbb686..ac3e999 100644
--- a/buildtools/src/main/resources/bookkeeper/server-suppressions.xml
+++ b/buildtools/src/main/resources/bookkeeper/server-suppressions.xml
@@ -36,7 +36,6 @@
     <suppress checks=".*" files=".*[\\/]streaming[\\/].*"/>
     <suppress checks=".*" files=".*[\\/]tools[\\/].*"/>
     <suppress checks=".*" files=".*[\\/]util[\\/].*"/>
-    <suppress checks=".*" files=".*[\\/]versioning[\\/].*"/>
     <suppress checks=".*" files=".*[\\/]zookeeper[\\/].*"/>
 
     <!-- suppress all checks in the generated directories -->

-- 
To stop receiving notification emails like this one, please contact
['"commits@bookkeeper.apache.org" <co...@bookkeeper.apache.org>'].