You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@zookeeper.apache.org by lvfangmin <gi...@git.apache.org> on 2018/09/19 03:18:38 UTC

[GitHub] zookeeper pull request #632: [ZOOKEEPER-3150] Add tree digest check and veri...

GitHub user lvfangmin opened a pull request:

    https://github.com/apache/zookeeper/pull/632

    [ZOOKEEPER-3150] Add tree digest check and verify data integrity when loading from disk

    Jira ZOOKEEPER-3114 will be divided into two parts:
    
    1. data integrity check when loading snapshot/txns from disk
    2. real time data consistency check when syncing and following leader
    
    This is the first part, which is going to check the data integrity by calculating the hash value of data tree, and compare the value when reload the snapshot/txns from disk.

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/lvfangmin/zookeeper ZOOKEEPER-3150

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/zookeeper/pull/632.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #632
    
----
commit a132ad06d987e09c2e9ebb3c44d2a04e8d2528ad
Author: Fangmin Lyu <al...@...>
Date:   2018-09-19T03:06:00Z

    Add tree digest check and verify consistency when loading from disk

----


---

[GitHub] zookeeper issue #632: [ZOOKEEPER-3150] Add tree digest check and verify data...

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit commented on the issue:

    https://github.com/apache/zookeeper/pull/632
  
    
    Refer to this link for build results (access rights to CI server needed): 
    https://builds.apache.org/job/PreCommit-ZOOKEEPER-github-pr-build/2388/



---

[GitHub] zookeeper pull request #632: [ZOOKEEPER-3150] Add tree digest check and veri...

