From commits-return-6130-archive-asf-public=cust-asf.ponee.io@zookeeper.apache.org Fri Jan 19 00:48:26 2018 Return-Path: X-Original-To: archive-asf-public@eu.ponee.io Delivered-To: archive-asf-public@eu.ponee.io Received: from cust-asf.ponee.io (cust-asf.ponee.io [163.172.22.183]) by mx-eu-01.ponee.io (Postfix) with ESMTP id D0F2E180654 for ; Fri, 19 Jan 2018 00:48:26 +0100 (CET) Received: by cust-asf.ponee.io (Postfix) id C0B43160C48; Thu, 18 Jan 2018 23:48:26 +0000 (UTC) Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by cust-asf.ponee.io (Postfix) with SMTP id BA012160C26 for ; Fri, 19 Jan 2018 00:48:25 +0100 (CET) Received: (qmail 24205 invoked by uid 500); 18 Jan 2018 23:48:24 -0000 Mailing-List: contact commits-help@zookeeper.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@zookeeper.apache.org Delivered-To: mailing list commits@zookeeper.apache.org Received: (qmail 24189 invoked by uid 99); 18 Jan 2018 23:48:24 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Thu, 18 Jan 2018 23:48:24 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id C2FC1DFD7B; Thu, 18 Jan 2018 23:48:24 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: phunt@apache.org To: commits@zookeeper.apache.org Message-Id: X-Mailer: ASF-Git Admin Mailer Subject: zookeeper git commit: ZOOKEEPER-2249: CRC check failed when preAllocSize smaller than node data Date: Thu, 18 Jan 2018 23:48:24 +0000 (UTC) Repository: zookeeper Updated Branches: refs/heads/branch-3.4 fd25ee3d2 -> 3fcf7bb46 ZOOKEEPER-2249: CRC check failed when preAllocSize smaller than node data This bug is caused by an issue in our file padding logic. We calculate an incorrect position to add padding when appending to the transaction log, this often corrupts a transaction. When the log is read the CRC check will correctly fail. Author: Abraham Fine Reviewers: phunt@apache.org Closes #436 from afine/ZOOKEEPER-2249 and squashes the following commits: f3bc06c1 [Abraham Fine] Improve testing and respond to phunt's comment 44ad0575 [Abraham Fine] ZOOKEEPER-2249: CRC check failed when preAllocSize smaller than node data Change-Id: Id8ba9ad730760cb78672127b8c0e02db60b4e87d (cherry picked from commit 4d629d289fc4b1079fff2244b92a337e4068e240) Signed-off-by: Patrick Hunt Project: http://git-wip-us.apache.org/repos/asf/zookeeper/repo Commit: http://git-wip-us.apache.org/repos/asf/zookeeper/commit/3fcf7bb4 Tree: http://git-wip-us.apache.org/repos/asf/zookeeper/tree/3fcf7bb4 Diff: http://git-wip-us.apache.org/repos/asf/zookeeper/diff/3fcf7bb4 Branch: refs/heads/branch-3.4 Commit: 3fcf7bb46223f1c53e84f5559b7a7751ef9c2322 Parents: fd25ee3 Author: Abraham Fine Authored: Thu Jan 18 15:43:11 2018 -0800 Committer: Patrick Hunt Committed: Thu Jan 18 15:46:51 2018 -0800 ---------------------------------------------------------------------- .../server/persistence/FileTxnLog.java | 46 ++++++++- .../zookeeper/server/persistence/Util.java | 25 ----- .../server/persistence/FileTxnLogTest.java | 101 +++++++++++++++++++ 3 files changed, 142 insertions(+), 30 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/zookeeper/blob/3fcf7bb4/src/java/main/org/apache/zookeeper/server/persistence/FileTxnLog.java ---------------------------------------------------------------------- diff --git a/src/java/main/org/apache/zookeeper/server/persistence/FileTxnLog.java b/src/java/main/org/apache/zookeeper/server/persistence/FileTxnLog.java index 724b855..0a70ea0 100644 --- a/src/java/main/org/apache/zookeeper/server/persistence/FileTxnLog.java +++ b/src/java/main/org/apache/zookeeper/server/persistence/FileTxnLog.java @@ -28,6 +28,7 @@ import java.io.IOException; import java.io.InputStream; import java.io.RandomAccessFile; import java.nio.ByteBuffer; +import java.nio.channels.FileChannel; import java.util.ArrayList; import java.util.LinkedList; import java.util.List; @@ -91,6 +92,7 @@ public class FileTxnLog implements TxnLog { private static final Logger LOG; static long preAllocSize = 65536 * 1024; + private static final ByteBuffer fill = ByteBuffer.allocateDirect(1); public final static int TXNLOG_MAGIC = ByteBuffer.wrap("ZKLG".getBytes()).getInt(); @@ -222,7 +224,7 @@ public class FileTxnLog implements TxnLog { currentSize = fos.getChannel().position(); streamsToFlush.add(fos); } - padFile(fos); + currentSize = padFile(fos.getChannel()); byte[] buf = Util.marshallTxnEntry(hdr, txn); if (buf == null || buf.length == 0) { throw new IOException("Faulty serialization for header " + @@ -237,12 +239,46 @@ public class FileTxnLog implements TxnLog { } /** - * pad the current file to increase its size - * @param out the outputstream to be padded + * pad the current file to increase its size to the next multiple of preAllocSize greater than the current size and position + * @param fileChannel the fileChannel of the file to be padded * @throws IOException */ - private void padFile(FileOutputStream out) throws IOException { - currentSize = Util.padLogFile(out, currentSize, preAllocSize); + private long padFile(FileChannel fileChannel) throws IOException { + long newFileSize = calculateFileSizeWithPadding(fileChannel.position(), currentSize, preAllocSize); + if (currentSize != newFileSize) { + fileChannel.write((ByteBuffer) fill.position(0), newFileSize - fill.remaining()); + currentSize = newFileSize; + } + return currentSize; + } + + /** + * Calculates a new file size with padding. We only return a new size if + * the current file position is sufficiently close (less than 4K) to end of + * file and preAllocSize is > 0. + * + * @param position the point in the file we have written to + * @param fileSize application keeps track of the current file size + * @param preAllocSize how many bytes to pad + * @return the new file size. It can be the same as fileSize if no + * padding was done. + * @throws IOException + */ + // VisibleForTesting + public static long calculateFileSizeWithPadding(long position, long fileSize, long preAllocSize) { + // If preAllocSize is positive and we are within 4KB of the known end of the file calculate a new file size + if (preAllocSize > 0 && position + 4096 >= fileSize) { + // If we have written more than we have previously preallocated we need to make sure the new + // file size is larger than what we already have + if (position > fileSize){ + fileSize = position + preAllocSize; + fileSize -= fileSize % preAllocSize; + } else { + fileSize += preAllocSize; + } + } + + return fileSize; } /** http://git-wip-us.apache.org/repos/asf/zookeeper/blob/3fcf7bb4/src/java/main/org/apache/zookeeper/server/persistence/Util.java ---------------------------------------------------------------------- diff --git a/src/java/main/org/apache/zookeeper/server/persistence/Util.java b/src/java/main/org/apache/zookeeper/server/persistence/Util.java index 7ef7f9c..656c2e8 100644 --- a/src/java/main/org/apache/zookeeper/server/persistence/Util.java +++ b/src/java/main/org/apache/zookeeper/server/persistence/Util.java @@ -21,7 +21,6 @@ package org.apache.zookeeper.server.persistence; import java.io.ByteArrayOutputStream; import java.io.EOFException; import java.io.File; -import java.io.FileOutputStream; import java.io.IOException; import java.io.RandomAccessFile; import java.io.Serializable; @@ -51,7 +50,6 @@ public class Util { private static final String SNAP_DIR="snapDir"; private static final String LOG_DIR="logDir"; private static final String DB_FORMAT_CONV="dbFormatConversion"; - private static final ByteBuffer fill = ByteBuffer.allocateDirect(1); public static String makeURIString(String dataDir, String dataLogDir, String convPolicy){ @@ -199,29 +197,6 @@ public class Util { } /** - * Grows the file to the specified number of bytes. This only happenes if - * the current file position is sufficiently close (less than 4K) to end of - * file. - * - * @param f output stream to pad - * @param currentSize application keeps track of the cuurent file size - * @param preAllocSize how many bytes to pad - * @return the new file size. It can be the same as currentSize if no - * padding was done. - * @throws IOException - */ - public static long padLogFile(FileOutputStream f,long currentSize, - long preAllocSize) throws IOException{ - long position = f.getChannel().position(); - if (position + 4096 >= currentSize) { - currentSize = currentSize + preAllocSize; - fill.position(0); - f.getChannel().write(fill, currentSize-fill.remaining()); - } - return currentSize; - } - - /** * Reads a transaction entry from the input archive. * @param ia archive to read from * @return null if the entry is corrupted or EOF has been reached; a buffer http://git-wip-us.apache.org/repos/asf/zookeeper/blob/3fcf7bb4/src/java/test/org/apache/zookeeper/server/persistence/FileTxnLogTest.java ---------------------------------------------------------------------- diff --git a/src/java/test/org/apache/zookeeper/server/persistence/FileTxnLogTest.java b/src/java/test/org/apache/zookeeper/server/persistence/FileTxnLogTest.java new file mode 100644 index 0000000..5f54d0e --- /dev/null +++ b/src/java/test/org/apache/zookeeper/server/persistence/FileTxnLogTest.java @@ -0,0 +1,101 @@ +/** + * 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.persistence; + +import org.apache.zookeeper.ZKTestCase; +import org.apache.zookeeper.ZooDefs; +import org.apache.zookeeper.test.ClientBase; +import org.apache.zookeeper.txn.CreateTxn; +import org.apache.zookeeper.txn.TxnHeader; +import org.junit.Assert; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.io.IOException; +import java.util.Arrays; + +public class FileTxnLogTest extends ZKTestCase { + protected static final Logger LOG = LoggerFactory.getLogger(FileTxnLogTest.class); + + private static final int KB = 1024; + + @Test + public void testInvalidPreallocSize() { + Assert.assertEquals("file should not be padded", + 10 * KB, FileTxnLog.calculateFileSizeWithPadding(7 * KB, 10 * KB, 0)); + Assert.assertEquals("file should not be padded", + 10 * KB, FileTxnLog.calculateFileSizeWithPadding(7 * KB, 10 * KB, -1)); + } + + @Test + public void testCalculateFileSizeWithPaddingWhenNotToCurrentSize() { + Assert.assertEquals("file should not be padded", + 10 * KB, FileTxnLog.calculateFileSizeWithPadding(5 * KB, 10 * KB, 10 * KB)); + } + + @Test + public void testCalculateFileSizeWithPaddingWhenCloseToCurrentSize() { + Assert.assertEquals("file should be padded an additional 10 KB", + 20 * KB, FileTxnLog.calculateFileSizeWithPadding(7 * KB, 10 * KB, 10 * KB)); + } + + @Test + public void testFileSizeGreaterThanPosition() { + Assert.assertEquals("file should be padded to 40 KB", + 40 * KB, FileTxnLog.calculateFileSizeWithPadding(31 * KB, 10 * KB, 10 * KB)); + } + + @Test + public void testPreAllocSizeSmallerThanTxnData() throws IOException { + File logDir = ClientBase.createTmpDir(); + FileTxnLog fileTxnLog = new FileTxnLog(logDir); + + // Set a small preAllocSize (.5 MB) + final int preAllocSize = 500 * KB; + fileTxnLog.setPreallocSize(preAllocSize); + + // Create dummy txn larger than preAllocSize + // Since the file padding inserts a 0, we will fill the data with 0xff to ensure we corrupt the data if we put the 0 in the data + byte[] data = new byte[2 * preAllocSize]; + Arrays.fill(data, (byte) 0xff); + + // Append and commit 2 transactions to the log + // Prior to ZOOKEEPER-2249, attempting to pad in association with the second transaction will corrupt the first + fileTxnLog.append(new TxnHeader(1, 1, 1, 1, ZooDefs.OpCode.create), + new CreateTxn("/testPreAllocSizeSmallerThanTxnData1", data, ZooDefs.Ids.OPEN_ACL_UNSAFE, false, 0)); + fileTxnLog.commit(); + fileTxnLog.append(new TxnHeader(1, 1, 2, 2, ZooDefs.OpCode.create), + new CreateTxn("/testPreAllocSizeSmallerThanTxnData2", new byte[]{}, ZooDefs.Ids.OPEN_ACL_UNSAFE, false, 0)); + fileTxnLog.commit(); + fileTxnLog.close(); + + // Read the log back from disk, this will throw a java.io.IOException: CRC check failed prior to ZOOKEEPER-2249 + FileTxnLog.FileTxnIterator fileTxnIterator = new FileTxnLog.FileTxnIterator(logDir, 0); + + // Verify the data in the first transaction + CreateTxn createTxn = (CreateTxn) fileTxnIterator.getTxn(); + Assert.assertTrue(Arrays.equals(createTxn.getData(), data)); + + // Verify the data in the second transaction + fileTxnIterator.next(); + createTxn = (CreateTxn) fileTxnIterator.getTxn(); + Assert.assertTrue(Arrays.equals(createTxn.getData(), new byte[]{})); + } +}