Return-Path: X-Original-To: apmail-hadoop-hdfs-commits-archive@minotaur.apache.org Delivered-To: apmail-hadoop-hdfs-commits-archive@minotaur.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id 4CE2110961 for ; Wed, 14 Aug 2013 23:32:52 +0000 (UTC) Received: (qmail 79523 invoked by uid 500); 14 Aug 2013 23:32:52 -0000 Delivered-To: apmail-hadoop-hdfs-commits-archive@hadoop.apache.org Received: (qmail 79434 invoked by uid 500); 14 Aug 2013 23:32:52 -0000 Mailing-List: contact hdfs-commits-help@hadoop.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: hdfs-dev@hadoop.apache.org Delivered-To: mailing list hdfs-commits@hadoop.apache.org Received: (qmail 79420 invoked by uid 99); 14 Aug 2013 23:32:52 -0000 Received: from athena.apache.org (HELO athena.apache.org) (140.211.11.136) by apache.org (qpsmtpd/0.29) with ESMTP; Wed, 14 Aug 2013 23:32:52 +0000 X-ASF-Spam-Status: No, hits=-2000.0 required=5.0 tests=ALL_TRUSTED X-Spam-Check-By: apache.org Received: from [140.211.11.4] (HELO eris.apache.org) (140.211.11.4) by apache.org (qpsmtpd/0.29) with ESMTP; Wed, 14 Aug 2013 23:32:50 +0000 Received: from eris.apache.org (localhost [127.0.0.1]) by eris.apache.org (Postfix) with ESMTP id 9011B23888CD; Wed, 14 Aug 2013 23:32:30 +0000 (UTC) Content-Type: text/plain; charset="utf-8" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit Subject: svn commit: r1514095 - in /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs: ./ src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/ src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/ Date: Wed, 14 Aug 2013 23:32:30 -0000 To: hdfs-commits@hadoop.apache.org From: wang@apache.org X-Mailer: svnmailer-1.0.9 Message-Id: <20130814233230.9011B23888CD@eris.apache.org> X-Virus-Checked: Checked by ClamAV on apache.org Author: wang Date: Wed Aug 14 23:32:29 2013 New Revision: 1514095 URL: http://svn.apache.org/r1514095 Log: HDFS-4816. transitionToActive blocks if the SBN is doing checkpoint image transfer. (Andrew Wang) Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/StandbyCheckpointer.java hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestStandbyCheckpoints.java Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt?rev=1514095&r1=1514094&r2=1514095&view=diff ============================================================================== --- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt (original) +++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt Wed Aug 14 23:32:29 2013 @@ -267,6 +267,9 @@ Release 2.3.0 - UNRELEASED HDFS-5065. TestSymlinkHdfsDisable fails on Windows. (ivanmi) + HDFS-4816. transitionToActive blocks if the SBN is doing checkpoint image + transfer. (Andrew Wang) + Release 2.1.1-beta - UNRELEASED INCOMPATIBLE CHANGES Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/StandbyCheckpointer.java URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/StandbyCheckpointer.java?rev=1514095&r1=1514094&r2=1514095&view=diff ============================================================================== --- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/StandbyCheckpointer.java (original) +++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/StandbyCheckpointer.java Wed Aug 14 23:32:29 2013 @@ -17,9 +17,17 @@ */ package org.apache.hadoop.hdfs.server.namenode.ha; +import static org.apache.hadoop.util.Time.now; + import java.io.IOException; import java.net.InetSocketAddress; import java.security.PrivilegedAction; +import java.util.concurrent.Callable; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; +import java.util.concurrent.ThreadFactory; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -38,10 +46,10 @@ import org.apache.hadoop.hdfs.util.Cance import org.apache.hadoop.net.NetUtils; import org.apache.hadoop.security.SecurityUtil; import org.apache.hadoop.security.UserGroupInformation; -import static org.apache.hadoop.util.Time.now; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; +import com.google.common.util.concurrent.ThreadFactoryBuilder; /** * Thread which runs inside the NN when it's in Standby state, @@ -57,6 +65,7 @@ public class StandbyCheckpointer { private final FSNamesystem namesystem; private long lastCheckpointTime; private final CheckpointerThread thread; + private final ThreadFactory uploadThreadFactory; private String activeNNAddress; private InetSocketAddress myNNAddress; @@ -72,6 +81,8 @@ public class StandbyCheckpointer { this.namesystem = ns; this.checkpointConf = new CheckpointConf(conf); this.thread = new CheckpointerThread(); + this.uploadThreadFactory = new ThreadFactoryBuilder().setDaemon(true) + .setNameFormat("TransferFsImageUpload-%d").build(); setNameNodeAddresses(conf); } @@ -142,7 +153,7 @@ public class StandbyCheckpointer { private void doCheckpoint() throws InterruptedException, IOException { assert canceler != null; - long txid; + final long txid; namesystem.writeLockInterruptibly(); try { @@ -171,9 +182,26 @@ public class StandbyCheckpointer { } // Upload the saved checkpoint back to the active - TransferFsImage.uploadImageFromStorage( - activeNNAddress, myNNAddress, - namesystem.getFSImage().getStorage(), txid); + // Do this in a separate thread to avoid blocking transition to active + // See HDFS-4816 + ExecutorService executor = + Executors.newSingleThreadExecutor(uploadThreadFactory); + Future upload = executor.submit(new Callable() { + @Override + public Void call() throws IOException { + TransferFsImage.uploadImageFromStorage( + activeNNAddress, myNNAddress, + namesystem.getFSImage().getStorage(), txid); + return null; + } + }); + executor.shutdown(); + try { + upload.get(); + } catch (ExecutionException e) { + throw new IOException("Exception during image upload: " + e.getMessage(), + e.getCause()); + } } /** @@ -301,6 +329,7 @@ public class StandbyCheckpointer { LOG.info("Checkpoint was cancelled: " + ce.getMessage()); canceledCount++; } catch (InterruptedException ie) { + LOG.info("Interrupted during checkpointing", ie); // Probably requested shutdown. continue; } catch (Throwable t) { Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestStandbyCheckpoints.java URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestStandbyCheckpoints.java?rev=1514095&r1=1514094&r2=1514095&view=diff ============================================================================== --- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestStandbyCheckpoints.java (original) +++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestStandbyCheckpoints.java Wed Aug 14 23:32:29 2013 @@ -239,6 +239,34 @@ public class TestStandbyCheckpoints { assertTrue(canceledOne); } + + /** + * Test cancellation of ongoing checkpoints when failover happens + * mid-checkpoint during image upload from standby to active NN. + */ + @Test(timeout=60000) + public void testCheckpointCancellationDuringUpload() throws Exception { + // don't compress, we want a big image + cluster.getConfiguration(0).setBoolean( + DFSConfigKeys.DFS_IMAGE_COMPRESS_KEY, false); + cluster.getConfiguration(1).setBoolean( + DFSConfigKeys.DFS_IMAGE_COMPRESS_KEY, false); + // Throttle SBN upload to make it hang during upload to ANN + cluster.getConfiguration(1).setLong( + DFSConfigKeys.DFS_IMAGE_TRANSFER_RATE_KEY, 100); + cluster.restartNameNode(0); + cluster.restartNameNode(1); + nn0 = cluster.getNameNode(0); + nn1 = cluster.getNameNode(1); + + cluster.transitionToActive(0); + + doEdits(0, 100); + HATestUtil.waitForStandbyToCatchUp(nn0, nn1); + HATestUtil.waitForCheckpoint(cluster, 1, ImmutableList.of(104)); + cluster.transitionToStandby(0); + cluster.transitionToActive(1); + } /** * Make sure that clients will receive StandbyExceptions even when a