From common-commits-return-77573-archive-asf-public=cust-asf.ponee.io@hadoop.apache.org Sat Jan 13 02:03:45 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 39385180621 for ; Sat, 13 Jan 2018 02:03:45 +0100 (CET) Received: by cust-asf.ponee.io (Postfix) id 29045160C42; Sat, 13 Jan 2018 01:03:45 +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 4855A160C20 for ; Sat, 13 Jan 2018 02:03:44 +0100 (CET) Received: (qmail 15254 invoked by uid 500); 13 Jan 2018 01:03:43 -0000 Mailing-List: contact common-commits-help@hadoop.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Delivered-To: mailing list common-commits@hadoop.apache.org Received: (qmail 15242 invoked by uid 99); 13 Jan 2018 01:03:43 -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; Sat, 13 Jan 2018 01:03:43 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 28412DFB0D; Sat, 13 Jan 2018 01:03:43 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: weichiu@apache.org To: common-commits@hadoop.apache.org Message-Id: <6d889a0a902043c79c408779b26f1898@git.apache.org> X-Mailer: ASF-Git Admin Mailer Subject: hadoop git commit: HDFS-6804. Add test for race condition between transferring block and appending block causes "Unexpected checksum mismatch exception". Contributed by Brahma Reddy Battula. Date: Sat, 13 Jan 2018 01:03:43 +0000 (UTC) Repository: hadoop Updated Branches: refs/heads/branch-2.8 1b3f3b19b -> a6c0c2ced HDFS-6804. Add test for race condition between transferring block and appending block causes "Unexpected checksum mismatch exception". Contributed by Brahma Reddy Battula. Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/a6c0c2ce Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/a6c0c2ce Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/a6c0c2ce Branch: refs/heads/branch-2.8 Commit: a6c0c2ced54fca0a7dbe546eedcc0cbd9efc18bd Parents: 1b3f3b1 Author: Wei-Chiu Chuang Authored: Fri Jan 12 17:02:51 2018 -0800 Committer: Wei-Chiu Chuang Committed: Fri Jan 12 17:02:51 2018 -0800 ---------------------------------------------------------------------- .../hdfs/server/datanode/BlockSender.java | 4 +- .../server/datanode/DataNodeFaultInjector.java | 4 ++ .../hdfs/TestAppendDifferentChecksum.java | 72 +++++++++++++++++++- 3 files changed, 78 insertions(+), 2 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/hadoop/blob/a6c0c2ce/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockSender.java ---------------------------------------------------------------------- diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockSender.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockSender.java index de9c7e4..ec16e94 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockSender.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockSender.java @@ -327,7 +327,9 @@ class BlockSender implements java.io.Closeable { metaIn.getLength() >= BlockMetadataHeader.getHeaderSize()) { checksumIn = new DataInputStream(new BufferedInputStream( metaIn, IO_FILE_BUFFER_SIZE)); - + // HDFS-11160/HDFS-11056 + DataNodeFaultInjector.get() + .waitForBlockSenderMetaInputStreamBeforeAppend(); csum = BlockMetadataHeader.readDataChecksum(checksumIn, block); keepMetaInOpen = true; } http://git-wip-us.apache.org/repos/asf/hadoop/blob/a6c0c2ce/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNodeFaultInjector.java ---------------------------------------------------------------------- diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNodeFaultInjector.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNodeFaultInjector.java index c271124..2b8f2f6 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNodeFaultInjector.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNodeFaultInjector.java @@ -62,4 +62,8 @@ public class DataNodeFaultInjector { public void throwTooManyOpenFiles() throws FileNotFoundException { } + + public void waitForBlockSenderMetaInputStreamBeforeAppend() + throws IOException { + } } http://git-wip-us.apache.org/repos/asf/hadoop/blob/a6c0c2ce/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestAppendDifferentChecksum.java ---------------------------------------------------------------------- diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestAppendDifferentChecksum.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestAppendDifferentChecksum.java index eedbdb9..d961eb7 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestAppendDifferentChecksum.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestAppendDifferentChecksum.java @@ -19,14 +19,23 @@ package org.apache.hadoop.hdfs; import java.io.IOException; import java.util.Random; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; +import com.google.common.base.Supplier; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hdfs.protocol.ExtendedBlock; +import org.apache.hadoop.hdfs.server.datanode.DataNode; +import org.apache.hadoop.hdfs.server.datanode.DataNodeFaultInjector; import org.apache.hadoop.io.IOUtils; +import org.apache.hadoop.test.GenericTestUtils; +import org.apache.hadoop.test.GenericTestUtils.LogCapturer; import org.apache.hadoop.util.Time; import org.junit.AfterClass; +import org.junit.Assert; import org.junit.BeforeClass; import org.junit.Ignore; import org.junit.Test; @@ -130,7 +139,68 @@ public class TestAppendDifferentChecksum { AppendTestUtil.check(fsWithCrc32, p, len); AppendTestUtil.check(fsWithCrc32C, p, len); } - + + @Test(timeout = 60000) + public void testChecksumErrorAppendWhileTransfer() + throws Exception { + DataNodeFaultInjector oldFi = DataNodeFaultInjector.get(); + LogCapturer logCapturer = + GenericTestUtils.LogCapturer.captureLogs(DataNode.LOG); + try { + Path f = new Path("/f"); + FSDataOutputStream o = + fs.create(f, false, 1024, (short) 1, 128 * 1024 * 1024); + try { + AppendTestUtil.write(o, 0, 64 * 1024 + 600); + } finally { + o.close(); + } + final CountDownLatch latch = new CountDownLatch(2); + DataNodeFaultInjector.set(new DataNodeFaultInjector() { + public void waitForBlockSenderMetaInputStreamBeforeAppend() + throws IOException { + latch.countDown(); + try { + latch.await(20, TimeUnit.SECONDS); + } catch (InterruptedException ignored) { + } + } + }); + cluster.startDataNodes(cluster.getConfiguration(0), 1, true, null, null); + fs.setReplication(f, (short) 2); + // STEP 1: Wait till the BlockSender creates the meta input stream and then + // append to same file. + while (latch.getCount() > 1) { + Thread.sleep(100); + } + o = fs.append(f); + try { + AppendTestUtil.write(o, 0, 1); + o.hflush(); + latch.countDown(); + // STEP 2: Wait till the transfer happens. + final ExtendedBlock b = cluster.getFileSystem().getClient() + .getLocatedBlocks(f.toString(), 0).get(0).getBlock(); + GenericTestUtils.waitFor(new Supplier() { + @Override public Boolean get() { + return cluster.getDataNodes().get(1).getFSDataset().contains(b); + } + }, 100, 600); + + } finally { + o.close(); + } + } finally { + DataNodeFaultInjector.set(oldFi); + String logs = logCapturer.getOutput(); + logCapturer.stopCapturing(); + Assert.assertFalse("There should not be any checkum exception thrown", + logs.contains("ChecksumException")); + } + } + + + private FileSystem createFsWithChecksum(String type, int bytes) throws IOException { Configuration conf = new Configuration(fs.getConf()); --------------------------------------------------------------------- To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org For additional commands, e-mail: common-commits-help@hadoop.apache.org