Posted by eribeiro <gi...@git.apache.org>.
Github user eribeiro commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/632#discussion_r224210799
  
    --- Diff: zookeeper-server/src/main/java/org/apache/zookeeper/server/NodeHashMapImpl.java ---
    @@ -0,0 +1,116 @@
    +/**
    + * 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.zookeeper.server;
    +
    +import java.util.Map;
    +import java.util.Set;
    +import java.util.concurrent.ConcurrentHashMap;
    +
    +import org.apache.zookeeper.server.util.DigestCalculator;
    +import org.apache.zookeeper.server.util.AdHash;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +/**
    + * a simple wrapper to ConcurrentHashMap that recalculates a digest after
    + * each mutation.
    + */
    +public class NodeHashMapImpl implements NodeHashMap {
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(NodeHashMap.class);
    +
    +    private final ConcurrentHashMap<String, DataNode> nodes;
    +
    +    private AdHash hash;
    +
    +    public NodeHashMapImpl() {
    +        nodes = new ConcurrentHashMap<String, DataNode>();
    +        hash = new AdHash();
    --- End diff --
    
    Uh... Those two lines couldn't be moved to respective declarations of `nodes` and `hash`?


---

[GitHub] zookeeper pull request #632: [ZOOKEEPER-3150] Add tree digest check and veri...

Posted by eolivelli <gi...@git.apache.org>.
Github user eolivelli commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/632#discussion_r223181405
  
    --- Diff: zookeeper-server/src/main/java/org/apache/zookeeper/server/DataTree.java ---
    @@ -1521,4 +1562,179 @@ public boolean removeWatch(String path, WatcherType type, Watcher watcher) {
         public ReferenceCountedACLCache getReferenceCountedAclCache() {
             return aclCache;
         }
    +
    +    /**
    +     * Add the digest to the historical list, and update the latest zxid digest.
    +     */
    +    private void logZxidDigest(long zxid, long digest) {
    +        ZxidDigest zxidDigest = new ZxidDigest(zxid, DigestCalculator.DIGEST_VERSION, digest);
    +        lastProcessedZxidDigest = zxidDigest;
    +        if (zxidDigest.zxid % 128 == 0) {
    +            synchronized (digestLog) {
    +                digestLog.add(zxidDigest);
    +                if (digestLog.size() > DIGEST_LOG_LIMIT) {
    +                    digestLog.poll();
    +                }
    +            }
    +        }
    +    }
    +
    +    /**
    +     * Serializing the digest to snapshot, this is done after the data tree 
    +     * is being serialized, so when we replay the txns and it hits this zxid 
    +     * we know we should be in a non-fuzzy state, and have the same digest. 
    +     *
    +     * @param oa the output stream to write to 
    +     * @return true if the digest is serialized successfully
    +     */
    +    public Boolean serializeZxidDigest(OutputArchive oa) throws IOException {
    +        if (!DigestCalculator.digestEnabled()) {
    +            return false;
    +        }
    +
    +        ZxidDigest zxidDigest = lastProcessedZxidDigest;
    +        if (zxidDigest == null) {
    +            // write an empty digest
    +            zxidDigest = new ZxidDigest();
    +        }
    +        zxidDigest.serialize(oa);
    +        return true;
    +    }
    +
    +    /**
    +     * Deserializing the zxid digest from the input stream and update the 
    +     * digestFromLoadedSnapshot.
    +     *
    +     * @param ia the input stream to read from
    +     * @return the true if it deserialized successfully
    +     */
    +    public Boolean deserializeZxidDigest(InputArchive ia) throws IOException {
    --- End diff --
    
    Nit: boolean


---

[GitHub] zookeeper pull request #632: [ZOOKEEPER-3150] Add tree digest check and veri...

Posted by eribeiro <gi...@git.apache.org>.
Github user eribeiro commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/632#discussion_r225016509
  
    --- Diff: zookeeper-server/src/main/java/org/apache/zookeeper/server/util/AdHash.java ---
    @@ -0,0 +1,84 @@
    +/**
    + * 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.zookeeper.server.util;
    +
    +/**
    + * This incremental hash is used to keep track of the hash of
    + * the data tree to that we can quickly validate that things
    + * are in sync.
    + *
    + * See the excellent paper: A New Paradigm for collision-free hashing:
    + *   Incrementality at reduced cost,  M. Bellare and D. Micciancio
    + */
    +public class AdHash {
    --- End diff --
    
    Sometimes it is better to make it less restrictive and loosen up afterwards than make it more open and not being able to close thereafter. ;) But sure, it can go without this restriction. 


---

[GitHub] zookeeper pull request #632: [ZOOKEEPER-3150] Add tree digest check and veri...

Posted by lvfangmin <gi...@git.apache.org>.
Github user lvfangmin commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/632#discussion_r224844559
  
    --- Diff: zookeeper-server/src/main/java/org/apache/zookeeper/server/util/AdHash.java ---
    @@ -0,0 +1,84 @@
    +/**
    + * 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.zookeeper.server.util;
    +
    +/**
    + * This incremental hash is used to keep track of the hash of
    + * the data tree to that we can quickly validate that things
    + * are in sync.
    + *
    + * See the excellent paper: A New Paradigm for collision-free hashing:
    + *   Incrementality at reduced cost,  M. Bellare and D. Micciancio
    + */
    +public class AdHash {
    --- End diff --
    
    We may want to extend this class to modify the adHash a bit in the future, so I would leave it as is for now.


---

[GitHub] zookeeper pull request #632: [ZOOKEEPER-3150] Add tree digest check and veri...

Posted by eribeiro <gi...@git.apache.org>.
Github user eribeiro commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/632#discussion_r225016824
  
    --- Diff: zookeeper-server/src/main/java/org/apache/zookeeper/server/util/AdHash.java ---
    @@ -0,0 +1,84 @@
    +/**
    + * 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.zookeeper.server.util;
    +
    +/**
    + * This incremental hash is used to keep track of the hash of
    + * the data tree to that we can quickly validate that things
    + * are in sync.
    + *
    + * See the excellent paper: A New Paradigm for collision-free hashing:
    + *   Incrementality at reduced cost,  M. Bellare and D. Micciancio
    + */
    +public class AdHash {
    --- End diff --
    
    nit pick: rename this class to `AdHasher` or `Digester` or `AddHash` 


---

[GitHub] zookeeper pull request #632: [ZOOKEEPER-3150] Add tree digest check and veri...

Posted by eribeiro <gi...@git.apache.org>.
Github user eribeiro commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/632#discussion_r224212868
  
    --- Diff: zookeeper-server/src/main/java/org/apache/zookeeper/server/DataNode.java ---
    @@ -37,6 +37,14 @@
      * 
      */
     public class DataNode implements Record {
    +
    +    // the digest value of this node, calculated from path, data and stat
    +    private long digest;
    +
    +    // indicate if the digest of this node is up to date or not, used to 
    +    // optimize the performance.
    +    boolean digestCached;
    --- End diff --
    
    `volatile boolean digestChanged;` ?


---

[GitHub] zookeeper issue #632: [ZOOKEEPER-3150] Add tree digest check and verify data...

Posted by lvfangmin <gi...@git.apache.org>.
Github user lvfangmin commented on the issue:

    https://github.com/apache/zookeeper/pull/632
  
    This is a really useful feature, which helps us find multiple data inconsistent issues, like ZOOKEEPER-3144, ZOOKEEPER-3127, ZOOKEEPER-3125. 
    
    It can avoid introducing new inconsistent bugs in ZooKeeper in the future, so please take a look when you have time. I'll introduce the 2nd part after this got reviewed and merged.
    
    For performance, we saw some very minor impact, will provide the micro-benchmark result.


---

[GitHub] zookeeper pull request #632: [ZOOKEEPER-3150] Add tree digest check and veri...

Posted by lvfangmin <gi...@git.apache.org>.
Github user lvfangmin commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/632#discussion_r223538616
  
    --- Diff: zookeeper-server/src/main/java/org/apache/zookeeper/server/DataTree.java ---
    @@ -1521,4 +1562,179 @@ public boolean removeWatch(String path, WatcherType type, Watcher watcher) {
         public ReferenceCountedACLCache getReferenceCountedAclCache() {
             return aclCache;
         }
    +
    +    /**
    +     * Add the digest to the historical list, and update the latest zxid digest.
    +     */
    +    private void logZxidDigest(long zxid, long digest) {
    +        ZxidDigest zxidDigest = new ZxidDigest(zxid, DigestCalculator.DIGEST_VERSION, digest);
    +        lastProcessedZxidDigest = zxidDigest;
    +        if (zxidDigest.zxid % 128 == 0) {
    --- End diff --
    
    I'll add the comment here, basically we want to only export the history of digest every 128 txns. It's a random number we picked, but not all random, in hex it's 80, which will print nicer when we dump the digest history.


---

[GitHub] zookeeper pull request #632: [ZOOKEEPER-3150] Add tree digest check and veri...

Posted by lvfangmin <gi...@git.apache.org>.
Github user lvfangmin commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/632#discussion_r224843869
  
    --- Diff: zookeeper-server/src/main/java/org/apache/zookeeper/server/NodeHashMapImpl.java ---
    @@ -0,0 +1,116 @@
    +/**
    + * 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.zookeeper.server;
    +
    +import java.util.Map;
    +import java.util.Set;
    +import java.util.concurrent.ConcurrentHashMap;
    +
    +import org.apache.zookeeper.server.util.DigestCalculator;
    +import org.apache.zookeeper.server.util.AdHash;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +/**
    + * a simple wrapper to ConcurrentHashMap that recalculates a digest after
    + * each mutation.
    + */
    +public class NodeHashMapImpl implements NodeHashMap {
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(NodeHashMap.class);
    +
    +    private final ConcurrentHashMap<String, DataNode> nodes;
    +
    +    private AdHash hash;
    +
    +    public NodeHashMapImpl() {
    +        nodes = new ConcurrentHashMap<String, DataNode>();
    +        hash = new AdHash();
    --- End diff --
    
    We can do that, previously we have other initialization in this constructor, but not anymore after I refactored it, will remove the constructor here.


---

[GitHub] zookeeper pull request #632: [ZOOKEEPER-3150] Add tree digest check and veri...

Posted by edwardoliveira <gi...@git.apache.org>.
Github user edwardoliveira commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/632#discussion_r223546118
  
    --- Diff: zookeeper-server/src/main/java/org/apache/zookeeper/server/DataTree.java ---
    @@ -1521,4 +1562,179 @@ public boolean removeWatch(String path, WatcherType type, Watcher watcher) {
         public ReferenceCountedACLCache getReferenceCountedAclCache() {
             return aclCache;
         }
    +
    +    /**
    +     * Add the digest to the historical list, and update the latest zxid digest.
    +     */
    +    private void logZxidDigest(long zxid, long digest) {
    +        ZxidDigest zxidDigest = new ZxidDigest(zxid, DigestCalculator.DIGEST_VERSION, digest);
    +        lastProcessedZxidDigest = zxidDigest;
    +        if (zxidDigest.zxid % 128 == 0) {
    --- End diff --
    
    Better than put a comment it is to extract as a constant...


---

[GitHub] zookeeper pull request #632: [ZOOKEEPER-3150] Add tree digest check and veri...

Posted by eribeiro <gi...@git.apache.org>.
Github user eribeiro commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/632#discussion_r224206681
  
    --- Diff: zookeeper-server/src/main/java/org/apache/zookeeper/server/DataTree.java ---
    @@ -1521,4 +1566,179 @@ public boolean removeWatch(String path, WatcherType type, Watcher watcher) {
         public ReferenceCountedACLCache getReferenceCountedAclCache() {
             return aclCache;
         }
    +
    +    /**
    +     * Add the digest to the historical list, and update the latest zxid digest.
    +     */
    +    private void logZxidDigest(long zxid, long digest) {
    +        ZxidDigest zxidDigest = new ZxidDigest(zxid, DigestCalculator.DIGEST_VERSION, digest);
    +        lastProcessedZxidDigest = zxidDigest;
    +        if (zxidDigest.zxid % DIGEST_LOG_INTERVAL == 0) {
    +            synchronized (digestLog) {
    +                digestLog.add(zxidDigest);
    +                if (digestLog.size() > DIGEST_LOG_LIMIT) {
    +                    digestLog.poll();
    +                }
    +            }
    +        }
    +    }
    +
    +    /**
    +     * Serializing the digest to snapshot, this is done after the data tree 
    +     * is being serialized, so when we replay the txns and it hits this zxid 
    +     * we know we should be in a non-fuzzy state, and have the same digest. 
    +     *
    +     * @param oa the output stream to write to 
    +     * @return true if the digest is serialized successfully
    +     */
    +    public boolean serializeZxidDigest(OutputArchive oa) throws IOException {
    +        if (!DigestCalculator.digestEnabled()) {
    +            return false;
    +        }
    +
    +        ZxidDigest zxidDigest = lastProcessedZxidDigest;
    +        if (zxidDigest == null) {
    +            // write an empty digest
    +            zxidDigest = new ZxidDigest();
    +        }
    +        zxidDigest.serialize(oa);
    +        return true;
    +    }
    +
    +    /**
    +     * Deserializing the zxid digest from the input stream and update the 
    +     * digestFromLoadedSnapshot.
    +     *
    +     * @param ia the input stream to read from
    +     * @return the true if it deserialized successfully
    +     */
    +    public boolean deserializeZxidDigest(InputArchive ia) throws IOException {
    +        if (!DigestCalculator.digestEnabled()) {
    +            return false;
    +        }
    +
    +        try  {
    +            ZxidDigest zxidDigest = new ZxidDigest();
    +            zxidDigest.deserialize(ia);
    +            if (zxidDigest.zxid > 0) {
    +                digestFromLoadedSnapshot = zxidDigest;
    +            }
    +            return true;
    +        } catch (EOFException e) {
    +            LOG.warn("Got EOF exception while reading the digest, " +
    +                    "likely due to the reading an older snapshot.");
    +            return false;
    +        }
    +    }
    +
    +    /**
    +     * Compares the actual tree's digest with that in the snapshot. 
    +     * Resets digestFromLoadedSnapshot after comparision.
    +     *
    +     * @param zxid zxid
    +     */
    +    public void compareSnapshotDigests(long zxid) {
    +        if (zxid == digestFromLoadedSnapshot.zxid) {
    +            if (DigestCalculator.DIGEST_VERSION != digestFromLoadedSnapshot.digestVersion) {
    +                LOG.info("Digest version changed, local: {}, new: {}, " + 
    +                        "skip comparing digest now.", 
    +                        digestFromLoadedSnapshot.digestVersion, DigestCalculator.DIGEST_VERSION);
    +                digestFromLoadedSnapshot = null;
    +                return;
    +            }
    +            if (getTreeDigest() != digestFromLoadedSnapshot.getDigest()) {
    +                reportDigestMismatch(zxid);
    +            }
    +            digestFromLoadedSnapshot = null;
    +        } else if (digestFromLoadedSnapshot.zxid != 0 && zxid > digestFromLoadedSnapshot.zxid) {
    +            LOG.error("Watching for zxid 0x{} during snapshot recovery, " +
    +                    "but it wasn't found.", 
    +                    Long.toHexString(digestFromLoadedSnapshot.zxid));
    +        }
    +    }
    +
    +    /**
    +     * Reports any mismatch in the transaction digest.
    +     * @param zxid zxid for which the error is being reported.
    +     */
    +    public void reportDigestMismatch(long zxid) {
    +        ServerMetrics.DIGEST_MISMATCHES_COUNT.add(1);
    +        RATE_LOGGER.rateLimitLog("Digests are not matching. Value is Zxid.", 
    +                String.valueOf(zxid));
    +
    +        for (DigestWatcher watcher: digestWatchers) {
    +            watcher.process(zxid);
    +        }
    +    }
    +
    +    public long getTreeDigest() {
    +        return nodes.getDigest();
    +    }
    +
    +    public ZxidDigest getDigestFromLoadedSnapshot() {
    +        return digestFromLoadedSnapshot;
    +    }
    +
    +    /**
    +     * Add digest mismatch event handler.
    +     *
    +     * @param digestWatcher the handler to add
    +     */
    +    public void addDigestWatcher(DigestWatcher digestWatcher) {
    +        digestWatchers.add(digestWatcher);
    +    }
    +
    +    /**
    +     * Return all the digests in the historical digest list.
    +     */
    +    public LinkedList<ZxidDigest> getDigestLog() {
    +        synchronized (digestLog) {
    +            return new LinkedList<ZxidDigest>(digestLog);
    --- End diff --
    
    Maybe wrap in `Collections.unmodifiableList()`?


---

[GitHub] zookeeper pull request #632: [ZOOKEEPER-3150] Add tree digest check and veri...

Posted by eribeiro <gi...@git.apache.org>.
Github user eribeiro commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/632#discussion_r224208545
  
    --- Diff: zookeeper-server/src/main/java/org/apache/zookeeper/server/DataTree.java ---
    @@ -154,6 +160,26 @@
     
         private final ReferenceCountedACLCache aclCache = new ReferenceCountedACLCache();
     
    +    // The maximum number of tree digests that we will keep in our history
    +    public static final int DIGEST_LOG_LIMIT = 1024;
    +
    +    // Dump digest every 128 txns, in hex it's 80, which will make it easier 
    +    // to align and compare between servers.
    +    public static final int DIGEST_LOG_INTERVAL = 128;
    +
    +    // If this is not null, we are actively looking for a target zxid that we
    +    // want to validate the digest for
    +    private ZxidDigest digestFromLoadedSnapshot;
    +
    +    // The digest associated with the highest zxid in the data tree.
    +    private volatile ZxidDigest lastProcessedZxidDigest;
    +
    +    // Will be notified when digest mismatch event triggered.
    +    private List<DigestWatcher> digestWatchers = new ArrayList<>();
    --- End diff --
    
    tip: when you are dealing with Observer pattern in Java where the list of observers can potentially change concurrently with the traversal of the list for notification this can lead to `ConcurrentModificationException`. It's better to use `CopyOnWriteArrayList` or `CopyOnWriteArraySet`, given some important constraints specified in the javadocs, because they guarantee that a traversal over the collection will never throw `ConcurrentModificationException`. 
    
    Besides that, it would be nice to use a `Set` instead of a `List` because then there's no risk of bloating the collection by adding duplicate observers. So, my suggestion would be to put:
    
    `private Set<DigestWatcher> digestWatchers = new CopyOnWriteArraySet<>();`


---

[GitHub] zookeeper pull request #632: [ZOOKEEPER-3150] Add tree digest check and veri...

Posted by eribeiro <gi...@git.apache.org>.
Github user eribeiro commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/632#discussion_r225016431
  
    --- Diff: zookeeper-server/src/main/java/org/apache/zookeeper/server/DataTree.java ---
    @@ -154,6 +160,26 @@
     
         private final ReferenceCountedACLCache aclCache = new ReferenceCountedACLCache();
     
    +    // The maximum number of tree digests that we will keep in our history
    +    public static final int DIGEST_LOG_LIMIT = 1024;
    +
    +    // Dump digest every 128 txns, in hex it's 80, which will make it easier 
    +    // to align and compare between servers.
    +    public static final int DIGEST_LOG_INTERVAL = 128;
    +
    +    // If this is not null, we are actively looking for a target zxid that we
    +    // want to validate the digest for
    +    private ZxidDigest digestFromLoadedSnapshot;
    +
    +    // The digest associated with the highest zxid in the data tree.
    +    private volatile ZxidDigest lastProcessedZxidDigest;
    +
    +    // Will be notified when digest mismatch event triggered.
    +    private List<DigestWatcher> digestWatchers = new ArrayList<>();
    --- End diff --
    
    Oh, good points. Firstly, one thing is: this list could be made `final`, right? :)
    
    > We won't have concurrent modification here
    
    So, `public void addDigestWatcher(DigestWatcher digestWatcher)` couldn't under any circumstances be called while `reportDigestMismatch` executes the loop at line 1669? Both are public methods, so that motivated me to write the first observation about this, but I can surely be missing something obvious...



---

[GitHub] zookeeper pull request #632: [ZOOKEEPER-3150] Add tree digest check and veri...

Posted by lvfangmin <gi...@git.apache.org>.
Github user lvfangmin commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/632#discussion_r224843197
  
    --- Diff: zookeeper-server/src/main/java/org/apache/zookeeper/server/DataTree.java ---
    @@ -1521,4 +1566,179 @@ public boolean removeWatch(String path, WatcherType type, Watcher watcher) {
         public ReferenceCountedACLCache getReferenceCountedAclCache() {
             return aclCache;
         }
    +
    +    /**
    +     * Add the digest to the historical list, and update the latest zxid digest.
    +     */
    +    private void logZxidDigest(long zxid, long digest) {
    +        ZxidDigest zxidDigest = new ZxidDigest(zxid, DigestCalculator.DIGEST_VERSION, digest);
    +        lastProcessedZxidDigest = zxidDigest;
    +        if (zxidDigest.zxid % DIGEST_LOG_INTERVAL == 0) {
    +            synchronized (digestLog) {
    +                digestLog.add(zxidDigest);
    +                if (digestLog.size() > DIGEST_LOG_LIMIT) {
    +                    digestLog.poll();
    +                }
    +            }
    +        }
    +    }
    +
    +    /**
    +     * Serializing the digest to snapshot, this is done after the data tree 
    +     * is being serialized, so when we replay the txns and it hits this zxid 
    +     * we know we should be in a non-fuzzy state, and have the same digest. 
    +     *
    +     * @param oa the output stream to write to 
    +     * @return true if the digest is serialized successfully
    +     */
    +    public boolean serializeZxidDigest(OutputArchive oa) throws IOException {
    +        if (!DigestCalculator.digestEnabled()) {
    +            return false;
    +        }
    +
    +        ZxidDigest zxidDigest = lastProcessedZxidDigest;
    +        if (zxidDigest == null) {
    +            // write an empty digest
    +            zxidDigest = new ZxidDigest();
    +        }
    +        zxidDigest.serialize(oa);
    +        return true;
    +    }
    +
    +    /**
    +     * Deserializing the zxid digest from the input stream and update the 
    +     * digestFromLoadedSnapshot.
    +     *
    +     * @param ia the input stream to read from
    +     * @return the true if it deserialized successfully
    +     */
    +    public boolean deserializeZxidDigest(InputArchive ia) throws IOException {
    +        if (!DigestCalculator.digestEnabled()) {
    +            return false;
    +        }
    +
    +        try  {
    +            ZxidDigest zxidDigest = new ZxidDigest();
    +            zxidDigest.deserialize(ia);
    +            if (zxidDigest.zxid > 0) {
    +                digestFromLoadedSnapshot = zxidDigest;
    +            }
    +            return true;
    --- End diff --
    
    I usually do early return if there are a bunch of code need to be handled after that early return, but I'm not a fan of moving 'return true" to the end of this function as a 'default' value, it's actually makes the code a bit harder to read to me, given that the 'early return' at line 1630 is also the kind of last line of this function.


---

[GitHub] zookeeper issue #632: [ZOOKEEPER-3150] Add tree digest check and verify data...

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit commented on the issue:

    https://github.com/apache/zookeeper/pull/632
  
    
    Refer to this link for build results (access rights to CI server needed): 
    https://builds.apache.org/job/PreCommit-ZOOKEEPER-github-pr-build/2430/



---

[GitHub] zookeeper pull request #632: [ZOOKEEPER-3150] Add tree digest check and veri...

Posted by lvfangmin <gi...@git.apache.org>.
Github user lvfangmin commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/632#discussion_r227533265
  
    --- Diff: zookeeper-server/src/main/java/org/apache/zookeeper/server/util/AdHash.java ---
    @@ -0,0 +1,84 @@
    +/**
    + * 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.zookeeper.server.util;
    +
    +/**
    + * This incremental hash is used to keep track of the hash of
    + * the data tree to that we can quickly validate that things
    + * are in sync.
    + *
    + * See the excellent paper: A New Paradigm for collision-free hashing:
    + *   Incrementality at reduced cost,  M. Bellare and D. Micciancio
    + */
    +public class AdHash {
    --- End diff --
    
    I'd like to keep it as is, since it's consistent with the name in the paper.


---

[GitHub] zookeeper issue #632: [ZOOKEEPER-3150] Add tree digest check and verify data...

Posted by lvfangmin <gi...@git.apache.org>.
Github user lvfangmin commented on the issue:

    https://github.com/apache/zookeeper/pull/632
  
    retest this please


---

[GitHub] zookeeper issue #632: [ZOOKEEPER-3150] Add tree digest check and verify data...

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit commented on the issue:

    https://github.com/apache/zookeeper/pull/632
  
    
    Refer to this link for build results (access rights to CI server needed): 
    https://builds.apache.org/job/PreCommit-ZOOKEEPER-github-pr-build/2374/



---

[GitHub] zookeeper pull request #632: [ZOOKEEPER-3150] Add tree digest check and veri...

Posted by eolivelli <gi...@git.apache.org>.
Github user eolivelli commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/632#discussion_r223181364
  
    --- Diff: zookeeper-server/src/main/java/org/apache/zookeeper/server/DataTree.java ---
    @@ -1521,4 +1562,179 @@ public boolean removeWatch(String path, WatcherType type, Watcher watcher) {
         public ReferenceCountedACLCache getReferenceCountedAclCache() {
             return aclCache;
         }
    +
    +    /**
    +     * Add the digest to the historical list, and update the latest zxid digest.
    +     */
    +    private void logZxidDigest(long zxid, long digest) {
    +        ZxidDigest zxidDigest = new ZxidDigest(zxid, DigestCalculator.DIGEST_VERSION, digest);
    +        lastProcessedZxidDigest = zxidDigest;
    +        if (zxidDigest.zxid % 128 == 0) {
    +            synchronized (digestLog) {
    +                digestLog.add(zxidDigest);
    +                if (digestLog.size() > DIGEST_LOG_LIMIT) {
    +                    digestLog.poll();
    +                }
    +            }
    +        }
    +    }
    +
    +    /**
    +     * Serializing the digest to snapshot, this is done after the data tree 
    +     * is being serialized, so when we replay the txns and it hits this zxid 
    +     * we know we should be in a non-fuzzy state, and have the same digest. 
    +     *
    +     * @param oa the output stream to write to 
    +     * @return true if the digest is serialized successfully
    +     */
    +    public Boolean serializeZxidDigest(OutputArchive oa) throws IOException {
    --- End diff --
    
    Nit: boolean not Boolean


---

[GitHub] zookeeper issue #632: [ZOOKEEPER-3150] Add tree digest check and verify data...

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit commented on the issue:

    https://github.com/apache/zookeeper/pull/632
  
    
    Refer to this link for build results (access rights to CI server needed): 
    https://builds.apache.org/job/PreCommit-ZOOKEEPER-github-pr-build/2375/



---

[GitHub] zookeeper pull request #632: [ZOOKEEPER-3150] Add tree digest check and veri...

Posted by eribeiro <gi...@git.apache.org>.
Github user eribeiro commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/632#discussion_r224209494
  
    --- Diff: zookeeper-server/src/main/java/org/apache/zookeeper/server/util/AdHash.java ---
    @@ -0,0 +1,84 @@
    +/**
    + * 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.zookeeper.server.util;
    +
    +/**
    + * This incremental hash is used to keep track of the hash of
    + * the data tree to that we can quickly validate that things
    + * are in sync.
    + *
    + * See the excellent paper: A New Paradigm for collision-free hashing:
    + *   Incrementality at reduced cost,  M. Bellare and D. Micciancio
    + */
    +public class AdHash {
    --- End diff --
    
    Does it make sense to make the class `final`?


---

[GitHub] zookeeper pull request #632: [ZOOKEEPER-3150] Add tree digest check and veri...

Posted by eribeiro <gi...@git.apache.org>.
Github user eribeiro commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/632#discussion_r224213395
  
    --- Diff: zookeeper-server/src/main/java/org/apache/zookeeper/server/DataNode.java ---
    @@ -37,6 +37,14 @@
      * 
      */
     public class DataNode implements Record {
    +
    +    // the digest value of this node, calculated from path, data and stat
    +    private long digest;
    --- End diff --
    
    `private volatile long digest;`?


---

[GitHub] zookeeper pull request #632: [ZOOKEEPER-3150] Add tree digest check and veri...

Posted by lvfangmin <gi...@git.apache.org>.
Github user lvfangmin commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/632#discussion_r224841261
  
    --- Diff: zookeeper-server/src/main/java/org/apache/zookeeper/server/DataNode.java ---
    @@ -37,6 +37,14 @@
      * 
      */
     public class DataNode implements Record {
    +
    +    // the digest value of this node, calculated from path, data and stat
    +    private long digest;
    --- End diff --
    
    It's a single thread read/write this value for now, so it doesn't matter, but I agree it would be better to have it as a volatile in case we need to visit this in different thread in the future, will change that.


---

[GitHub] zookeeper pull request #632: [ZOOKEEPER-3150] Add tree digest check and veri...

Posted by eribeiro <gi...@git.apache.org>.
Github user eribeiro commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/632#discussion_r225016751
  
    --- Diff: zookeeper-server/src/main/java/org/apache/zookeeper/server/command/HashCommand.java ---
    @@ -0,0 +1,49 @@
    +/**
    + * 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.zookeeper.server.command;
    +
    +import java.io.PrintWriter;
    +import java.util.List;
    +
    +import org.apache.zookeeper.server.DataTree.ZxidDigest;
    +import org.apache.zookeeper.server.ServerCnxn;
    +
    +/**
    + * Command used to dump the latest digest histories.
    + */
    +public class HashCommand extends AbstractFourLetterCommand {
    --- End diff --
    
    Wouldn't be the case of renaming this class to `DigestCommand` or `DigestLogCommand`?


---

[GitHub] zookeeper pull request #632: [ZOOKEEPER-3150] Add tree digest check and veri...

Posted by lvfangmin <gi...@git.apache.org>.
Github user lvfangmin commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/632#discussion_r224842221
  
    --- Diff: zookeeper-server/src/main/java/org/apache/zookeeper/server/DataTree.java ---
    @@ -154,6 +160,26 @@
     
         private final ReferenceCountedACLCache aclCache = new ReferenceCountedACLCache();
     
    +    // The maximum number of tree digests that we will keep in our history
    +    public static final int DIGEST_LOG_LIMIT = 1024;
    +
    +    // Dump digest every 128 txns, in hex it's 80, which will make it easier 
    +    // to align and compare between servers.
    +    public static final int DIGEST_LOG_INTERVAL = 128;
    +
    +    // If this is not null, we are actively looking for a target zxid that we
    +    // want to validate the digest for
    +    private ZxidDigest digestFromLoadedSnapshot;
    +
    +    // The digest associated with the highest zxid in the data tree.
    +    private volatile ZxidDigest lastProcessedZxidDigest;
    +
    +    // Will be notified when digest mismatch event triggered.
    +    private List<DigestWatcher> digestWatchers = new ArrayList<>();
    --- End diff --
    
    We won't have concurrent modification here, this will be a static list and registered in the constructor of this class, so I don't expect to change this dynamically.
    



---

[GitHub] zookeeper pull request #632: [ZOOKEEPER-3150] Add tree digest check and veri...

Posted by eribeiro <gi...@git.apache.org>.
Github user eribeiro commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/632#discussion_r225016598
  
    --- Diff: zookeeper-server/src/main/java/org/apache/zookeeper/server/DataTree.java ---
    @@ -1521,4 +1566,179 @@ public boolean removeWatch(String path, WatcherType type, Watcher watcher) {
         public ReferenceCountedACLCache getReferenceCountedAclCache() {
             return aclCache;
         }
    +
    +    /**
    +     * Add the digest to the historical list, and update the latest zxid digest.
    +     */
    +    private void logZxidDigest(long zxid, long digest) {
    +        ZxidDigest zxidDigest = new ZxidDigest(zxid, DigestCalculator.DIGEST_VERSION, digest);
    +        lastProcessedZxidDigest = zxidDigest;
    +        if (zxidDigest.zxid % DIGEST_LOG_INTERVAL == 0) {
    +            synchronized (digestLog) {
    +                digestLog.add(zxidDigest);
    +                if (digestLog.size() > DIGEST_LOG_LIMIT) {
    +                    digestLog.poll();
    +                }
    +            }
    +        }
    +    }
    +
    +    /**
    +     * Serializing the digest to snapshot, this is done after the data tree 
    +     * is being serialized, so when we replay the txns and it hits this zxid 
    +     * we know we should be in a non-fuzzy state, and have the same digest. 
    +     *
    +     * @param oa the output stream to write to 
    +     * @return true if the digest is serialized successfully
    +     */
    +    public boolean serializeZxidDigest(OutputArchive oa) throws IOException {
    +        if (!DigestCalculator.digestEnabled()) {
    +            return false;
    +        }
    +
    +        ZxidDigest zxidDigest = lastProcessedZxidDigest;
    +        if (zxidDigest == null) {
    +            // write an empty digest
    +            zxidDigest = new ZxidDigest();
    +        }
    +        zxidDigest.serialize(oa);
    +        return true;
    +    }
    +
    +    /**
    +     * Deserializing the zxid digest from the input stream and update the 
    +     * digestFromLoadedSnapshot.
    +     *
    +     * @param ia the input stream to read from
    +     * @return the true if it deserialized successfully
    +     */
    +    public boolean deserializeZxidDigest(InputArchive ia) throws IOException {
    +        if (!DigestCalculator.digestEnabled()) {
    +            return false;
    +        }
    +
    +        try  {
    +            ZxidDigest zxidDigest = new ZxidDigest();
    +            zxidDigest.deserialize(ia);
    +            if (zxidDigest.zxid > 0) {
    +                digestFromLoadedSnapshot = zxidDigest;
    +            }
    +            return true;
    --- End diff --
    
    Good point.


---

[GitHub] zookeeper pull request #632: [ZOOKEEPER-3150] Add tree digest check and veri...

Posted by lvfangmin <gi...@git.apache.org>.
Github user lvfangmin commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/632#discussion_r227532976
  
    --- Diff: zookeeper-server/src/main/java/org/apache/zookeeper/server/command/HashCommand.java ---
    @@ -0,0 +1,49 @@
    +/**
    + * 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.zookeeper.server.command;
    +
    +import java.io.PrintWriter;
    +import java.util.List;
    +
    +import org.apache.zookeeper.server.DataTree.ZxidDigest;
    +import org.apache.zookeeper.server.ServerCnxn;
    +
    +/**
    + * Command used to dump the latest digest histories.
    + */
    +public class HashCommand extends AbstractFourLetterCommand {
    --- End diff --
    
    That seems more consistent, will do.


---

[GitHub] zookeeper issue #632: [ZOOKEEPER-3150] Add tree digest check and verify data...

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit commented on the issue:

    https://github.com/apache/zookeeper/pull/632
  
    
    Refer to this link for build results (access rights to CI server needed): 
    https://builds.apache.org/job/PreCommit-ZOOKEEPER-github-pr-build/2385/



---

[GitHub] zookeeper issue #632: [ZOOKEEPER-3150] Add tree digest check and verify data...

Posted by lvfangmin <gi...@git.apache.org>.
Github user lvfangmin commented on the issue:

    https://github.com/apache/zookeeper/pull/632
  
    Rebase onto latest master.


---

[GitHub] zookeeper pull request #632: [ZOOKEEPER-3150] Add tree digest check and veri...

Posted by eolivelli <gi...@git.apache.org>.
Github user eolivelli commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/632#discussion_r223181383
  
    --- Diff: zookeeper-server/src/main/java/org/apache/zookeeper/server/DataTree.java ---
    @@ -1521,4 +1562,179 @@ public boolean removeWatch(String path, WatcherType type, Watcher watcher) {
         public ReferenceCountedACLCache getReferenceCountedAclCache() {
             return aclCache;
         }
    +
    +    /**
    +     * Add the digest to the historical list, and update the latest zxid digest.
    +     */
    +    private void logZxidDigest(long zxid, long digest) {
    +        ZxidDigest zxidDigest = new ZxidDigest(zxid, DigestCalculator.DIGEST_VERSION, digest);
    +        lastProcessedZxidDigest = zxidDigest;
    +        if (zxidDigest.zxid % 128 == 0) {
    --- End diff --
    
    Can you explain this magic value '128' ? Maybe a comment will help.
    
    Maybe I am missing something


---

[GitHub] zookeeper pull request #632: [ZOOKEEPER-3150] Add tree digest check and veri...

Posted by eribeiro <gi...@git.apache.org>.
Github user eribeiro commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/632#discussion_r224206365
  
    --- Diff: zookeeper-server/src/main/java/org/apache/zookeeper/server/DataTree.java ---
    @@ -1521,4 +1566,179 @@ public boolean removeWatch(String path, WatcherType type, Watcher watcher) {
         public ReferenceCountedACLCache getReferenceCountedAclCache() {
             return aclCache;
         }
    +
    +    /**
    +     * Add the digest to the historical list, and update the latest zxid digest.
    +     */
    +    private void logZxidDigest(long zxid, long digest) {
    +        ZxidDigest zxidDigest = new ZxidDigest(zxid, DigestCalculator.DIGEST_VERSION, digest);
    +        lastProcessedZxidDigest = zxidDigest;
    +        if (zxidDigest.zxid % DIGEST_LOG_INTERVAL == 0) {
    +            synchronized (digestLog) {
    +                digestLog.add(zxidDigest);
    +                if (digestLog.size() > DIGEST_LOG_LIMIT) {
    +                    digestLog.poll();
    +                }
    +            }
    +        }
    +    }
    +
    +    /**
    +     * Serializing the digest to snapshot, this is done after the data tree 
    +     * is being serialized, so when we replay the txns and it hits this zxid 
    +     * we know we should be in a non-fuzzy state, and have the same digest. 
    +     *
    +     * @param oa the output stream to write to 
    +     * @return true if the digest is serialized successfully
    +     */
    +    public boolean serializeZxidDigest(OutputArchive oa) throws IOException {
    +        if (!DigestCalculator.digestEnabled()) {
    +            return false;
    +        }
    +
    +        ZxidDigest zxidDigest = lastProcessedZxidDigest;
    +        if (zxidDigest == null) {
    +            // write an empty digest
    +            zxidDigest = new ZxidDigest();
    +        }
    +        zxidDigest.serialize(oa);
    +        return true;
    +    }
    +
    +    /**
    +     * Deserializing the zxid digest from the input stream and update the 
    +     * digestFromLoadedSnapshot.
    +     *
    +     * @param ia the input stream to read from
    +     * @return the true if it deserialized successfully
    +     */
    +    public boolean deserializeZxidDigest(InputArchive ia) throws IOException {
    +        if (!DigestCalculator.digestEnabled()) {
    +            return false;
    +        }
    +
    +        try  {
    +            ZxidDigest zxidDigest = new ZxidDigest();
    +            zxidDigest.deserialize(ia);
    +            if (zxidDigest.zxid > 0) {
    +                digestFromLoadedSnapshot = zxidDigest;
    +            }
    +            return true;
    --- End diff --
    
    nit: put the `return true;` as the last line in the method. Therefore, this is the default behavior, and it return false early in case of failure (line 1617 and 1630).


---