You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@zookeeper.apache.org by enixon <gi...@git.apache.org> on 2018/08/31 22:50:45 UTC

[GitHub] zookeeper pull request #615: ZOOKEEPER-3137: add a utility to truncate logs ...

GitHub user enixon opened a pull request:

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

    ZOOKEEPER-3137: add a utility to truncate logs to a zxid

    

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

    $ git pull https://github.com/enixon/zookeeper add-chop

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

    https://github.com/apache/zookeeper/pull/615.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 #615
    
----
commit 77540aadc862397b32e3d11fb40c1427ed5729cb
Author: Brian Nixon <ni...@...>
Date:   2018-08-31T22:44:34Z

    ZOOKEEPER-3137: add a utility to truncate logs to a zxid

----


---

[GitHub] zookeeper pull request #615: ZOOKEEPER-3137: add a utility to truncate logs ...

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

    https://github.com/apache/zookeeper/pull/615#discussion_r215129868
  
    --- Diff: src/java/main/org/apache/zookeeper/server/util/LogChopper.java ---
    @@ -0,0 +1,152 @@
    +/**
    + * 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;
    +
    +import org.apache.jute.BinaryInputArchive;
    +import org.apache.jute.BinaryOutputArchive;
    +import org.apache.jute.Record;
    +import org.apache.zookeeper.server.persistence.FileHeader;
    +import org.apache.zookeeper.server.persistence.FileTxnLog;
    +import org.apache.zookeeper.txn.TxnHeader;
    +
    +import java.io.BufferedInputStream;
    +import java.io.BufferedOutputStream;
    +import java.io.EOFException;
    +import java.io.FileInputStream;
    +import java.io.FileOutputStream;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.io.OutputStream;
    +import java.util.zip.Adler32;
    +import java.util.zip.Checksum;
    +
    +/**
    + * this class will chop the log at the specified zxid
    + */
    +public class LogChopper {
    +    public static void main(String args[]) {
    +        if (args.length != 3) {
    +            System.out.println("Usage: LogChopper zxid_to_chop_to txn_log_to_chop chopped_filename");
    +            System.out.println("    this program will read the txn_log_to_chop file and copy all the transactions");
    +            System.out.println("    from it up to (and including) the given zxid into chopped_filename.");
    +            System.exit(1);
    +        }
    +        long zxid = Long.decode(args[0]);
    +        String txnLog = args[1];
    +        String choppedLog = args[2];
    +
    +        int rc = 2;
    +        try (
    +            InputStream is = new BufferedInputStream(new FileInputStream(txnLog));
    +            OutputStream os = new BufferedOutputStream(new FileOutputStream(choppedLog))
    +        ) {
    +            if (chop(is, os, zxid)) {
    +                rc = 0;
    +            }
    +        } catch (Exception e) {
    +            System.out.println("Got exception: " + e.getMessage());
    +        }
    +        System.exit(rc);
    +    }
    +
    +    public static boolean chop(InputStream is, OutputStream os, long zxid) throws IOException {
    +        BinaryInputArchive logStream = BinaryInputArchive.getArchive(is);
    +        BinaryOutputArchive choppedStream = BinaryOutputArchive.getArchive(os);
    +        FileHeader fhdr = new FileHeader();
    +        fhdr.deserialize(logStream, "fileheader");
    +
    +        if (fhdr.getMagic() != FileTxnLog.TXNLOG_MAGIC) {
    +            System.err.println("Invalid magic number in txn log file");
    +            return false;
    +        }
    +        System.out.println("ZooKeeper Transactional Log File with dbid "
    +                + fhdr.getDbid() + " txnlog format version "
    +                + fhdr.getVersion());
    +
    +        fhdr.serialize(choppedStream, "fileheader");
    +        int count = 0;
    +        boolean hasZxid = false;
    +        long previousZxid = -1;
    +        while (true) {
    +            long crcValue;
    +            byte[] bytes;
    +            try {
    +                crcValue = logStream.readLong("crcvalue");
    +
    +                bytes = logStream.readBuffer("txnEntry");
    +            } catch (EOFException e) {
    +                System.out.println("EOF reached after " + count + " txns.");
    +                // returning false because nothing was chopped
    +                return false;
    +            }
    +            if (bytes.length == 0) {
    +                // Since we preallocate, we define EOF to be an
    +                // empty transaction
    +                System.out.println("EOF reached after " + count + " txns.");
    +                // returning false because nothing was chopped
    +                return false;
    +            }
    +
    +            Checksum crc = new Adler32();
    +            crc.update(bytes, 0, bytes.length);
    +            if (crcValue != crc.getValue()) {
    +                throw new IOException("CRC doesn't match " + crcValue +
    +                        " vs " + crc.getValue());
    +            }
    +            TxnHeader hdr = new TxnHeader();
    +            Record txn = SerializeUtils.deserializeTxn(bytes, hdr);
    +            if (logStream.readByte("EOR") != 'B') {
    +                System.out.println("Last transaction was partial.");
    +                throw new EOFException("Last transaction was partial.");
    +            }
    +
    +            long txnZxid = hdr.getZxid();
    +            if (txnZxid == zxid) {
    +                hasZxid = true;
    +            }
    +
    +            // logging the gap to make the inconsistency investigation easier
    +            if (previousZxid != -1 && txnZxid != previousZxid + 1) {
    +                long txnEpoch = ZxidUtils.getEpochFromZxid(txnZxid);
    +                long txnCounter = ZxidUtils.getCounterFromZxid(txnZxid);
    +                long previousEpoch = ZxidUtils.getEpochFromZxid(previousZxid);
    +                if (txnEpoch == previousEpoch || txnCounter != 1) {
    --- End diff --
    
    Would it be better if we print more targeted diagnostic information here for both conditions, instead of reuse a single error message? For example:
    * Log gap between txns of the same epoch.
    * Log gap of the new epoch (from 1 to txnCounter)
    Might make it more easier to debug log.


---

[GitHub] zookeeper issue #615: ZOOKEEPER-3137: add a utility to truncate logs to a zx...

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

    https://github.com/apache/zookeeper/pull/615
  
    @hanm it sounds like @anmolnar was okay with the form of my TxnLogToolkit patch. I'll add it on top of this pull request and then will consider it ready to merge.


---

[GitHub] zookeeper pull request #615: ZOOKEEPER-3137: add a utility to truncate logs ...

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

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


---

[GitHub] zookeeper issue #615: ZOOKEEPER-3137: add a utility to truncate logs to a zx...

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

    https://github.com/apache/zookeeper/pull/615
  
    @hanm, I've made your recommended changes with the exception noted in my comment. 
    
    @anmolnar , looking at TxnLogToolkit now - I think this should be easy to integrate  :)


---

[GitHub] zookeeper issue #615: ZOOKEEPER-3137: add a utility to truncate logs to a zx...

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

    https://github.com/apache/zookeeper/pull/615
  
    @enixon Thanks for looking into the integration.
    Code sharing is not so important. TxnLogToolkit is intended to be a collection of all cli tools related to ZooKeeper transaction logs. So it's perfectly fine if you just drop the code into and add an extra option to trigger the new functionality. I really appreciate the effort.
    
    @hanm TxnLogToolkit is kind of the rewrite of LogFormatter with extra functionality, so LogFormatter can be just deco'd with no harm.


---

[GitHub] zookeeper issue #615: ZOOKEEPER-3137: add a utility to truncate logs to a zx...

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

    https://github.com/apache/zookeeper/pull/615
  
    Sample usage is something like:
    
    `java -cp build/zookeeper-3.6.0-SNAPSHOT.jar:build/lib/* org.apache.zookeeper.server.util.LogChopper 0x33500000003 log.3330000000b shorterlog.3330000000b`


---

[GitHub] zookeeper issue #615: ZOOKEEPER-3137: add a utility to truncate logs to a zx...

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

    https://github.com/apache/zookeeper/pull/615
  
    @enixon Your patch looks good, this is going to be a useful tool for ZooKeeper.
    Have you considered adding this functionality to `TxnLogToolkit` (a tool developed by me :), instead of creating a new one?


---

[GitHub] zookeeper issue #615: ZOOKEEPER-3137: add a utility to truncate logs to a zx...

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

    https://github.com/apache/zookeeper/pull/615
  
    Integrate into TxnLogToolkit is a good idea. I did not mention it as I thought that might require a separate JIRA, because LogFormatter can also be integrated (it's the least powerful tool of the three) and then deprecated, and also the current place of TxnLogToolkit is a little bit weird, IMO server/util makes more sense than server/persistence. 


---

[GitHub] zookeeper pull request #615: ZOOKEEPER-3137: add a utility to truncate logs ...

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

    https://github.com/apache/zookeeper/pull/615#discussion_r215446231
  
    --- Diff: src/java/main/org/apache/zookeeper/server/util/LogChopper.java ---
    @@ -0,0 +1,152 @@
    +/**
    + * 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;
    +
    +import org.apache.jute.BinaryInputArchive;
    +import org.apache.jute.BinaryOutputArchive;
    +import org.apache.jute.Record;
    +import org.apache.zookeeper.server.persistence.FileHeader;
    +import org.apache.zookeeper.server.persistence.FileTxnLog;
    +import org.apache.zookeeper.txn.TxnHeader;
    +
    +import java.io.BufferedInputStream;
    +import java.io.BufferedOutputStream;
    +import java.io.EOFException;
    +import java.io.FileInputStream;
    +import java.io.FileOutputStream;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.io.OutputStream;
    +import java.util.zip.Adler32;
    +import java.util.zip.Checksum;
    +
    +/**
    + * this class will chop the log at the specified zxid
    + */
    +public class LogChopper {
    +    public static void main(String args[]) {
    +        if (args.length != 3) {
    +            System.out.println("Usage: LogChopper zxid_to_chop_to txn_log_to_chop chopped_filename");
    +            System.out.println("    this program will read the txn_log_to_chop file and copy all the transactions");
    +            System.out.println("    from it up to (and including) the given zxid into chopped_filename.");
    +            System.exit(1);
    +        }
    +        long zxid = Long.decode(args[0]);
    +        String txnLog = args[1];
    +        String choppedLog = args[2];
    +
    +        int rc = 2;
    +        try (
    +            InputStream is = new BufferedInputStream(new FileInputStream(txnLog));
    +            OutputStream os = new BufferedOutputStream(new FileOutputStream(choppedLog))
    +        ) {
    +            if (chop(is, os, zxid)) {
    +                rc = 0;
    +            }
    +        } catch (Exception e) {
    +            System.out.println("Got exception: " + e.getMessage());
    +        }
    +        System.exit(rc);
    +    }
    +
    +    public static boolean chop(InputStream is, OutputStream os, long zxid) throws IOException {
    +        BinaryInputArchive logStream = BinaryInputArchive.getArchive(is);
    +        BinaryOutputArchive choppedStream = BinaryOutputArchive.getArchive(os);
    +        FileHeader fhdr = new FileHeader();
    +        fhdr.deserialize(logStream, "fileheader");
    +
    +        if (fhdr.getMagic() != FileTxnLog.TXNLOG_MAGIC) {
    +            System.err.println("Invalid magic number in txn log file");
    +            return false;
    +        }
    +        System.out.println("ZooKeeper Transactional Log File with dbid "
    +                + fhdr.getDbid() + " txnlog format version "
    +                + fhdr.getVersion());
    +
    +        fhdr.serialize(choppedStream, "fileheader");
    +        int count = 0;
    +        boolean hasZxid = false;
    +        long previousZxid = -1;
    +        while (true) {
    +            long crcValue;
    +            byte[] bytes;
    +            try {
    +                crcValue = logStream.readLong("crcvalue");
    +
    +                bytes = logStream.readBuffer("txnEntry");
    +            } catch (EOFException e) {
    +                System.out.println("EOF reached after " + count + " txns.");
    +                // returning false because nothing was chopped
    +                return false;
    +            }
    +            if (bytes.length == 0) {
    +                // Since we preallocate, we define EOF to be an
    +                // empty transaction
    +                System.out.println("EOF reached after " + count + " txns.");
    +                // returning false because nothing was chopped
    +                return false;
    +            }
    +
    +            Checksum crc = new Adler32();
    +            crc.update(bytes, 0, bytes.length);
    +            if (crcValue != crc.getValue()) {
    +                throw new IOException("CRC doesn't match " + crcValue +
    +                        " vs " + crc.getValue());
    +            }
    +            TxnHeader hdr = new TxnHeader();
    +            Record txn = SerializeUtils.deserializeTxn(bytes, hdr);
    +            if (logStream.readByte("EOR") != 'B') {
    +                System.out.println("Last transaction was partial.");
    +                throw new EOFException("Last transaction was partial.");
    +            }
    +
    +            long txnZxid = hdr.getZxid();
    +            if (txnZxid == zxid) {
    +                hasZxid = true;
    +            }
    +
    +            // logging the gap to make the inconsistency investigation easier
    +            if (previousZxid != -1 && txnZxid != previousZxid + 1) {
    +                long txnEpoch = ZxidUtils.getEpochFromZxid(txnZxid);
    +                long txnCounter = ZxidUtils.getCounterFromZxid(txnZxid);
    +                long previousEpoch = ZxidUtils.getEpochFromZxid(previousZxid);
    +                if (txnEpoch == previousEpoch || txnCounter != 1) {
    +                    System.out.println(
    +                        String.format("There is gap between %x and %x",
    +                                previousZxid, txnZxid));
    +                }
    +            }
    +            previousZxid = txnZxid;
    +
    +            if (hdr.getZxid() > zxid) {
    --- End diff --
    
    I'd like to leave this check after the gap detection. This covers the one scenario where there is a gap in the log that the target zxid just happens to fall in. Otherwise the operator will only learn that the target is not in the log, not that/why the log is corrupt.


---

[GitHub] zookeeper pull request #615: ZOOKEEPER-3137: add a utility to truncate logs ...

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

    https://github.com/apache/zookeeper/pull/615#discussion_r215129287
  
    --- Diff: src/java/main/org/apache/zookeeper/server/util/LogChopper.java ---
    @@ -0,0 +1,152 @@
    +/**
    + * 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;
    +
    +import org.apache.jute.BinaryInputArchive;
    +import org.apache.jute.BinaryOutputArchive;
    +import org.apache.jute.Record;
    +import org.apache.zookeeper.server.persistence.FileHeader;
    +import org.apache.zookeeper.server.persistence.FileTxnLog;
    +import org.apache.zookeeper.txn.TxnHeader;
    +
    +import java.io.BufferedInputStream;
    +import java.io.BufferedOutputStream;
    +import java.io.EOFException;
    +import java.io.FileInputStream;
    +import java.io.FileOutputStream;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.io.OutputStream;
    +import java.util.zip.Adler32;
    +import java.util.zip.Checksum;
    +
    +/**
    + * this class will chop the log at the specified zxid
    + */
    +public class LogChopper {
    +    public static void main(String args[]) {
    +        if (args.length != 3) {
    +            System.out.println("Usage: LogChopper zxid_to_chop_to txn_log_to_chop chopped_filename");
    +            System.out.println("    this program will read the txn_log_to_chop file and copy all the transactions");
    +            System.out.println("    from it up to (and including) the given zxid into chopped_filename.");
    +            System.exit(1);
    +        }
    +        long zxid = Long.decode(args[0]);
    --- End diff --
    
    Should we move this into the following try block so it can catch parse exception if the zxid passed is not valid (e.g. not a number)?


---

[GitHub] zookeeper pull request #615: ZOOKEEPER-3137: add a utility to truncate logs ...

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

    https://github.com/apache/zookeeper/pull/615#discussion_r215129442
  
    --- Diff: src/java/main/org/apache/zookeeper/server/util/LogChopper.java ---
    @@ -0,0 +1,152 @@
    +/**
    + * 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;
    +
    +import org.apache.jute.BinaryInputArchive;
    +import org.apache.jute.BinaryOutputArchive;
    +import org.apache.jute.Record;
    +import org.apache.zookeeper.server.persistence.FileHeader;
    +import org.apache.zookeeper.server.persistence.FileTxnLog;
    +import org.apache.zookeeper.txn.TxnHeader;
    +
    +import java.io.BufferedInputStream;
    +import java.io.BufferedOutputStream;
    +import java.io.EOFException;
    +import java.io.FileInputStream;
    +import java.io.FileOutputStream;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.io.OutputStream;
    +import java.util.zip.Adler32;
    +import java.util.zip.Checksum;
    +
    +/**
    + * this class will chop the log at the specified zxid
    + */
    +public class LogChopper {
    +    public static void main(String args[]) {
    +        if (args.length != 3) {
    +            System.out.println("Usage: LogChopper zxid_to_chop_to txn_log_to_chop chopped_filename");
    +            System.out.println("    this program will read the txn_log_to_chop file and copy all the transactions");
    +            System.out.println("    from it up to (and including) the given zxid into chopped_filename.");
    +            System.exit(1);
    +        }
    +        long zxid = Long.decode(args[0]);
    +        String txnLog = args[1];
    +        String choppedLog = args[2];
    +
    +        int rc = 2;
    +        try (
    +            InputStream is = new BufferedInputStream(new FileInputStream(txnLog));
    +            OutputStream os = new BufferedOutputStream(new FileOutputStream(choppedLog))
    +        ) {
    +            if (chop(is, os, zxid)) {
    +                rc = 0;
    +            }
    +        } catch (Exception e) {
    +            System.out.println("Got exception: " + e.getMessage());
    +        }
    +        System.exit(rc);
    +    }
    +
    +    public static boolean chop(InputStream is, OutputStream os, long zxid) throws IOException {
    +        BinaryInputArchive logStream = BinaryInputArchive.getArchive(is);
    +        BinaryOutputArchive choppedStream = BinaryOutputArchive.getArchive(os);
    +        FileHeader fhdr = new FileHeader();
    +        fhdr.deserialize(logStream, "fileheader");
    +
    +        if (fhdr.getMagic() != FileTxnLog.TXNLOG_MAGIC) {
    +            System.err.println("Invalid magic number in txn log file");
    +            return false;
    +        }
    +        System.out.println("ZooKeeper Transactional Log File with dbid "
    +                + fhdr.getDbid() + " txnlog format version "
    +                + fhdr.getVersion());
    +
    +        fhdr.serialize(choppedStream, "fileheader");
    +        int count = 0;
    +        boolean hasZxid = false;
    +        long previousZxid = -1;
    +        while (true) {
    +            long crcValue;
    +            byte[] bytes;
    +            try {
    +                crcValue = logStream.readLong("crcvalue");
    +
    +                bytes = logStream.readBuffer("txnEntry");
    +            } catch (EOFException e) {
    +                System.out.println("EOF reached after " + count + " txns.");
    +                // returning false because nothing was chopped
    +                return false;
    +            }
    +            if (bytes.length == 0) {
    +                // Since we preallocate, we define EOF to be an
    +                // empty transaction
    +                System.out.println("EOF reached after " + count + " txns.");
    +                // returning false because nothing was chopped
    +                return false;
    +            }
    +
    +            Checksum crc = new Adler32();
    +            crc.update(bytes, 0, bytes.length);
    +            if (crcValue != crc.getValue()) {
    +                throw new IOException("CRC doesn't match " + crcValue +
    +                        " vs " + crc.getValue());
    +            }
    +            TxnHeader hdr = new TxnHeader();
    +            Record txn = SerializeUtils.deserializeTxn(bytes, hdr);
    +            if (logStream.readByte("EOR") != 'B') {
    +                System.out.println("Last transaction was partial.");
    +                throw new EOFException("Last transaction was partial.");
    +            }
    +
    +            long txnZxid = hdr.getZxid();
    +            if (txnZxid == zxid) {
    +                hasZxid = true;
    +            }
    +
    +            // logging the gap to make the inconsistency investigation easier
    +            if (previousZxid != -1 && txnZxid != previousZxid + 1) {
    +                long txnEpoch = ZxidUtils.getEpochFromZxid(txnZxid);
    +                long txnCounter = ZxidUtils.getCounterFromZxid(txnZxid);
    +                long previousEpoch = ZxidUtils.getEpochFromZxid(previousZxid);
    +                if (txnEpoch == previousEpoch || txnCounter != 1) {
    +                    System.out.println(
    +                        String.format("There is gap between %x and %x",
    +                                previousZxid, txnZxid));
    +                }
    +            }
    +            previousZxid = txnZxid;
    +
    +            if (hdr.getZxid() > zxid) {
    --- End diff --
    
    `hdr.getZxid()` is previously assigned to `txnZxid`, so maybe use `txnZxid` here instead? 
    
    Also, would it be better to move this check before the gap detection logic to terminate earlier?


---

[GitHub] zookeeper pull request #615: ZOOKEEPER-3137: add a utility to truncate logs ...

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

    https://github.com/apache/zookeeper/pull/615#discussion_r215129214
  
    --- Diff: src/java/main/org/apache/zookeeper/server/util/LogChopper.java ---
    @@ -0,0 +1,152 @@
    +/**
    + * 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;
    +
    +import org.apache.jute.BinaryInputArchive;
    +import org.apache.jute.BinaryOutputArchive;
    +import org.apache.jute.Record;
    +import org.apache.zookeeper.server.persistence.FileHeader;
    +import org.apache.zookeeper.server.persistence.FileTxnLog;
    +import org.apache.zookeeper.txn.TxnHeader;
    +
    +import java.io.BufferedInputStream;
    +import java.io.BufferedOutputStream;
    +import java.io.EOFException;
    +import java.io.FileInputStream;
    +import java.io.FileOutputStream;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.io.OutputStream;
    +import java.util.zip.Adler32;
    +import java.util.zip.Checksum;
    +
    +/**
    + * this class will chop the log at the specified zxid
    + */
    +public class LogChopper {
    +    public static void main(String args[]) {
    +        if (args.length != 3) {
    +            System.out.println("Usage: LogChopper zxid_to_chop_to txn_log_to_chop chopped_filename");
    +            System.out.println("    this program will read the txn_log_to_chop file and copy all the transactions");
    +            System.out.println("    from it up to (and including) the given zxid into chopped_filename.");
    +            System.exit(1);
    +        }
    +        long zxid = Long.decode(args[0]);
    +        String txnLog = args[1];
    +        String choppedLog = args[2];
    +
    +        int rc = 2;
    --- End diff --
    
    We have recently added `ExitCode` enum, maybe use those for exit code?


---

[GitHub] zookeeper issue #615: ZOOKEEPER-3137: add a utility to truncate logs to a zx...

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

    https://github.com/apache/zookeeper/pull/615
  
    @enixon Do you plan to update this PR with your trial integration branch, or you prefer the patch land as is? 


---

[GitHub] zookeeper issue #615: ZOOKEEPER-3137: add a utility to truncate logs to a zx...

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

    https://github.com/apache/zookeeper/pull/615
  
    @anmolnar , you can find a trial integration of the chop function with TxnLogToolkit at https://github.com/enixon/zookeeper/commit/b71ce04f70873f590c6cf5c24a691a1f1af82f48 . It's not the cleanest integration since there's not much direct overlap between the dump method and the chop method but I think we can make some useful decisions here that will help centralize work on the tool for future patches.
    
    Let me know what you think!


---

[GitHub] zookeeper pull request #615: ZOOKEEPER-3137: add a utility to truncate logs ...

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

    https://github.com/apache/zookeeper/pull/615#discussion_r215129999
  
    --- Diff: src/java/main/org/apache/zookeeper/server/util/LogChopper.java ---
    @@ -0,0 +1,152 @@
    +/**
    + * 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;
    +
    +import org.apache.jute.BinaryInputArchive;
    +import org.apache.jute.BinaryOutputArchive;
    +import org.apache.jute.Record;
    +import org.apache.zookeeper.server.persistence.FileHeader;
    +import org.apache.zookeeper.server.persistence.FileTxnLog;
    +import org.apache.zookeeper.txn.TxnHeader;
    +
    +import java.io.BufferedInputStream;
    +import java.io.BufferedOutputStream;
    +import java.io.EOFException;
    +import java.io.FileInputStream;
    +import java.io.FileOutputStream;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.io.OutputStream;
    +import java.util.zip.Adler32;
    +import java.util.zip.Checksum;
    +
    +/**
    + * this class will chop the log at the specified zxid
    + */
    +public class LogChopper {
    --- End diff --
    
    This is public, so it needs to be annotated with interface annotation.
    See https://github.com/apache/zookeeper/commit/0cb4011dac7ec28637426cafd98b4f8f299ef61d#diff-40373c146f7f620581c943009c8b796f


---