From 1c468c2346072a4d88aeec65dfadc7a120cdc43e Mon Sep 17 00:00:00 2001 From: Zhe Zhang Date: Fri, 23 Oct 2015 13:58:26 -0700 Subject: [PATCH] HDFS-8808. dfs.image.transfer.bandwidthPerSec should not apply to -bootstrapStandby. Contributed by Zhe Zhang. Conflicts: hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/BootstrapStandby.java Change-Id: I99c314bb932b96a25f9115772267176e0b815614 --- hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt | 3 + .../org/apache/hadoop/hdfs/DFSConfigKeys.java | 5 ++ .../hdfs/server/namenode/Checkpointer.java | 2 +- .../hdfs/server/namenode/ImageServlet.java | 38 +++++++++--- .../server/namenode/SecondaryNameNode.java | 2 +- .../hdfs/server/namenode/TransferFsImage.java | 5 +- .../server/namenode/ha/BootstrapStandby.java | 2 +- .../src/main/resources/hdfs-default.xml | 20 +++++- .../hdfs/server/namenode/TestCheckpoint.java | 2 +- .../namenode/ha/TestBootstrapStandby.java | 61 +++++++++++++++++++ 10 files changed, 124 insertions(+), 16 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt index 312b12e1008..b88e2ee28a1 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt @@ -727,6 +727,9 @@ Release 2.8.0 - UNRELEASED HDFS-9264. Minor cleanup of operations on FsVolumeList#volumes. (Walter Su via lei) + HDFS-8808. dfs.image.transfer.bandwidthPerSec should not apply to + -bootstrapStandby (zhz) + OPTIMIZATIONS HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java index 6a75ac36a77..380ec1ba02a 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java @@ -507,6 +507,11 @@ public class DFSConfigKeys extends CommonConfigurationKeys { "dfs.image.transfer.bandwidthPerSec"; public static final long DFS_IMAGE_TRANSFER_RATE_DEFAULT = 0; //no throttling + public static final String DFS_IMAGE_TRANSFER_BOOTSTRAP_STANDBY_RATE_KEY = + "dfs.image.transfer-bootstrap-standby.bandwidthPerSec"; + public static final long DFS_IMAGE_TRANSFER_BOOTSTRAP_STANDBY_RATE_DEFAULT = + 0; //no throttling + // Image transfer timeout public static final String DFS_IMAGE_TRANSFER_TIMEOUT_KEY = "dfs.image.transfer.timeout"; public static final int DFS_IMAGE_TRANSFER_TIMEOUT_DEFAULT = 60 * 1000; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Checkpointer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Checkpointer.java index 90876298540..83d835aca81 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Checkpointer.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Checkpointer.java @@ -222,7 +222,7 @@ class Checkpointer extends Daemon { "image with txid " + sig.mostRecentCheckpointTxId); MD5Hash downloadedHash = TransferFsImage.downloadImageToStorage( backupNode.nnHttpAddress, sig.mostRecentCheckpointTxId, bnStorage, - true); + true, false); bnImage.saveDigestAndRenameCheckpointImage(NameNodeFile.IMAGE, sig.mostRecentCheckpointTxId, downloadedHash); lastApplied = sig.mostRecentCheckpointTxId; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ImageServlet.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ImageServlet.java index 9f4b20cdebb..d8dfa54c391 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ImageServlet.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ImageServlet.java @@ -81,6 +81,7 @@ public class ImageServlet extends HttpServlet { private static final String STORAGEINFO_PARAM = "storageInfo"; private static final String LATEST_FSIMAGE_VALUE = "latest"; private static final String IMAGE_FILE_TYPE = "imageFile"; + private static final String IS_BOOTSTRAP_STANDBY = "bootstrapstandby"; @Override public void doGet(final HttpServletRequest request, @@ -153,8 +154,10 @@ public class ImageServlet extends HttpServlet { // detected by the client side as an inaccurate length header. } // send file + DataTransferThrottler throttler = parsedParams.isBootstrapStandby ? + getThrottlerForBootstrapStandby(conf) : getThrottler(conf); TransferFsImage.copyFileToStream(response.getOutputStream(), - file, fis, getThrottler(conf)); + file, fis, throttler); } finally { IOUtils.closeStream(fis); } @@ -211,8 +214,8 @@ public class ImageServlet extends HttpServlet { * @param conf configuration * @return a data transfer throttler */ - public final static DataTransferThrottler getThrottler(Configuration conf) { - long transferBandwidth = + public static DataTransferThrottler getThrottler(Configuration conf) { + long transferBandwidth = conf.getLong(DFSConfigKeys.DFS_IMAGE_TRANSFER_RATE_KEY, DFSConfigKeys.DFS_IMAGE_TRANSFER_RATE_DEFAULT); DataTransferThrottler throttler = null; @@ -221,7 +224,20 @@ public class ImageServlet extends HttpServlet { } return throttler; } - + + private static DataTransferThrottler getThrottlerForBootstrapStandby( + Configuration conf) { + long transferBandwidth = + conf.getLong( + DFSConfigKeys.DFS_IMAGE_TRANSFER_BOOTSTRAP_STANDBY_RATE_KEY, + DFSConfigKeys.DFS_IMAGE_TRANSFER_BOOTSTRAP_STANDBY_RATE_DEFAULT); + DataTransferThrottler throttler = null; + if (transferBandwidth > 0) { + throttler = new DataTransferThrottler(transferBandwidth); + } + return throttler; + } + @VisibleForTesting static boolean isValidRequestor(ServletContext context, String remoteUser, Configuration conf) throws IOException { @@ -295,13 +311,14 @@ public class ImageServlet extends HttpServlet { } static String getParamStringForImage(NameNodeFile nnf, long txid, - StorageInfo remoteStorageInfo) { + StorageInfo remoteStorageInfo, boolean isBootstrapStandby) { final String imageType = nnf == null ? "" : "&" + IMAGE_FILE_TYPE + "=" + nnf.name(); return "getimage=1&" + TXID_PARAM + "=" + txid + imageType - + "&" + STORAGEINFO_PARAM + "=" + - remoteStorageInfo.toColonSeparatedString(); + + "&" + STORAGEINFO_PARAM + "=" + + remoteStorageInfo.toColonSeparatedString() + "&" + + IS_BOOTSTRAP_STANDBY + "=" + isBootstrapStandby; } static String getParamStringForLog(RemoteEditLog log, @@ -319,6 +336,7 @@ public class ImageServlet extends HttpServlet { private long startTxId, endTxId, txId; private String storageInfoString; private boolean fetchLatest; + private boolean isBootstrapStandby; /** * @param request the object from which this servlet reads the url contents @@ -330,7 +348,7 @@ public class ImageServlet extends HttpServlet { ) throws IOException { @SuppressWarnings("unchecked") Map pmap = request.getParameterMap(); - isGetImage = isGetEdit = fetchLatest = false; + isGetImage = isGetEdit = fetchLatest = isBootstrapStandby = false; for (Map.Entry entry : pmap.entrySet()) { String key = entry.getKey(); @@ -342,6 +360,10 @@ public class ImageServlet extends HttpServlet { String imageType = ServletUtil.getParameter(request, IMAGE_FILE_TYPE); nnf = imageType == null ? NameNodeFile.IMAGE : NameNodeFile .valueOf(imageType); + String bootstrapStandby = ServletUtil.getParameter(request, + IS_BOOTSTRAP_STANDBY); + isBootstrapStandby = bootstrapStandby != null && + Boolean.parseBoolean(bootstrapStandby); } catch (NumberFormatException nfe) { if (request.getParameter(TXID_PARAM).equals(LATEST_FSIMAGE_VALUE)) { fetchLatest = true; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNode.java index 29e332b1eeb..9909d4add4d 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNode.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNode.java @@ -421,7 +421,7 @@ public class SecondaryNameNode implements Runnable, LOG.info("Image has changed. Downloading updated image from NN."); MD5Hash downloadedHash = TransferFsImage.downloadImageToStorage( nnHostPort, sig.mostRecentCheckpointTxId, - dstImage.getStorage(), true); + dstImage.getStorage(), true, false); dstImage.saveDigestAndRenameCheckpointImage(NameNodeFile.IMAGE, sig.mostRecentCheckpointTxId, downloadedHash); } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/TransferFsImage.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/TransferFsImage.java index f1b74f4bb5f..e6e75b0352c 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/TransferFsImage.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/TransferFsImage.java @@ -102,9 +102,10 @@ public class TransferFsImage { } public static MD5Hash downloadImageToStorage(URL fsName, long imageTxId, - Storage dstStorage, boolean needDigest) throws IOException { + Storage dstStorage, boolean needDigest, boolean isBootstrapStandby) + throws IOException { String fileid = ImageServlet.getParamStringForImage(null, - imageTxId, dstStorage); + imageTxId, dstStorage, isBootstrapStandby); String fileName = NNStorage.getCheckpointImageFileName(imageTxId); List dstFiles = dstStorage.getFiles( diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/BootstrapStandby.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/BootstrapStandby.java index 26fd9a0ba4b..031170822c6 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/BootstrapStandby.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/BootstrapStandby.java @@ -317,7 +317,7 @@ public class BootstrapStandby implements Tool, Configurable { // Download that checkpoint into our storage directories. MD5Hash hash = TransferFsImage.downloadImageToStorage( - otherHttpAddr, imageTxId, storage, true); + otherHttpAddr, imageTxId, storage, true, true); image.saveDigestAndRenameCheckpointImage(NameNodeFile.IMAGE, imageTxId, hash); } catch (IOException ioe) { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml index 594ea23f24f..d81acc149be 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml @@ -1061,15 +1061,31 @@ dfs.image.transfer.bandwidthPerSec 0 - Maximum bandwidth used for image transfer in bytes per second. + Maximum bandwidth used for regular image transfers (instead of + bootstrapping the standby namenode), in bytes per second. This can help keep normal namenode operations responsive during checkpointing. The maximum bandwidth and timeout in dfs.image.transfer.timeout should be set such that normal image transfers can complete successfully. - A default value of 0 indicates that throttling is disabled. + A default value of 0 indicates that throttling is disabled. + The maximum bandwidth used for bootstrapping standby namenode is + configured with dfs.image.transfer-bootstrap-standby.bandwidthPerSec. + + dfs.image.transfer-bootstrap-standby.bandwidthPerSec + 0 + + Maximum bandwidth used for transferring image to bootstrap standby + namenode, in bytes per second. + A default value of 0 indicates that throttling is disabled. This default + value should be used in most cases, to ensure timely HA operations. + The maximum bandwidth used for regular image transfers is configured + with dfs.image.transfer.bandwidthPerSec. + + + dfs.image.transfer.chunksize 65536 diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCheckpoint.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCheckpoint.java index 0e8f9b7bd9c..ad10919584c 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCheckpoint.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCheckpoint.java @@ -2004,7 +2004,7 @@ public class TestCheckpoint { .when(dstImage).toColonSeparatedString(); try { - TransferFsImage.downloadImageToStorage(fsName, 0, dstImage, false); + TransferFsImage.downloadImageToStorage(fsName, 0, dstImage, false, false); fail("Storage info was not verified"); } catch (IOException ioe) { String msg = StringUtils.stringifyException(ioe); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestBootstrapStandby.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestBootstrapStandby.java index 7abc5024a9b..a849233c49f 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestBootstrapStandby.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestBootstrapStandby.java @@ -24,11 +24,14 @@ import static org.junit.Assert.fail; import java.io.File; import java.io.IOException; import java.net.URI; +import java.util.concurrent.TimeoutException; +import com.google.common.base.Supplier; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileUtil; +import org.apache.hadoop.hdfs.DFSConfigKeys; import org.apache.hadoop.hdfs.MiniDFSCluster; import org.apache.hadoop.hdfs.MiniDFSNNTopology; import org.apache.hadoop.hdfs.server.namenode.CheckpointSignature; @@ -205,6 +208,64 @@ public class TestBootstrapStandby { assertEquals(0, rc); } + /** + * Test that bootstrapping standby NN is not limited by + * {@link DFSConfigKeys#DFS_IMAGE_TRANSFER_RATE_KEY}, but is limited by + * {@link DFSConfigKeys#DFS_IMAGE_TRANSFER_BOOTSTRAP_STANDBY_RATE_KEY} + * created by HDFS-8808. + */ + @Test + public void testRateThrottling() throws Exception { + cluster.getConfiguration(0).setLong( + DFSConfigKeys.DFS_IMAGE_TRANSFER_RATE_KEY, 1); + cluster.restartNameNode(0); + cluster.waitActive(); + nn0 = cluster.getNameNode(0); + cluster.transitionToActive(0); + // Each edit has at least 1 byte. So the lowRate definitely should cause + // a timeout, if enforced. If lowRate is not enforced, any reasonable test + // machine should at least download an image with 5 edits in 5 seconds. + for (int i = 0; i < 5; i++) { + nn0.getRpcServer().rollEditLog(); + } + // A very low DFS_IMAGE_TRANSFER_RATE_KEY value won't affect bootstrapping + GenericTestUtils.waitFor(new Supplier() { + public Boolean get() { + try { + testSuccessfulBaseCase(); + return true; + } catch (Exception e) { + return false; + } + } + }, 500, 5000); + + shutdownCluster(); + setupCluster(); + cluster.getConfiguration(0).setLong( + DFSConfigKeys.DFS_IMAGE_TRANSFER_BOOTSTRAP_STANDBY_RATE_KEY, 1); + cluster.restartNameNode(0); + cluster.waitActive(); + nn0 = cluster.getNameNode(0); + cluster.transitionToActive(0); + // A very low DFS_IMAGE_TRANSFER_BOOTSTRAP_STANDBY_RATE_KEY value should + // cause timeout + try { + GenericTestUtils.waitFor(new Supplier() { + public Boolean get() { + try { + testSuccessfulBaseCase(); + return true; + } catch (Exception e) { + return false; + } + } + }, 500, 5000); + fail("Did not timeout"); + } catch (TimeoutException e) { + LOG.info("Encountered expected timeout."); + } + } private void removeStandbyNameDirs() { for (URI u : cluster.getNameDirs(1)) { assertTrue(u.getScheme().equals("file"));